You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by janewangfb <gi...@git.apache.org> on 2017/08/17 18:31:39 UTC

[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

GitHub user janewangfb opened a pull request:

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

    [SPARK-4131] Support "Writing data into the filesystem from queries"

    ## What changes were proposed in this pull request?
    
    This PR implements the sql feature:
    INSERT OVERWRITE [LOCAL] DIRECTORY directory1
      [ROW FORMAT row_format] [STORED AS file_format] 
      SELECT ... FROM ...
    
    ## How was this patch tested?
    Added new unittests and also pulled the code to fb-spark so that we could test writing to hdfs directory.
    


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

    $ git pull https://github.com/janewangfb/spark port_local_directory

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

    https://github.com/apache/spark/pull/18975.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 #18975
    
----
commit 6ca7771a39fe7c6ca5f82acc7d76dff00238de68
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-17T01:32:23Z

    add insert overwrite local directory

commit a975536ea3fed0674410ebc65538e134992779b1
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-17T03:59:25Z

    Add Unittests

commit a15bf4e0c12a37127788528f1e69b0462a0e287b
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-17T05:23:34Z

    fix local path

commit 9f596fda9dccdb0d5541340e8d9f2bf83573066a
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-17T18:25:04Z

    Merge branch 'master' into port_local_directory

----


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137607929
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,81 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + *
    + * @param storage storage format used to describe how the query result is stored.
    + * @param provider the data source type to be used
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: String,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def innerChildren: Seq[LogicalPlan] = Seq(query)
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    --- End diff --
    
    updated


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81576 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81576/testReport)** for PR 18975 at commit [`7919041`](https://github.com/apache/spark/commit/79190414db6a825f4489cb71a6050a4c3aa0fd46).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137385849
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -178,11 +179,60 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   * Parameters used for writing query to a table:
    +   *   (tableIdentifier, partitionKeys, overwrite, exists).
    +   */
    +  type InsertTableParams = (TableIdentifier, Map[String, Option[String]], Boolean, Boolean)
    +
    +  /**
    +   * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider).
    +   */
    +  type InsertDirParams = (Boolean, CatalogStorageFormat, Option[String])
    +
    +  /**
    +   * Add an
    +   *   INSERT INTO [TABLE] or
    +   *   INSERT OVERWRITE TABLE or
    +   *   INSERT OVERWRITE [LOCAL] DIRECTORY
    --- End diff --
    
    Let us put the complete syntax here.
    ```
    {{{
      INSERT OVERWRITE TABLE tableIdentifier [partitionSpec [IF NOT EXISTS]]?                           
      INSERT INTO [TABLE] tableIdentifier [partitionSpec]                                                  
      INSERT OVERWRITE [LOCAL] DIRECTORY path=STRING [rowFormat] [createFileFormat]                        
      INSERT OVERWRITE [LOCAL] DIRECTORY [path=STRING] tableProvider [OPTIONS options=tablePropertyList]
    }}}
    ```



---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137398917
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala ---
    @@ -360,6 +360,27 @@ case class InsertIntoTable(
     }
     
     /**
    + * Insert query result into a directory.
    --- End diff --
    
    added


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136516892
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,73 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + *
    + * @param storage storage format used to describe how the query result is stored.
    + * @param provider the data source type to be used
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: String,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def innerChildren: Seq[LogicalPlan] = Seq(query)
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    assert(innerChildren.length == 1)
    +    assert(storage.locationUri.nonEmpty, "Directory path is required")
    +    assert(!provider.isEmpty, "Data source is required")
    +
    +    // Create the relation based on the input logical plan: `query`.
    +    val pathOption = storage.locationUri.map("path" -> CatalogUtils.URIToString(_))
    +    val dataSource = DataSource(
    +      sparkSession,
    +      className = provider,
    +      options = storage.properties ++ pathOption,
    +      catalogTable = None)
    +
    --- End diff --
    
    You can add an extra check here. 
    
    `val isFileFormat = classOf[FileFormat].isAssignableFrom(dataSource.providingClass)`
    
    If `isFileFormat ` is false, return an error. 
    
    A simple example is like 
    ```Scala
          sql(
            s"""
              |INSERT OVERWRITE DIRECTORY '${path}'
              |USING JDBC
              |OPTIONS (uRl '$url1', DbTaBlE 'TEST.PEOPLE1', User 'testUser', PassWord 'testPass')
              |SELECT 1, 2
            """.stripMargin)
    ```
    
    Currently, the above query can pass. We should get an exception 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 issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137629284
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,150 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10")
    +
    +      sql(
    +        s"""
    +           |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +           |STORED AS orc
    +           |SELECT * FROM src where key < 10
    +         """.stripMargin)
    +
    +      // use orc data source to check the data of path is right.
    +      withTempView("orc_source") {
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY VIEW orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from src where key < 10"))
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to local dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val path = dir.toURI.getPath
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        checkAnswer(
    +          spark.read.orc(dir.getCanonicalPath),
    +          sql("select * from test_insert_table"))
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val pathUri = dir.toURI
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        checkAnswer(
    +          spark.read.orc(dir.getCanonicalPath),
    +          sql("select * from test_insert_table"))
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to dir to illegal path") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      val e = intercept[IllegalArgumentException] {
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY 'abc://a'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +      }.getMessage
    +
    +      assert(e.contains("Wrong FS: abc://a, expected: file:///"))
    +    }
    +  }
    +
    +  test("insert overwrite to dir with mixed syntax") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      val e = intercept[ParseException] {
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY 'file://tmp'
    +             |USING json
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +      }.getMessage
    +
    +      assert(e.contains("mismatched input 'ROW'"))
    +    }
    +  }
    +
    +  test("insert overwrite to dir with multi inserts") {
    --- End diff --
    
    added


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

Posted by janewangfb <gi...@git.apache.org>.
GitHub user janewangfb reopened a pull request:

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

    [SPARK-4131] Support "Writing data into the filesystem from queries"

    ## What changes were proposed in this pull request?
    
    This PR implements the sql feature:
    INSERT OVERWRITE [LOCAL] DIRECTORY directory1
      [ROW FORMAT row_format] [STORED AS file_format] 
      SELECT ... FROM ...
    
    ## How was this patch tested?
    Added new unittests and also pulled the code to fb-spark so that we could test writing to hdfs directory.
    


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

    $ git pull https://github.com/janewangfb/spark port_local_directory

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

    https://github.com/apache/spark/pull/18975.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 #18975
    
----
commit 6ca7771a39fe7c6ca5f82acc7d76dff00238de68
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-17T01:32:23Z

    add insert overwrite local directory

commit a975536ea3fed0674410ebc65538e134992779b1
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-17T03:59:25Z

    Add Unittests

commit a15bf4e0c12a37127788528f1e69b0462a0e287b
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-17T05:23:34Z

    fix local path

commit 9f596fda9dccdb0d5541340e8d9f2bf83573066a
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-17T18:25:04Z

    Merge branch 'master' into port_local_directory

commit b9db02ecf130a8524fc9e864e1758117094f04cf
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-17T18:50:50Z

    fix style

commit e516bec41bf877070023affbb79cc7823e2d6af8
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-18T04:06:38Z

    Merge branch 'master' into port_local_directory

commit e05624ff15092773c329182e85cd2f942151def0
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-18T06:53:27Z

    condense storage

commit 7f5664dcf5e55f3cc2ad91a000bb2c8ab58f596a
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-18T07:13:00Z

    change InsertInto to InsertIntoTable

commit d50b3a2a722783e4bf971382946fce3e61ddee3b
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-19T16:25:28Z

    add InsertIntoDirectory

commit 61a18a29955e71a89eecfeab6a01ba725be157ce
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-19T16:36:32Z

    update insertInto

commit 4c19aafff3167206eaf9895f66ae5bf5301725cf
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-19T18:16:57Z

    SQLQuerySuite passed

commit 47fde8a6591cbdad5656b51c3b55e3647d7b8d1f
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-19T18:37:39Z

    fix comments

commit 068662a5abaaa693529320bb855b7a3323915bf8
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-21T03:55:09Z

    Merge branch 'master' into port_local_directory

commit da7065ba5b30a777878ee1f8cb6c5fb4cfa42212
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-21T16:35:14Z

    Add tableProdier

commit 7f4b488a3aff850949c4d414eea8c071265a7d4c
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-21T20:54:46Z

    Add InsertIntoDataSourceDirCommand

commit 051018ef776e8634295e9e60b353694feefd72cb
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-21T20:55:48Z

    Merge branch 'master' into port_local_directory

commit 73f605ed955657e6db4d29503ea1df403c304ba4
Author: Jane Wang <ja...@fb.com>
Date:   2017-08-21T20:59:12Z

    fix style

----


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

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

    https://github.com/apache/spark/pull/18975#discussion_r155518105
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,82 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + *
    + * @param storage storage format used to describe how the query result is stored.
    + * @param provider the data source type to be used
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: String,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def children: Seq[LogicalPlan] = Seq(query)
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty, "Directory path is required")
    +    assert(provider.nonEmpty, "Data source is required")
    +
    +    // Create the relation based on the input logical plan: `query`.
    +    val pathOption = storage.locationUri.map("path" -> CatalogUtils.URIToString(_))
    +
    +    val dataSource = DataSource(
    +      sparkSession,
    +      className = provider,
    +      options = storage.properties ++ pathOption,
    +      catalogTable = None)
    +
    +    val isFileFormat = classOf[FileFormat].isAssignableFrom(dataSource.providingClass)
    +    if (!isFileFormat) {
    +      throw new SparkException(
    +        "Only Data Sources providing FileFormat are supported: " + dataSource.providingClass)
    +    }
    +
    +    val saveMode = if (overwrite) SaveMode.Overwrite else SaveMode.ErrorIfExists
    +    try {
    +      sparkSession.sessionState.executePlan(dataSource.planForWriting(saveMode, query))
    +      dataSource.writeAndRead(saveMode, query)
    --- End diff --
    
    @gatorsmile Thanks for you reply, I'll try to fix this.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136726866
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1509,4 +1509,86 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    if (ctx.LOCAL != null) {
    +      throw new ParseException(
    +        "LOCAL is not supported in INSERT OVERWRITE DIRECTORY to data source", ctx)
    +    }
    +
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path).map(string).getOrElse("")
    +
    +    if (!path.isEmpty && storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " +
    +          "you can only specify one of them.", ctx)
    +    }
    +    if (path.isEmpty && storage.locationUri.isEmpty) {
    +      throw new ParseException(
    +        "You need to specify directory path or 'path' in OPTIONS, but not both", ctx)
    --- End diff --
    
    nit: `but not both` does not seem in sync with the check because both `path` and `locationUri` are not specified for the check to be true.


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

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

    https://github.com/apache/spark/pull/18975#discussion_r134382724
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala ---
    @@ -0,0 +1,77 @@
    +/*
    + * 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.execution
    +
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.internal.io.FileCommitProtocol
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.catalyst.catalog.BucketSpec
    +import org.apache.spark.sql.catalyst.expressions.Attribute
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.execution.command.DataWritingCommand
    +import org.apache.spark.sql.execution.datasources.FileFormatWriter
    +import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc}
    +
    +// Base trait from which all hive insert statement physical execution extends.
    +private[hive] trait SaveAsHiveFile extends DataWritingCommand {
    +
    +  protected def saveAsHiveFile(sparkSession: SparkSession,
    +                               plan: SparkPlan,
    +                               hadoopConf: Configuration,
    +                               fileSinkConf: FileSinkDesc,
    +                               outputLocation: String,
    +                               partitionAttributes: Seq[Attribute] = Nil,
    +                               bucketSpec: Option[BucketSpec] = None,
    +                               options: Map[String, String] = Map.empty): Unit = {
    --- End diff --
    
    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 issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #80874 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80874/testReport)** for PR 18975 at commit [`7f5664d`](https://github.com/apache/spark/commit/7f5664dcf5e55f3cc2ad91a000bb2c8ab58f596a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/80874/
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136727120
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,145 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile {
    +
    +  override def children: Seq[LogicalPlan] = query :: Nil
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +
    +//    val Array(cols, types) = children.head.output.foldLeft(Array("", "")) { case (r, a) =>
    --- End diff --
    
    deadcode ?


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137680153
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala ---
    @@ -360,6 +360,31 @@ case class InsertIntoTable(
     }
     
     /**
    + * Insert query result into a directory.
    + *
    + * @param isLocal Indicates whether the specified directory is local directory
    + * @param storage Info about output file, row and what serialization format
    + * @param provider Specifies what data source to use; only used for data source file.
    + * @param child The query to be executed
    + * @param overwrite If true, the existing directory will be overwritten
    + *
    + * Note that this plan is unresolved and has to be replaced by the concrete implementations
    + * sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scaladuring analysis.
    --- End diff --
    
    Could you fix it?


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r138525454
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whether overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile with HiveTmpPath {
    --- End diff --
    
    why do we separate `SaveAsHiveFile` and `HiveTmpPath`, while we always use them together?


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134352371
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1509,4 +1509,84 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path) match {
    +      case Some(s) => string(s)
    +      case None => ""
    +    }
    +
    +    if (!path.isEmpty && storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " +
    +          "you can only specify one of them.", ctx)
    +    }
    +    if (path.isEmpty && !storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "You need to specify directory path or 'path' in OPTIONS, but not both", ctx)
    +    }
    +
    +    if (!path.isEmpty) {
    +      val customLocation = Some(CatalogUtils.stringToURI(path))
    +      storage = storage.copy(locationUri = customLocation)
    +    }
    +
    +    val provider = ctx.tableProvider.qualifiedName.getText
    +
    +    (false, storage, Some(provider))
    +  }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   path
    --- End diff --
    
    `LOCAL?`


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

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

    https://github.com/apache/spark/pull/18975#discussion_r136724143
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -178,11 +179,50 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider).
    +   */
    +  type InsertDirParams = (Boolean, CatalogStorageFormat, Option[String])
    +
    +  /**
    +   * Add an
    +   *   INSERT INTO [TABLE] or
    +   *   INSERT OVERWRITE TABLE or
    +   *   INSERT OVERWRITE [LOCAL] DIRECTORY
    +   * operation to logical plan
        */
       private def withInsertInto(
           ctx: InsertIntoContext,
           query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    assert(ctx.children.size == 1)
    +
    +    ctx.getChild(0) match {
    +      case c if c. isInstanceOf[InsertIntoTableContext] =>
    --- End diff --
    
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136680298
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala ---
    @@ -140,6 +141,10 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast
             parts, query, overwrite, false) if parts.isEmpty =>
           InsertIntoDataSourceCommand(l, query, overwrite)
     
    +    case InsertIntoDir(_, storage, provider, query, overwrite)
    +      if provider.isDefined && provider.get.toLowerCase(Locale.ROOT) != DDLUtils.HIVE_PROVIDER =>
    +      InsertIntoDataSourceDirCommand(storage, provider.get, query, overwrite)
    --- End diff --
    
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137919837
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1512,4 +1512,81 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    if (ctx.LOCAL != null) {
    +      throw new ParseException(
    +        "LOCAL is not supported in INSERT OVERWRITE DIRECTORY to data source", ctx)
    +    }
    +
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path).map(string).getOrElse("")
    +
    +    if (!(path.isEmpty ^ storage.locationUri.isEmpty)) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS should be specified one, but not both", ctx)
    --- End diff --
    
    Users must specify one and only one


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137399408
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.execution
    +
    +import java.io.{File, IOException}
    +import java.net.URI
    +import java.text.SimpleDateFormat
    +import java.util.{Date, Locale, Random}
    +
    +import scala.util.control.NonFatal
    +
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.exec.TaskRunner
    +
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.execution.command.RunnableCommand
    +import org.apache.spark.sql.hive.HiveExternalCatalog
    +import org.apache.spark.sql.hive.client.HiveVersion
    +
    +// Base trait for getting a temporary location for writing data
    +private[hive] trait HiveTmpPath extends RunnableCommand {
    +
    +  var createdTempDir: Option[Path] = None
    +
    +  private var stagingDir: String = ""
    +
    +  def getExternalTmpPath(
    +      sparkSession: SparkSession,
    +      hadoopConf: Configuration,
    +      path: Path): Path = {
    +    import org.apache.spark.sql.hive.client.hive._
    +
    +    // Before Hive 1.1, when inserting into a table, Hive will create the staging directory under
    +    // a common scratch directory. After the writing is finished, Hive will simply empty the table
    +    // directory and move the staging directory to it.
    +    // After Hive 1.1, Hive will create the staging directory under the table directory, and when
    +    // moving staging directory to table directory, Hive will still empty the table directory, but
    +    // will exclude the staging directory there.
    +    // We have to follow the Hive behavior here, to avoid troubles. For example, if we create
    +    // staging directory under the table director for Hive prior to 1.1, the staging directory will
    +    // be removed by Hive when Hive is trying to empty the table directory.
    +    val hiveVersionsUsingOldExternalTempPath: Set[HiveVersion] = Set(v12, v13, v14, v1_0)
    +    val hiveVersionsUsingNewExternalTempPath: Set[HiveVersion] = Set(v1_1, v1_2, v2_0, v2_1)
    +
    +    // Ensure all the supported versions are considered here.
    +    assert(hiveVersionsUsingNewExternalTempPath ++ hiveVersionsUsingOldExternalTempPath ==
    +      allSupportedHiveVersions)
    +
    +    val externalCatalog = sparkSession.sharedState.externalCatalog
    +    val hiveVersion = externalCatalog.asInstanceOf[HiveExternalCatalog].client.version
    +    stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging")
    +    val scratchDir = hadoopConf.get("hive.exec.scratchdir", "/tmp/hive")
    +
    +    if (hiveVersionsUsingOldExternalTempPath.contains(hiveVersion)) {
    +      oldVersionExternalTempPath(path, hadoopConf, scratchDir)
    +    } else if (hiveVersionsUsingNewExternalTempPath.contains(hiveVersion)) {
    +      newVersionExternalTempPath(path, hadoopConf, stagingDir)
    +    } else {
    +      throw new IllegalStateException("Unsupported hive version: " + hiveVersion.fullVersion)
    +    }
    +  }
    +
    +  def deleteExternalTmpPath(hadoopConf : Configuration) : Unit = {
    +    // Attempt to delete the staging directory and the inclusive files. If failed, the files are
    +    // expected to be dropped at the normal termination of VM since deleteOnExit is used.
    +    try {
    +      createdTempDir.foreach { path =>
    +        val fs = path.getFileSystem(hadoopConf)
    +        if (fs.delete(path, true)) {
    +          // If we successfully delete the staging directory, remove it from FileSystem's cache.
    +          fs.cancelDeleteOnExit(path)
    +        }
    +      }
    +    } catch {
    +      case NonFatal(e) =>
    +        logWarning(s"Unable to delete staging directory: $stagingDir.\n" + e)
    --- End diff --
    
    we define staging dir as class member already.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #80917 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80917/testReport)** for PR 18975 at commit [`068662a`](https://github.com/apache/spark/commit/068662a5abaaa693529320bb855b7a3323915bf8).


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137919040
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,82 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + *
    + * @param storage storage format used to describe how the query result is stored.
    + * @param provider the data source type to be used
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: String,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def children: Seq[LogicalPlan] = Seq(query)
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty, "Directory path is required")
    +    assert(!provider.isEmpty, "Data source is required")
    --- End diff --
    
    nit: `provider.nonEmpty` for being consistent with above.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Added the support for write out data source format.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81593 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81593/testReport)** for PR 18975 at commit [`81382df`](https://github.com/apache/spark/commit/81382dfec9ba764bc429900e52971851b63bcfb9).


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134682413
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,67 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: Option[String],
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def innerChildren: Seq[LogicalPlan] = Seq(query)
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    assert(innerChildren.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +    assert(provider.isDefined)
    --- End diff --
    
    Could you please add the helper messages for the above three asserts? 


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137061444
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,145 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile {
    +
    +  override def children: Seq[LogicalPlan] = query :: Nil
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +
    +//    val Array(cols, types) = children.head.output.foldLeft(Array("", "")) { case (r, a) =>
    --- End diff --
    
    I was waiting for comment from @gatorsmile about using CatalogTable with table identifier and database. if he is ok with that, I will remove this code.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137386317
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -178,11 +179,60 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   * Parameters used for writing query to a table:
    +   *   (tableIdentifier, partitionKeys, overwrite, exists).
    +   */
    +  type InsertTableParams = (TableIdentifier, Map[String, Option[String]], Boolean, Boolean)
    +
    +  /**
    +   * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider).
    +   */
    +  type InsertDirParams = (Boolean, CatalogStorageFormat, Option[String])
    +
    +  /**
    +   * Add an
    +   *   INSERT INTO [TABLE] or
    +   *   INSERT OVERWRITE TABLE or
    +   *   INSERT OVERWRITE [LOCAL] DIRECTORY
    +   * operation to logical plan
        */
       private def withInsertInto(
           ctx: InsertIntoContext,
           query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    ctx match {
    +      case table : InsertIntoTableContext =>
    --- End diff --
    
    Nit: `table :`-> `table:`
    The same issue in line 206


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81346 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81346/testReport)** for PR 18975 at commit [`52350e8`](https://github.com/apache/spark/commit/52350e8adb7b07540f635c103f36e065c151f8cd).


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137455718
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,138 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile with HiveTmpPath {
    +
    +  override def children: Seq[LogicalPlan] = query :: Nil
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +
    +    val hiveTable = HiveClientImpl.toHiveTable(CatalogTable(
    +      identifier = TableIdentifier(storage.locationUri.get.toString, Some("default")),
    +      tableType = org.apache.spark.sql.catalyst.catalog.CatalogTableType.VIEW,
    +      storage = storage,
    +      schema = query.schema
    +    ))
    +    hiveTable.getMetadata.put(serdeConstants.SERIALIZATION_LIB,
    +      storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
    +
    +    val tableDesc = new TableDesc(
    +      hiveTable.getInputFormatClass,
    +      hiveTable.getOutputFormatClass,
    +      hiveTable.getMetadata
    +    )
    +
    +    val hadoopConf = sparkSession.sessionState.newHadoopConf()
    +    val jobConf = new JobConf(hadoopConf)
    +
    +    val targetPath = new Path(storage.locationUri.get)
    +    val writeToPath =
    +      if (isLocal) {
    +        val localFileSystem = FileSystem.getLocal(jobConf)
    +        localFileSystem.makeQualified(targetPath)
    +      } else {
    +        val qualifiedPath = FileUtils.makeQualified(targetPath, hadoopConf)
    +        val dfs = qualifiedPath.getFileSystem(jobConf)
    +        if (!dfs.exists(qualifiedPath)) {
    +          dfs.mkdirs(qualifiedPath.getParent)
    +        }
    +        qualifiedPath
    +      }
    +
    +    val tmpPath = getExternalTmpPath(sparkSession, hadoopConf, writeToPath)
    +    val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc(
    +      tmpPath.toString, tableDesc, false)
    +
    +    try {
    +      saveAsHiveFile(
    +        sparkSession = sparkSession,
    +        plan = children.head,
    +        hadoopConf = hadoopConf,
    +        fileSinkConf = fileSinkConf,
    +        outputLocation = tmpPath.toString)
    +
    +      val fs = writeToPath.getFileSystem(hadoopConf)
    +      if (overwrite && fs.exists(writeToPath)) {
    +        val existFiles = fs.listStatus(writeToPath)
    --- End diff --
    
    ```Scala
            fs.listStatus(writeToPath).foreach { existFile =>
              if (Option(existFile.getPath) != createdTempDir) fs.delete(existFile.getPath, true)
            }
    ```


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137919154
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1512,4 +1512,81 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    if (ctx.LOCAL != null) {
    +      throw new ParseException(
    +        "LOCAL is not supported in INSERT OVERWRITE DIRECTORY to data source", ctx)
    --- End diff --
    
    If we don't support `LOCAL` for data source, should we remove it from the parsing rule?


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137607545
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,138 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile with HiveTmpPath {
    +
    +  override def children: Seq[LogicalPlan] = query :: Nil
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +
    +    val hiveTable = HiveClientImpl.toHiveTable(CatalogTable(
    +      identifier = TableIdentifier(storage.locationUri.get.toString, Some("default")),
    +      tableType = org.apache.spark.sql.catalyst.catalog.CatalogTableType.VIEW,
    +      storage = storage,
    +      schema = query.schema
    +    ))
    +    hiveTable.getMetadata.put(serdeConstants.SERIALIZATION_LIB,
    +      storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
    +
    +    val tableDesc = new TableDesc(
    +      hiveTable.getInputFormatClass,
    +      hiveTable.getOutputFormatClass,
    +      hiveTable.getMetadata
    +    )
    +
    +    val hadoopConf = sparkSession.sessionState.newHadoopConf()
    +    val jobConf = new JobConf(hadoopConf)
    +
    +    val targetPath = new Path(storage.locationUri.get)
    +    val writeToPath =
    +      if (isLocal) {
    +        val localFileSystem = FileSystem.getLocal(jobConf)
    +        localFileSystem.makeQualified(targetPath)
    +      } else {
    +        val qualifiedPath = FileUtils.makeQualified(targetPath, hadoopConf)
    +        val dfs = qualifiedPath.getFileSystem(jobConf)
    +        if (!dfs.exists(qualifiedPath)) {
    +          dfs.mkdirs(qualifiedPath.getParent)
    +        }
    +        qualifiedPath
    +      }
    +
    +    val tmpPath = getExternalTmpPath(sparkSession, hadoopConf, writeToPath)
    +    val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc(
    +      tmpPath.toString, tableDesc, false)
    +
    +    try {
    +      saveAsHiveFile(
    +        sparkSession = sparkSession,
    +        plan = children.head,
    +        hadoopConf = hadoopConf,
    +        fileSinkConf = fileSinkConf,
    +        outputLocation = tmpPath.toString)
    +
    +      val fs = writeToPath.getFileSystem(hadoopConf)
    +      if (overwrite && fs.exists(writeToPath)) {
    +        val existFiles = fs.listStatus(writeToPath)
    +        existFiles.foreach {
    +          existFile =>
    +            if (existFile.getPath != createdTempDir.get) {
    +              fs.delete(existFile.getPath, true)
    +            }
    +        }
    +      }
    +
    +      val tmpFiles = fs.listStatus(tmpPath)
    +      tmpFiles.foreach {
    +        tmpFile =>
    +          fs.rename(tmpFile.getPath, writeToPath)
    +      }
    +
    +      val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging")
    +      deleteExternalTmpPath(hadoopConf, stagingDir)
    +    } catch {
    +      case e : Throwable =>
    --- End diff --
    
    updated


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137062047
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,145 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile {
    +
    +  override def children: Seq[LogicalPlan] = query :: Nil
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +
    +//    val Array(cols, types) = children.head.output.foldLeft(Array("", "")) { case (r, a) =>
    +//      r(0) = r(0) + a.name + ","
    +//      r(1) = r(1) + a.dataType.catalogString + ":"
    +//      r
    +//    }
    +//
    +//    val properties = new Properties()
    +//    properties.put("columns", cols.dropRight(1))
    +//    properties.put("columns.types", types.dropRight(1))
    +//    properties.put(serdeConstants.SERIALIZATION_LIB,
    +//      storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
    +//
    +//    import scala.collection.JavaConverters._
    +//    properties.putAll(storage.properties.asJava)
    +//
    +//    val tableDesc = new TableDesc(
    +//      Utils.classForName(storage.inputFormat.get).asInstanceOf[Class[_ <: InputFormat[_, _]]],
    +//      Utils.classForName(storage.outputFormat.get),
    +//      properties
    +//    )
    --- End diff --
    
    Removed.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136706616
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -178,11 +179,50 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider).
    +   */
    +  type InsertDirParams = (Boolean, CatalogStorageFormat, Option[String])
    +
    +  /**
    +   * Add an
    +   *   INSERT INTO [TABLE] or
    +   *   INSERT OVERWRITE TABLE or
    +   *   INSERT OVERWRITE [LOCAL] DIRECTORY
    +   * operation to logical plan
        */
       private def withInsertInto(
           ctx: InsertIntoContext,
           query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    assert(ctx.children.size == 1)
    +
    +    ctx.getChild(0) match {
    +      case c if c. isInstanceOf[InsertIntoTableContext] =>
    --- End diff --
    
    `match` can do type detection.. this can be simplified as :
    
    ```
          case tableContext: InsertIntoTableContext => withInsertIntoTable(tableContext, query)
          case dirContext: InsertOverwriteDirectoryContext => withInsertOverwriteDirectory(dirContext, query)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134887579
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1509,4 +1509,84 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path) match {
    +      case Some(s) => string(s)
    +      case None => ""
    +    }
    +
    +    if (!path.isEmpty && storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " +
    +          "you can only specify one of them.", ctx)
    +    }
    +    if (path.isEmpty && !storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "You need to specify directory path or 'path' in OPTIONS, but not both", ctx)
    +    }
    +
    +    if (!path.isEmpty) {
    +      val customLocation = Some(CatalogUtils.stringToURI(path))
    +      storage = storage.copy(locationUri = customLocation)
    +    }
    +
    +    val provider = ctx.tableProvider.qualifiedName.getText
    +
    +    (false, storage, Some(provider))
    +  }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE [LOCAL] DIRECTORY
    +   *   path
    +   *   [ROW FORMAT row_format]
    +   *   [STORED AS file_format]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteHiveDir(
    +      ctx: InsertOverwriteHiveDirContext): InsertDirParams = withOrigin(ctx) {
    +    validateRowFormatFileFormat(ctx.rowFormat, ctx.createFileFormat, ctx)
    +    val rowStorage = Option(ctx.rowFormat).map(visitRowFormat)
    +      .getOrElse(CatalogStorageFormat.empty)
    +    val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat)
    +      .getOrElse(CatalogStorageFormat.empty)
    +
    +    val path = string(ctx.path)
    +    // The path field is required
    +    if (path.isEmpty) {
    +      operationNotAllowed("INSERT OVERWRITE DIRECTORY must be accompanied by path", ctx)
    +    }
    +
    +    val defaultStorage = HiveSerDe.getDefaultStorage(conf)
    +
    +    val storage = CatalogStorageFormat(
    +      locationUri = Some(CatalogUtils.stringToURI(path)),
    +      inputFormat = fileStorage.inputFormat.orElse(defaultStorage.inputFormat),
    +      outputFormat = fileStorage.outputFormat.orElse(defaultStorage.outputFormat),
    +      serde = rowStorage.serde.orElse(fileStorage.serde).orElse(defaultStorage.serde),
    +      compressed = false,
    +      properties = rowStorage.properties ++ fileStorage.properties)
    +
    +    (ctx.LOCAL != null, storage, None)
    --- End diff --
    
    Even if for hive serde tables, we still do it. See
    https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala#L797-L801



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137918839
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -192,12 +245,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
             "partitions with value: " + dynamicPartitionKeys.keys.mkString("[", ",", "]"), ctx)
         }
     
    -    InsertIntoTable(
    -      UnresolvedRelation(tableIdent),
    -      partitionKeys,
    -      query,
    -      ctx.OVERWRITE != null,
    -      ctx.EXISTS != null)
    +    (tableIdent, partitionKeys, ctx.OVERWRITE() != null, ctx.EXISTS() != null)
    +  }
    +
    +  /**
    +   * Write to a file, returning a [[InsertIntoDir]] logical plan.
    --- End diff --
    
    Write to a dir?


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81050 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81050/testReport)** for PR 18975 at commit [`64f37f4`](https://github.com/apache/spark/commit/64f37f4a2517fb841a47d6fb105e01f10e700fc3).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137607749
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,81 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + *
    + * @param storage storage format used to describe how the query result is stored.
    + * @param provider the data source type to be used
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: String,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def innerChildren: Seq[LogicalPlan] = Seq(query)
    --- End diff --
    
    updated.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136724258
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -178,11 +179,50 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider).
    +   */
    +  type InsertDirParams = (Boolean, CatalogStorageFormat, Option[String])
    +
    +  /**
    +   * Add an
    +   *   INSERT INTO [TABLE] or
    +   *   INSERT OVERWRITE TABLE or
    +   *   INSERT OVERWRITE [LOCAL] DIRECTORY
    +   * operation to logical plan
        */
       private def withInsertInto(
           ctx: InsertIntoContext,
           query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    assert(ctx.children.size == 1)
    --- End diff --
    
    it means for insert into can only specify one case, either
    insert overwrite local directory path rowFormat....
    or
    insert overwrite local directiory path using....



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

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

    https://github.com/apache/spark/pull/18975#discussion_r134842881
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1509,4 +1509,84 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path) match {
    +      case Some(s) => string(s)
    +      case None => ""
    +    }
    +
    +    if (!path.isEmpty && storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " +
    +          "you can only specify one of them.", ctx)
    +    }
    +    if (path.isEmpty && !storage.locationUri.isDefined) {
    --- End diff --
    
    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 issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #80960 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80960/testReport)** for PR 18975 at commit [`bc5424c`](https://github.com/apache/spark/commit/bc5424cba17a72f5717b5ce6a2816bd7ab069b82).
     * This patch **fails due to an unknown error code, -9**.
     * 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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134351265
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -2040,4 +2040,80 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
           assert(setOfPath.size() == pathSizeToDeleteOnExit)
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    import org.apache.spark.util.Utils
    +
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    checkAnswer(
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}' SELECT * FROM src where key < 10"),
    +      Seq.empty[Row])
    +
    +    checkAnswer(
    +      sql(s"""INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}'
    +             |STORED AS orc
    +             |SELECT * FROM src where key < 10""".stripMargin),
    +      Seq.empty[Row])
    +
    +    // use orc data source to check the data of path is right.
    +    sql(
    +      s"""CREATE TEMPORARY TABLE orc_source
    +         |USING org.apache.spark.sql.hive.orc
    +         |OPTIONS (
    +         |  PATH '${path.getCanonicalPath}'
    +         |)
    +       """.stripMargin)
    +    checkAnswer(
    +      sql("select * from orc_source"),
    +      sql("select * from src where key < 10").collect()
    +    )
    +
    +    Utils.deleteRecursively(path)
    +    dropTempTable("orc_source")
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    import org.apache.spark.util.Utils
    +
    +    sparkContext
    +      .parallelize(1 to 10)
    +      .map(i => TestData(i, i.toString))
    +      .toDF()
    +      .registerTempTable("test_insert_table")
    +
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    checkAnswer(
    +      sql(
    +        s"""
    +           |INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}'
    +           |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +           |SELECT * FROM test_insert_table
    +         """.stripMargin),
    +      Seq.empty[Row])
    +
    +    checkAnswer(
    +      sql(s"""
    +        INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table""".stripMargin),
    +      Seq.empty[Row])
    +
    +    // use orc data source to check the data of path is right.
    +    sql(
    +      s"""CREATE TEMPORARY TABLE orc_source
    +         |USING org.apache.spark.sql.hive.orc
    +         |OPTIONS (
    +         |  PATH '${path.getCanonicalPath}'
    +         |)
    +       """.stripMargin)
    +    checkAnswer(
    +      sql("select * from orc_source"),
    +      sql("select * from test_insert_table").collect()
    +    )
    +    Utils.deleteRecursively(path)
    +    dropTempTable("test_insert_table")
    --- End diff --
    
    Please use `withTempView`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    How does Hive guarantee the atomicity of renaming/moving staging directory after we insert the data there? 


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137448976
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala ---
    @@ -32,7 +32,8 @@ import org.apache.spark.sql.catalyst.dsl.plans.DslLogicalPlan
     import org.apache.spark.sql.catalyst.expressions.JsonTuple
     import org.apache.spark.sql.catalyst.parser.ParseException
     import org.apache.spark.sql.catalyst.plans.PlanTest
    -import org.apache.spark.sql.catalyst.plans.logical.{Generate, LogicalPlan, Project, ScriptTransformation}
    +import org.apache.spark.sql.catalyst.plans.logical.{Generate, InsertIntoDir, LogicalPlan,
    +    Project, ScriptTransformation}
    --- End diff --
    
    We do not have a limit of characters. If it is too long, our style is 
    ```Scala
    import org.apache.spark.sql.catalyst.plans.logical.{Generate, InsertIntoDir, LogicalPlan}
    import org.apache.spark.sql.catalyst.plans.logical.{Project, ScriptTransformation}
    ```


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137401217
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala ---
    @@ -0,0 +1,78 @@
    +/*
    + * 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.execution
    +
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.internal.io.FileCommitProtocol
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.catalyst.catalog.BucketSpec
    +import org.apache.spark.sql.catalyst.expressions.Attribute
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.execution.command.DataWritingCommand
    +import org.apache.spark.sql.execution.datasources.FileFormatWriter
    +import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc}
    +
    +// Base trait from which all hive insert statement physical execution extends.
    +private[hive] trait SaveAsHiveFile extends DataWritingCommand {
    +
    +  protected def saveAsHiveFile(
    +      sparkSession: SparkSession,
    +      plan: SparkPlan,
    +      hadoopConf: Configuration,
    +      fileSinkConf: FileSinkDesc,
    +      outputLocation: String,
    +      partitionAttributes: Seq[Attribute] = Nil,
    +      bucketSpec: Option[BucketSpec] = None,
    +      options: Map[String, String] = Map.empty): Unit = {
    +
    +    val sessionState = sparkSession.sessionState
    --- End diff --
    
    remove this?


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    @tejasapatil So far, we disallow overwriting a path that is also being read from. Atomicity is not guaranteed even if we use the staging directory. Atomicity still cannot be achieved if we use moving/renaming the directory, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/80804/
    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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136497740
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala ---
    @@ -140,6 +141,10 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast
             parts, query, overwrite, false) if parts.isEmpty =>
           InsertIntoDataSourceCommand(l, query, overwrite)
     
    +    case InsertIntoDir(_, storage, provider, query, overwrite)
    +      if provider.isDefined && provider.get.toLowerCase(Locale.ROOT) != DDLUtils.HIVE_PROVIDER =>
    +      InsertIntoDataSourceDirCommand(storage, provider.get, query, overwrite)
    --- End diff --
    
    We need to block both cases of `InsertIntoDir ` and `InsertIntoHiveDirCommand `: `cannot overwrite a path that is also being read from`. See the example:
    https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala#L178-L187


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    @gatorsmile can you please review?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137403025
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -2346,6 +2347,45 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
         }
       }
     
    +  test("insert overwrite directory") {
    --- End diff --
    
    DDLSuite.scala is becoming bigger and bigger.
    
    Move these two data source only test cases to `org.apache.spark.sql.sources.InsertSuite`?


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81571 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81571/testReport)** for PR 18975 at commit [`449249e`](https://github.com/apache/spark/commit/449249e527e6b748e6bb111747bacab17390516c).


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r138680470
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whether overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile with HiveTmpPath {
    --- End diff --
    
    Sure, will submit a follow-up PR soon. 


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81366/
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136685529
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,134 @@
    +/*
    + * 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.execution
    +
    +import java.util.Properties
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile {
    +
    +  override def children: Seq[LogicalPlan] = query :: Nil
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +
    +    val Array(cols, types) = children.head.output.foldLeft(Array("", "")) { case (r, a) =>
    +      r(0) = r(0) + a.name + ","
    +      r(1) = r(1) + a.dataType.catalogString + ":"
    +      r
    +    }
    +
    +    val properties = new Properties()
    +    properties.put("columns", cols.dropRight(1))
    +    properties.put("columns.types", types.dropRight(1))
    +
    +    val sqlContext = sparkSession.sqlContext
    +
    +    properties.put(serdeConstants.SERIALIZATION_LIB,
    +      storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
    +
    +    import scala.collection.JavaConverters._
    +    properties.putAll(storage.properties.asJava)
    +
    +    var tableDesc = new TableDesc(
    +      Utils.classForName(storage.inputFormat.get).asInstanceOf[Class[_ <: InputFormat[_, _]]],
    +      Utils.classForName(storage.outputFormat.get),
    +      properties
    +    )
    --- End diff --
    
    If we use val hiveTable = HiveClientImpl.toHiveTable(dummyCatalogTableWithUserSpecifiedStorage), it requires a tablename and a database name. but in our case, we dont have it.
    
    I commented out my original code and implement the dummy hive table. Let me know if that's ok to.



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

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

    https://github.com/apache/spark/pull/18975#discussion_r137453643
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala ---
    @@ -366,4 +367,43 @@ class InsertSuite extends DataSourceTest with SharedSQLContext {
             Row(Array(1, 2), Array("a", "b")))
         }
       }
    +
    +  test("insert overwrite directory") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      val v1 =
    +        s"""
    +           | INSERT OVERWRITE DIRECTORY '${path}'
    +           | USING json
    +           | OPTIONS (a 1, b 0.1, c TRUE)
    --- End diff --
    
    Find an option from `JSONOptions` and verify it works properly.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137452482
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala ---
    @@ -524,6 +525,50 @@ class DDLParserSuite extends PlanTest with SharedSQLContext {
         assert(e.message.contains("you can only specify one of them."))
       }
     
    +  test("insert overwrite directory") {
    +    val v1 = "INSERT OVERWRITE DIRECTORY '/tmp/file' USING parquet SELECT 1 as a"
    +    parser.parsePlan(v1) match {
    +      case InsertIntoDir(_, storage, provider, query, overwrite) =>
    +        assert(storage.locationUri != None && storage.locationUri.get.toString == "/tmp/file")
    +      case other =>
    +        fail(s"Expected to parse ${classOf[InsertIntoDataSourceDirCommand].getClass.getName}" +
    +          " from query," + s"got ${other.getClass.getName}: $v1")
    --- End diff --
    
    Nit: `s" from query, got ${other.getClass.getName}: $v1"`


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136727065
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,81 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + *
    + * @param storage storage format used to describe how the query result is stored.
    + * @param provider the data source type to be used
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: String,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def innerChildren: Seq[LogicalPlan] = Seq(query)
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    assert(innerChildren.length == 1)
    +    assert(storage.locationUri.nonEmpty, "Directory path is required")
    +    assert(!provider.isEmpty, "Data source is required")
    +
    +    // Create the relation based on the input logical plan: `query`.
    +    val pathOption = storage.locationUri.map("path" -> CatalogUtils.URIToString(_))
    +
    +    val dataSource = DataSource(
    +      sparkSession,
    +      className = provider,
    +      options = storage.properties ++ pathOption,
    +      catalogTable = None)
    +
    +    val isFileFormat = classOf[FileFormat].isAssignableFrom(dataSource.providingClass)
    +    if (!isFileFormat) {
    +      throw new SparkException(
    +        "Only Data Sources providing FileFormat are supported.")
    --- End diff --
    
    nit: you could also include `dataSource.providingClass` in there to make it easy to reason / debug.


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

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

    https://github.com/apache/spark/pull/18975#discussion_r136261767
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: Option[String],
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def innerChildren: Seq[LogicalPlan] = Seq(query)
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    assert(innerChildren.length == 1)
    +    assert(storage.locationUri.nonEmpty, "Directory path is required")
    +    assert(provider.isDefined, "Data source is required")
    +
    +    // Create the relation based on the input logical plan: `data`.
    +    val pathOption = storage.locationUri.map("path" -> CatalogUtils.URIToString(_))
    +    val dataSource = DataSource(
    --- End diff --
    
    This test case only covers `FileFormat`. For the other external data sources that extend our data source API interfaces, e.g., `CreatableRelationProvider`, we can block it with a reasonable message.


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

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

    https://github.com/apache/spark/pull/18975#discussion_r155185768
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,82 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + *
    + * @param storage storage format used to describe how the query result is stored.
    + * @param provider the data source type to be used
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: String,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def children: Seq[LogicalPlan] = Seq(query)
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty, "Directory path is required")
    +    assert(provider.nonEmpty, "Data source is required")
    +
    +    // Create the relation based on the input logical plan: `query`.
    +    val pathOption = storage.locationUri.map("path" -> CatalogUtils.URIToString(_))
    +
    +    val dataSource = DataSource(
    +      sparkSession,
    +      className = provider,
    +      options = storage.properties ++ pathOption,
    +      catalogTable = None)
    +
    +    val isFileFormat = classOf[FileFormat].isAssignableFrom(dataSource.providingClass)
    +    if (!isFileFormat) {
    +      throw new SparkException(
    +        "Only Data Sources providing FileFormat are supported: " + dataSource.providingClass)
    +    }
    +
    +    val saveMode = if (overwrite) SaveMode.Overwrite else SaveMode.ErrorIfExists
    +    try {
    +      sparkSession.sessionState.executePlan(dataSource.planForWriting(saveMode, query))
    +      dataSource.writeAndRead(saveMode, query)
    --- End diff --
    
    The implementation here confused me, just want to leave a question here why we should call both `writeAndRead` and `planForWriting`?
    @janewangfb @gatorsmile @cloud-fan 


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

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


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137607475
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,138 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile with HiveTmpPath {
    +
    +  override def children: Seq[LogicalPlan] = query :: Nil
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +
    +    val hiveTable = HiveClientImpl.toHiveTable(CatalogTable(
    +      identifier = TableIdentifier(storage.locationUri.get.toString, Some("default")),
    +      tableType = org.apache.spark.sql.catalyst.catalog.CatalogTableType.VIEW,
    +      storage = storage,
    +      schema = query.schema
    +    ))
    +    hiveTable.getMetadata.put(serdeConstants.SERIALIZATION_LIB,
    +      storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
    +
    +    val tableDesc = new TableDesc(
    +      hiveTable.getInputFormatClass,
    +      hiveTable.getOutputFormatClass,
    +      hiveTable.getMetadata
    +    )
    +
    +    val hadoopConf = sparkSession.sessionState.newHadoopConf()
    +    val jobConf = new JobConf(hadoopConf)
    +
    +    val targetPath = new Path(storage.locationUri.get)
    +    val writeToPath =
    +      if (isLocal) {
    +        val localFileSystem = FileSystem.getLocal(jobConf)
    +        localFileSystem.makeQualified(targetPath)
    +      } else {
    +        val qualifiedPath = FileUtils.makeQualified(targetPath, hadoopConf)
    +        val dfs = qualifiedPath.getFileSystem(jobConf)
    +        if (!dfs.exists(qualifiedPath)) {
    +          dfs.mkdirs(qualifiedPath.getParent)
    +        }
    +        qualifiedPath
    +      }
    +
    +    val tmpPath = getExternalTmpPath(sparkSession, hadoopConf, writeToPath)
    +    val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc(
    +      tmpPath.toString, tableDesc, false)
    +
    +    try {
    +      saveAsHiveFile(
    +        sparkSession = sparkSession,
    +        plan = children.head,
    +        hadoopConf = hadoopConf,
    +        fileSinkConf = fileSinkConf,
    +        outputLocation = tmpPath.toString)
    +
    +      val fs = writeToPath.getFileSystem(hadoopConf)
    +      if (overwrite && fs.exists(writeToPath)) {
    +        val existFiles = fs.listStatus(writeToPath)
    --- End diff --
    
    updated.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136682501
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -241,11 +241,21 @@ query
         : ctes? queryNoWith
         ;
     
    -insertInto
    +insertIntoTable
         : INSERT OVERWRITE TABLE tableIdentifier (partitionSpec (IF NOT EXISTS)?)?
         | INSERT INTO TABLE? tableIdentifier partitionSpec?
         ;
     
    +insertOverwriteDirectory
    +    : INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat?                     #insertOverwriteHiveDir
    +    | INSERT OVERWRITE DIRECTORY (path=STRING)? tableProvider (OPTIONS options=tablePropertyList)?   #insertOverwriteDir
    --- End diff --
    
    ok. added local.


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137063333
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,132 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10")
    +
    +      sql(
    +        s"""
    +           |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +           |STORED AS orc
    +           |SELECT * FROM src where key < 10
    +         """.stripMargin)
    +
    +      // use orc data source to check the data of path is right.
    +      withTempView("orc_source") {
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY VIEW orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from src where key < 10").collect())
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to local dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val path = dir.toURI.getPath
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        withTempView("orc_source") {
    +          sql(
    +            s"""
    +               |CREATE TEMPORARY VIEW orc_source
    +               |USING org.apache.spark.sql.hive.orc
    +               |OPTIONS (
    +               |  PATH '${dir.getCanonicalPath}'
    +               |)
    +             """.stripMargin)
    +
    +          checkAnswer(
    +            sql("select * from orc_source"),
    +            sql("select * from test_insert_table").collect())
    +        }
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val pathUri = dir.toURI
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        withTempView("orc_source") {
    +          sql(
    +            s"""
    +               |CREATE TEMPORARY VIEW orc_source
    +               |USING org.apache.spark.sql.hive.orc
    +               |OPTIONS (
    +               |  PATH '${dir.getCanonicalPath}'
    +               |)
    +             """.stripMargin)
    +
    +          checkAnswer(
    +            sql("select * from orc_source"),
    --- End diff --
    
    updated.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137399745
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala ---
    @@ -234,12 +82,8 @@ case class InsertIntoHiveTable(
       override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
         assert(children.length == 1)
     
    -    val sessionState = sparkSession.sessionState
         val externalCatalog = sparkSession.sharedState.externalCatalog
    -    val hiveVersion = externalCatalog.asInstanceOf[HiveExternalCatalog].client.version
    -    val hadoopConf = sessionState.newHadoopConf()
    -    val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging")
    --- End diff --
    
    How about keeping `stagingDir `?


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137916510
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala ---
    @@ -360,6 +360,31 @@ case class InsertIntoTable(
     }
     
     /**
    + * Insert query result into a directory.
    + *
    + * @param isLocal Indicates whether the specified directory is local directory
    + * @param storage Info about output file, row and what serialization format
    + * @param provider Specifies what data source to use; only used for data source file.
    + * @param child The query to be executed
    + * @param overwrite If true, the existing directory will be overwritten
    + *
    + * Note that this plan is unresolved and has to be replaced by the concrete implementations
    + * sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scaladuring analysis.
    --- End diff --
    
    oops. sorry for that. fixed.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81449 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81449/testReport)** for PR 18975 at commit [`62370fd`](https://github.com/apache/spark/commit/62370fd4edbe7ca6f3b5de765658419717898c98).


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Looks pretty well, left few minor comments. Thanks for working on this.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137063137
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,132 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10")
    +
    +      sql(
    +        s"""
    +           |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +           |STORED AS orc
    +           |SELECT * FROM src where key < 10
    +         """.stripMargin)
    +
    +      // use orc data source to check the data of path is right.
    +      withTempView("orc_source") {
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY VIEW orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from src where key < 10").collect())
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to local dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val path = dir.toURI.getPath
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        withTempView("orc_source") {
    +          sql(
    +            s"""
    +               |CREATE TEMPORARY VIEW orc_source
    +               |USING org.apache.spark.sql.hive.orc
    +               |OPTIONS (
    +               |  PATH '${dir.getCanonicalPath}'
    +               |)
    +             """.stripMargin)
    +
    +          checkAnswer(
    +            sql("select * from orc_source"),
    +            sql("select * from test_insert_table").collect())
    --- End diff --
    
    removed


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137605311
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala ---
    @@ -524,6 +525,50 @@ class DDLParserSuite extends PlanTest with SharedSQLContext {
         assert(e.message.contains("you can only specify one of them."))
       }
     
    +  test("insert overwrite directory") {
    +    val v1 = "INSERT OVERWRITE DIRECTORY '/tmp/file' USING parquet SELECT 1 as a"
    +    parser.parsePlan(v1) match {
    +      case InsertIntoDir(_, storage, provider, query, overwrite) =>
    +        assert(storage.locationUri != None && storage.locationUri.get.toString == "/tmp/file")
    +      case other =>
    +        fail(s"Expected to parse ${classOf[InsertIntoDataSourceDirCommand].getClass.getName}" +
    +          " from query," + s"got ${other.getClass.getName}: $v1")
    --- End diff --
    
    updated.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137929091
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1512,4 +1512,81 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    if (ctx.LOCAL != null) {
    +      throw new ParseException(
    +        "LOCAL is not supported in INSERT OVERWRITE DIRECTORY to data source", ctx)
    +    }
    +
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path).map(string).getOrElse("")
    +
    +    if (!(path.isEmpty ^ storage.locationUri.isEmpty)) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS should be specified one, but not both", ctx)
    --- End diff --
    
    I think the original message is clear and simple. 


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136417593
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1509,4 +1509,84 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path) match {
    +      case Some(s) => string(s)
    +      case None => ""
    +    }
    +
    +    if (!path.isEmpty && storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " +
    +          "you can only specify one of them.", ctx)
    +    }
    +    if (path.isEmpty && !storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "You need to specify directory path or 'path' in OPTIONS, but not both", ctx)
    +    }
    +
    +    if (!path.isEmpty) {
    +      val customLocation = Some(CatalogUtils.stringToURI(path))
    +      storage = storage.copy(locationUri = customLocation)
    +    }
    +
    +    val provider = ctx.tableProvider.qualifiedName.getText
    +
    +    (false, storage, Some(provider))
    +  }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE [LOCAL] DIRECTORY
    +   *   path
    +   *   [ROW FORMAT row_format]
    +   *   [STORED AS file_format]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteHiveDir(
    +      ctx: InsertOverwriteHiveDirContext): InsertDirParams = withOrigin(ctx) {
    +    validateRowFormatFileFormat(ctx.rowFormat, ctx.createFileFormat, ctx)
    +    val rowStorage = Option(ctx.rowFormat).map(visitRowFormat)
    +      .getOrElse(CatalogStorageFormat.empty)
    +    val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat)
    +      .getOrElse(CatalogStorageFormat.empty)
    +
    +    val path = string(ctx.path)
    +    // The path field is required
    +    if (path.isEmpty) {
    +      operationNotAllowed("INSERT OVERWRITE DIRECTORY must be accompanied by path", ctx)
    +    }
    +
    +    val defaultStorage = HiveSerDe.getDefaultStorage(conf)
    +
    +    val storage = CatalogStorageFormat(
    +      locationUri = Some(CatalogUtils.stringToURI(path)),
    +      inputFormat = fileStorage.inputFormat.orElse(defaultStorage.inputFormat),
    +      outputFormat = fileStorage.outputFormat.orElse(defaultStorage.outputFormat),
    +      serde = rowStorage.serde.orElse(fileStorage.serde).orElse(defaultStorage.serde),
    +      compressed = false,
    +      properties = rowStorage.properties ++ fileStorage.properties)
    +
    +    (ctx.LOCAL != null, storage, None)
    --- End diff --
    
    got it. 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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136258558
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,118 @@
    +/*
    + * 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.execution
    +
    +import java.util.Properties
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.util.Utils
    +
    +
    +case class InsertIntoHiveDirCommand(
    --- End diff --
    
    Add the description like https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala#L48-L77


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

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

    https://github.com/apache/spark/pull/18975#discussion_r136517343
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -241,11 +241,21 @@ query
         : ctes? queryNoWith
         ;
     
    -insertInto
    +insertIntoTable
         : INSERT OVERWRITE TABLE tableIdentifier (partitionSpec (IF NOT EXISTS)?)?
         | INSERT INTO TABLE? tableIdentifier partitionSpec?
         ;
     
    +insertOverwriteDirectory
    +    : INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat?                     #insertOverwriteHiveDir
    +    | INSERT OVERWRITE DIRECTORY (path=STRING)? tableProvider (OPTIONS options=tablePropertyList)?   #insertOverwriteDir
    --- End diff --
    
    We should also have `LOCAL?` here. This is just for usability. Our current parser always issue a confusing error message. We can issue a better exception in the visit function.


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

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

    https://github.com/apache/spark/pull/18975#discussion_r136262362
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,83 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      checkAnswer(
    +        sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10"),
    +        Seq.empty[Row])
    +
    +      checkAnswer(
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |STORED AS orc
    +             |SELECT * FROM src where key < 10
    +          """.stripMargin),
    +        Seq.empty[Row])
    +
    +      // use orc data source to check the data of path is right.
    +      withTempView("orc_source") {
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY VIEW orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from src where key < 10").collect())
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val path = dir.toURI.getPath
    +
    +        checkAnswer(
    +          sql(
    +            s"""
    +               |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    --- End diff --
    
    Add a test case for non-local path?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81338 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81338/testReport)** for PR 18975 at commit [`52350e8`](https://github.com/apache/spark/commit/52350e8adb7b07540f635c103f36e065c151f8cd).
     * This patch **fails due to an unknown error code, -9**.
     * 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    @gatorsmile : Yes. Hive is not 100% atomic as stuff can go wrong between removing old data and renaming staging location. But its superior in these regards:
    
    - Hive would output "no data" OR "complete data". Here we can have "no data" OR "incomplete data" OR "complete data". The "incomplete data" part worries me. Staging dir helps achieving "you either see nothing OR everything" behaviour.
    - The window of "you see nothing" is much bigger here compared to Hive as the output location is cleaned up before execution.
    



---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137601757
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala ---
    @@ -524,6 +525,50 @@ class DDLParserSuite extends PlanTest with SharedSQLContext {
         assert(e.message.contains("you can only specify one of them."))
       }
     
    +  test("insert overwrite directory") {
    +    val v1 = "INSERT OVERWRITE DIRECTORY '/tmp/file' USING parquet SELECT 1 as a"
    +    parser.parsePlan(v1) match {
    +      case InsertIntoDir(_, storage, provider, query, overwrite) =>
    +        assert(storage.locationUri != None && storage.locationUri.get.toString == "/tmp/file")
    +      case other =>
    +        fail(s"Expected to parse ${classOf[InsertIntoDataSourceDirCommand].getClass.getName}" +
    +          " from query," + s"got ${other.getClass.getName}: $v1")
    +    }
    +
    +    val v2 = "INSERT OVERWRITE DIRECTORY USING parquet SELECT 1 as a"
    +    val e2 = intercept[ParseException] {
    +      parser.parsePlan(v2)
    +    }
    +    assert(e2.message.contains(
    +      "Directory path and 'path' in OPTIONS should be specified one, but not both"))
    +
    +    val v3 =
    +      """
    +        | INSERT OVERWRITE DIRECTORY USING json
    +        | OPTIONS ('path' '/tmp/file', a 1, b 0.1, c TRUE)
    +        | SELECT 1 as a
    +      """.stripMargin
    +    parser.parsePlan(v3) match {
    +      case InsertIntoDir(_, storage, provider, query, overwrite) =>
    +        assert(storage.locationUri != None && provider == Some("json"))
    --- End diff --
    
    added


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136726921
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1509,4 +1509,86 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    if (ctx.LOCAL != null) {
    +      throw new ParseException(
    +        "LOCAL is not supported in INSERT OVERWRITE DIRECTORY to data source", ctx)
    +    }
    +
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path).map(string).getOrElse("")
    +
    +    if (!path.isEmpty && storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " +
    +          "you can only specify one of them.", ctx)
    +    }
    +    if (path.isEmpty && storage.locationUri.isEmpty) {
    --- End diff --
    
    you can collapse this check and one above by doing XOR


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/80944/
    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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    I am not opposing the solution of the staging directory, but just want to understand what kind of services we can guarantee. 


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137388263
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala ---
    @@ -360,6 +360,27 @@ case class InsertIntoTable(
     }
     
     /**
    + * Insert query result into a directory.
    + *
    + * @param isLocal Indicates whether the specified directory is local directory
    + * @param storage Info about output file, row and what serialization format
    + * @param provider Specifies what data source to use; only used for data source file.
    + * @param child The query to be executed
    + * @param overwrite If true, the existing directory will be overwritten
    + */
    +case class InsertIntoDir(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    provider: Option[String],
    +    child: LogicalPlan,
    +    overwrite: Boolean = true)
    +  extends LogicalPlan {
    +
    +  override def children: Seq[LogicalPlan] = child :: Nil
    +  override def output: Seq[Attribute] = Seq.empty
    --- End diff --
    
    Set `override lazy val resolved: Boolean = false`


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    @janewangfb Thank you for working on it! The implementation in the current PR is very specific to Hive table. To support such a command, could you also support data source 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81049/
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137453352
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala ---
    @@ -524,6 +525,50 @@ class DDLParserSuite extends PlanTest with SharedSQLContext {
         assert(e.message.contains("you can only specify one of them."))
       }
     
    +  test("insert overwrite directory") {
    +    val v1 = "INSERT OVERWRITE DIRECTORY '/tmp/file' USING parquet SELECT 1 as a"
    +    parser.parsePlan(v1) match {
    +      case InsertIntoDir(_, storage, provider, query, overwrite) =>
    +        assert(storage.locationUri != None && storage.locationUri.get.toString == "/tmp/file")
    +      case other =>
    +        fail(s"Expected to parse ${classOf[InsertIntoDataSourceDirCommand].getClass.getName}" +
    +          " from query," + s"got ${other.getClass.getName}: $v1")
    +    }
    +
    +    val v2 = "INSERT OVERWRITE DIRECTORY USING parquet SELECT 1 as a"
    +    val e2 = intercept[ParseException] {
    +      parser.parsePlan(v2)
    +    }
    +    assert(e2.message.contains(
    +      "Directory path and 'path' in OPTIONS should be specified one, but not both"))
    +
    +    val v3 =
    +      """
    +        | INSERT OVERWRITE DIRECTORY USING json
    +        | OPTIONS ('path' '/tmp/file', a 1, b 0.1, c TRUE)
    +        | SELECT 1 as a
    +      """.stripMargin
    +    parser.parsePlan(v3) match {
    +      case InsertIntoDir(_, storage, provider, query, overwrite) =>
    +        assert(storage.locationUri != None && provider == Some("json"))
    --- End diff --
    
    Also add another assert for `storage.properties`, which should contain the properties of options.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Jenkin test 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137455095
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,138 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile with HiveTmpPath {
    +
    +  override def children: Seq[LogicalPlan] = query :: Nil
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +
    +    val hiveTable = HiveClientImpl.toHiveTable(CatalogTable(
    +      identifier = TableIdentifier(storage.locationUri.get.toString, Some("default")),
    +      tableType = org.apache.spark.sql.catalyst.catalog.CatalogTableType.VIEW,
    +      storage = storage,
    +      schema = query.schema
    +    ))
    +    hiveTable.getMetadata.put(serdeConstants.SERIALIZATION_LIB,
    +      storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
    +
    +    val tableDesc = new TableDesc(
    +      hiveTable.getInputFormatClass,
    +      hiveTable.getOutputFormatClass,
    +      hiveTable.getMetadata
    +    )
    +
    +    val hadoopConf = sparkSession.sessionState.newHadoopConf()
    +    val jobConf = new JobConf(hadoopConf)
    +
    +    val targetPath = new Path(storage.locationUri.get)
    +    val writeToPath =
    +      if (isLocal) {
    +        val localFileSystem = FileSystem.getLocal(jobConf)
    +        localFileSystem.makeQualified(targetPath)
    +      } else {
    +        val qualifiedPath = FileUtils.makeQualified(targetPath, hadoopConf)
    +        val dfs = qualifiedPath.getFileSystem(jobConf)
    +        if (!dfs.exists(qualifiedPath)) {
    +          dfs.mkdirs(qualifiedPath.getParent)
    +        }
    +        qualifiedPath
    +      }
    +
    +    val tmpPath = getExternalTmpPath(sparkSession, hadoopConf, writeToPath)
    +    val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc(
    +      tmpPath.toString, tableDesc, false)
    +
    +    try {
    +      saveAsHiveFile(
    +        sparkSession = sparkSession,
    +        plan = children.head,
    +        hadoopConf = hadoopConf,
    +        fileSinkConf = fileSinkConf,
    +        outputLocation = tmpPath.toString)
    +
    +      val fs = writeToPath.getFileSystem(hadoopConf)
    +      if (overwrite && fs.exists(writeToPath)) {
    +        val existFiles = fs.listStatus(writeToPath)
    +        existFiles.foreach {
    +          existFile =>
    +            if (existFile.getPath != createdTempDir.get) {
    +              fs.delete(existFile.getPath, true)
    +            }
    +        }
    +      }
    +
    +      val tmpFiles = fs.listStatus(tmpPath)
    --- End diff --
    
    Nit:
    ```Scala
          fs.listStatus(tmpPath).foreach { tmpFile =>
            fs.rename(tmpFile.getPath, writeToPath)
          }
    ```



---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136258236
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala ---
    @@ -155,6 +156,9 @@ object HiveAnalysis extends Rule[LogicalPlan] {
     
         case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) =>
           CreateHiveTableAsSelectCommand(tableDesc, query, mode)
    +
    +    case InsertIntoDir(isLocal, storage, _, child, overwrite) =>
    --- End diff --
    
    `provider.toLowerCase(Locale.ROOT) == HIVE_PROVIDER`


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

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

    https://github.com/apache/spark/pull/18975#discussion_r136738182
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -178,11 +179,50 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider).
    +   */
    +  type InsertDirParams = (Boolean, CatalogStorageFormat, Option[String])
    +
    +  /**
    +   * Add an
    +   *   INSERT INTO [TABLE] or
    +   *   INSERT OVERWRITE TABLE or
    +   *   INSERT OVERWRITE [LOCAL] DIRECTORY
    +   * operation to logical plan
        */
       private def withInsertInto(
           ctx: InsertIntoContext,
           query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    assert(ctx.children.size == 1)
    --- End diff --
    
    It can be rewritten to 
    ```Scala
      private def withInsertInto(
          ctx: InsertIntoContext,
          query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
        ctx match {
          case c: InsertIntoTableContext =>
            withInsertIntoTable(c, query)
          case dir: InsertOverwriteDirContext =>
            val (isLocal, storage, provider) = visitInsertOverwriteDir(dir)
            InsertIntoDir(isLocal, storage, provider, query, overwrite = true)
          case hiveDir: InsertOverwriteHiveDirContext =>
            val (isLocal, storage, provider) = visitInsertOverwriteHiveDir(hiveDir)
            InsertIntoDir(isLocal, storage, provider, query, overwrite = true)
        }
      }
    ```


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

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

    https://github.com/apache/spark/pull/18975#discussion_r136261943
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,83 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      checkAnswer(
    +        sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10"),
    +        Seq.empty[Row])
    --- End diff --
    
    We do not need to check the result is empty.
    `sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10")` is good enough


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

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

    https://github.com/apache/spark/pull/18975#discussion_r136739979
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,132 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10")
    +
    +      sql(
    +        s"""
    +           |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +           |STORED AS orc
    +           |SELECT * FROM src where key < 10
    +         """.stripMargin)
    +
    +      // use orc data source to check the data of path is right.
    +      withTempView("orc_source") {
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY VIEW orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from src where key < 10").collect())
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to local dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val path = dir.toURI.getPath
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        withTempView("orc_source") {
    +          sql(
    +            s"""
    +               |CREATE TEMPORARY VIEW orc_source
    +               |USING org.apache.spark.sql.hive.orc
    +               |OPTIONS (
    +               |  PATH '${dir.getCanonicalPath}'
    +               |)
    +             """.stripMargin)
    +
    +          checkAnswer(
    +            sql("select * from orc_source"),
    +            sql("select * from test_insert_table").collect())
    --- 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 issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    There is a difference in Hive's semantics vs what this PR is doing. In Hive, the query execution writes to a staging location and the destination location is cleared + re-populated after the end of  query execution (it happens in `MoveTask`). This PR will first wipe the destination location and then perform the query execution to populate the destination location with desired data. 
    
    I like the hive model because: 
    - If the query execution fails, you atleast have the old data. Insert overwrite to table does the same thing. This PR will leave the output location empty.
    - Hive achieves atomicity by using a staging dir. With this PR, I am not sure what happens to the output location if the some tasks have written out the final data while rest are still working. Would it have partial output ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r133815498
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1499,4 +1500,34 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE or INSERT INTO [LOCAL] DIRECOTRY
    +   * operation to the logical plan.
    +   */
    +  protected override def withInsertInto(ctx: InsertIntoContext,
    +                                        query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    --- End diff --
    
    Indents


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #80804 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80804/testReport)** for PR 18975 at commit [`b9db02e`](https://github.com/apache/spark/commit/b9db02ecf130a8524fc9e864e1758117094f04cf).
     * 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136508091
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,134 @@
    +/*
    + * 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.execution
    +
    +import java.util.Properties
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile {
    +
    +  override def children: Seq[LogicalPlan] = query :: Nil
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +
    +    val Array(cols, types) = children.head.output.foldLeft(Array("", "")) { case (r, a) =>
    +      r(0) = r(0) + a.name + ","
    +      r(1) = r(1) + a.dataType.catalogString + ":"
    +      r
    +    }
    +
    +    val properties = new Properties()
    +    properties.put("columns", cols.dropRight(1))
    +    properties.put("columns.types", types.dropRight(1))
    +
    +    val sqlContext = sparkSession.sqlContext
    +
    +    properties.put(serdeConstants.SERIALIZATION_LIB,
    +      storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
    +
    +    import scala.collection.JavaConverters._
    +    properties.putAll(storage.properties.asJava)
    +
    +    var tableDesc = new TableDesc(
    --- End diff --
    
    `var` -> `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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81481 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81481/testReport)** for PR 18975 at commit [`28fcb39`](https://github.com/apache/spark/commit/28fcb39028d93ec6ecea9eecf289c0e88b6c9ae6).


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136418037
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala ---
    @@ -140,6 +141,9 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast
             parts, query, overwrite, false) if parts.isEmpty =>
           InsertIntoDataSourceCommand(l, query, overwrite)
     
    +    case InsertIntoDir(_, storage, provider, query, overwrite) if provider.nonEmpty =>
    --- End diff --
    
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134684398
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -2040,4 +2040,83 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
           assert(setOfPath.size() == pathSizeToDeleteOnExit)
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    --- End diff --
    
    Move all these test cases to `InsertIntoHiveTableSuite`. Also we should rename that suite to `InsertSuite`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81297 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81297/testReport)** for PR 18975 at commit [`e2db5e1`](https://github.com/apache/spark/commit/e2db5e1e0cc491480828328e07b7bb619dc05bbd).
     * This patch **fails SparkR 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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136419843
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,118 @@
    +/*
    + * 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.execution
    +
    +import java.util.Properties
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.util.Utils
    +
    +
    +case class InsertIntoHiveDirCommand(
    --- End diff --
    
    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 issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    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 pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134847542
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -2040,4 +2040,83 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
           assert(setOfPath.size() == pathSizeToDeleteOnExit)
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    --- End diff --
    
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134681065
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1509,4 +1509,84 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path) match {
    +      case Some(s) => string(s)
    +      case None => ""
    +    }
    +
    +    if (!path.isEmpty && storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " +
    +          "you can only specify one of them.", ctx)
    +    }
    +    if (path.isEmpty && !storage.locationUri.isDefined) {
    --- End diff --
    
    `!storage.locationUri.isDefined` -> `storage.locationUri.isEmpty`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/80917/
    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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137455829
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,138 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile with HiveTmpPath {
    +
    +  override def children: Seq[LogicalPlan] = query :: Nil
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +
    +    val hiveTable = HiveClientImpl.toHiveTable(CatalogTable(
    +      identifier = TableIdentifier(storage.locationUri.get.toString, Some("default")),
    +      tableType = org.apache.spark.sql.catalyst.catalog.CatalogTableType.VIEW,
    +      storage = storage,
    +      schema = query.schema
    +    ))
    +    hiveTable.getMetadata.put(serdeConstants.SERIALIZATION_LIB,
    +      storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
    +
    +    val tableDesc = new TableDesc(
    +      hiveTable.getInputFormatClass,
    +      hiveTable.getOutputFormatClass,
    +      hiveTable.getMetadata
    +    )
    +
    +    val hadoopConf = sparkSession.sessionState.newHadoopConf()
    +    val jobConf = new JobConf(hadoopConf)
    +
    +    val targetPath = new Path(storage.locationUri.get)
    +    val writeToPath =
    +      if (isLocal) {
    +        val localFileSystem = FileSystem.getLocal(jobConf)
    +        localFileSystem.makeQualified(targetPath)
    +      } else {
    +        val qualifiedPath = FileUtils.makeQualified(targetPath, hadoopConf)
    +        val dfs = qualifiedPath.getFileSystem(jobConf)
    +        if (!dfs.exists(qualifiedPath)) {
    +          dfs.mkdirs(qualifiedPath.getParent)
    +        }
    +        qualifiedPath
    +      }
    +
    +    val tmpPath = getExternalTmpPath(sparkSession, hadoopConf, writeToPath)
    +    val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc(
    +      tmpPath.toString, tableDesc, false)
    +
    +    try {
    +      saveAsHiveFile(
    +        sparkSession = sparkSession,
    +        plan = children.head,
    +        hadoopConf = hadoopConf,
    +        fileSinkConf = fileSinkConf,
    +        outputLocation = tmpPath.toString)
    +
    +      val fs = writeToPath.getFileSystem(hadoopConf)
    +      if (overwrite && fs.exists(writeToPath)) {
    +        val existFiles = fs.listStatus(writeToPath)
    +        existFiles.foreach {
    +          existFile =>
    +            if (existFile.getPath != createdTempDir.get) {
    +              fs.delete(existFile.getPath, true)
    +            }
    +        }
    +      }
    +
    +      val tmpFiles = fs.listStatus(tmpPath)
    +      tmpFiles.foreach {
    +        tmpFile =>
    +          fs.rename(tmpFile.getPath, writeToPath)
    +      }
    +
    +      val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging")
    +      deleteExternalTmpPath(hadoopConf, stagingDir)
    +    } catch {
    +      case e : Throwable =>
    --- End diff --
    
    `e :` -> `e:`


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137058514
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1509,4 +1509,86 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    if (ctx.LOCAL != null) {
    +      throw new ParseException(
    +        "LOCAL is not supported in INSERT OVERWRITE DIRECTORY to data source", ctx)
    +    }
    +
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path).map(string).getOrElse("")
    +
    +    if (!path.isEmpty && storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " +
    +          "you can only specify one of them.", ctx)
    +    }
    +    if (path.isEmpty && storage.locationUri.isEmpty) {
    +      throw new ParseException(
    +        "You need to specify directory path or 'path' in OPTIONS, but not both", ctx)
    --- End diff --
    
    No. the first one makes sure that we dont have two places define path. the second one makes ure that we have at least one place defines a path.  The exception message explains.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137920269
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1512,4 +1512,81 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    if (ctx.LOCAL != null) {
    +      throw new ParseException(
    +        "LOCAL is not supported in INSERT OVERWRITE DIRECTORY to data source", ctx)
    +    }
    +
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path).map(string).getOrElse("")
    +
    +    if (!(path.isEmpty ^ storage.locationUri.isEmpty)) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS should be specified one, but not both", ctx)
    --- End diff --
    
    We go into this condition when both are not specified. But the message is not for it. `Directory path and 'path' in OPTIONS should be specified one and only one, but not both or both not`.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137918600
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    --- End diff --
    
    fixed


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81333/
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137929001
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -192,12 +245,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
             "partitions with value: " + dynamicPartitionKeys.keys.mkString("[", ",", "]"), ctx)
         }
     
    -    InsertIntoTable(
    -      UnresolvedRelation(tableIdent),
    -      partitionKeys,
    -      query,
    -      ctx.OVERWRITE != null,
    -      ctx.EXISTS != null)
    +    (tableIdent, partitionKeys, ctx.OVERWRITE() != null, ctx.EXISTS() != null)
    --- End diff --
    
    updated.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81576 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81576/testReport)** for PR 18975 at commit [`7919041`](https://github.com/apache/spark/commit/79190414db6a825f4489cb71a6050a4c3aa0fd46).


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136418390
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala ---
    @@ -155,6 +156,9 @@ object HiveAnalysis extends Rule[LogicalPlan] {
     
         case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) =>
           CreateHiveTableAsSelectCommand(tableDesc, query, mode)
    +
    +    case InsertIntoDir(isLocal, storage, _, child, overwrite) =>
    --- End diff --
    
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137106989
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -241,11 +241,21 @@ query
         : ctes? queryNoWith
         ;
     
    -insertInto
    +insertIntoTable
    --- End diff --
    
    updated.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137406502
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala ---
    @@ -0,0 +1,78 @@
    +/*
    + * 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.execution
    +
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.internal.io.FileCommitProtocol
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.catalyst.catalog.BucketSpec
    +import org.apache.spark.sql.catalyst.expressions.Attribute
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.execution.command.DataWritingCommand
    +import org.apache.spark.sql.execution.datasources.FileFormatWriter
    +import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc}
    +
    +// Base trait from which all hive insert statement physical execution extends.
    +private[hive] trait SaveAsHiveFile extends DataWritingCommand {
    +
    +  protected def saveAsHiveFile(
    +      sparkSession: SparkSession,
    +      plan: SparkPlan,
    +      hadoopConf: Configuration,
    +      fileSinkConf: FileSinkDesc,
    +      outputLocation: String,
    +      partitionAttributes: Seq[Attribute] = Nil,
    +      bucketSpec: Option[BucketSpec] = None,
    +      options: Map[String, String] = Map.empty): Unit = {
    --- End diff --
    
    removed


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137929053
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala ---
    @@ -360,6 +360,31 @@ case class InsertIntoTable(
     }
     
     /**
    + * Insert query result into a directory.
    + *
    + * @param isLocal Indicates whether the specified directory is local directory
    + * @param storage Info about output file, row and what serialization format
    + * @param provider Specifies what data source to use; only used for data source file.
    + * @param child The query to be executed
    + * @param overwrite If true, the existing directory will be overwritten
    + *
    + * Note that this plan is unresolved and has to be replaced by the concrete implementations
    + * during analysis.
    + */
    +case class InsertIntoDir(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    provider: Option[String],
    +    child: LogicalPlan,
    +    overwrite: Boolean = true)
    +  extends LogicalPlan {
    +
    +  override def children: Seq[LogicalPlan] = child :: Nil
    --- End diff --
    
    updated.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137919007
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1512,4 +1512,81 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    if (ctx.LOCAL != null) {
    +      throw new ParseException(
    +        "LOCAL is not supported in INSERT OVERWRITE DIRECTORY to data source", ctx)
    +    }
    +
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path).map(string).getOrElse("")
    +
    +    if (!(path.isEmpty ^ storage.locationUri.isEmpty)) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS should be specified one, but not both", ctx)
    --- End diff --
    
    When both are not specified?


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81346/
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136740379
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,132 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10")
    +
    +      sql(
    +        s"""
    +           |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +           |STORED AS orc
    +           |SELECT * FROM src where key < 10
    +         """.stripMargin)
    +
    +      // use orc data source to check the data of path is right.
    +      withTempView("orc_source") {
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY VIEW orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from src where key < 10").collect())
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to local dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val path = dir.toURI.getPath
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        withTempView("orc_source") {
    +          sql(
    +            s"""
    +               |CREATE TEMPORARY VIEW orc_source
    +               |USING org.apache.spark.sql.hive.orc
    +               |OPTIONS (
    +               |  PATH '${dir.getCanonicalPath}'
    +               |)
    +             """.stripMargin)
    +
    +          checkAnswer(
    +            sql("select * from orc_source"),
    +            sql("select * from test_insert_table").collect())
    +        }
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val pathUri = dir.toURI
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        withTempView("orc_source") {
    +          sql(
    +            s"""
    +               |CREATE TEMPORARY VIEW orc_source
    +               |USING org.apache.spark.sql.hive.orc
    +               |OPTIONS (
    +               |  PATH '${dir.getCanonicalPath}'
    +               |)
    +             """.stripMargin)
    +
    +          checkAnswer(
    +            sql("select * from orc_source"),
    --- End diff --
    
    You do not need the temp view `orc_source `. Below is an example.
    ```Scala
            checkAnswer(
              spark.read.orc(dir.getCanonicalPath),
              sql("select * from test_insert_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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136739126
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,145 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile {
    +
    +  override def children: Seq[LogicalPlan] = query :: Nil
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +
    +//    val Array(cols, types) = children.head.output.foldLeft(Array("", "")) { case (r, a) =>
    +//      r(0) = r(0) + a.name + ","
    +//      r(1) = r(1) + a.dataType.catalogString + ":"
    +//      r
    +//    }
    +//
    +//    val properties = new Properties()
    +//    properties.put("columns", cols.dropRight(1))
    +//    properties.put("columns.types", types.dropRight(1))
    +//    properties.put(serdeConstants.SERIALIZATION_LIB,
    +//      storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
    +//
    +//    import scala.collection.JavaConverters._
    +//    properties.putAll(storage.properties.asJava)
    +//
    +//    val tableDesc = new TableDesc(
    +//      Utils.classForName(storage.inputFormat.get).asInstanceOf[Class[_ <: InputFormat[_, _]]],
    +//      Utils.classForName(storage.outputFormat.get),
    +//      properties
    +//    )
    --- End diff --
    
    Yes. The following dummy table looks fine to me. Could you remove the above lines? Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    @gatorsmile Originally, because we have alot of hive sqls that we wanted to support in spark, I implemented hive syntax for this command:
    https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DML#LanguageManualDML-Writingdataintothefilesystemfromqueries
    
    But now I see that in SparkSqlParser.scala, we have both visitCreateTable and visitCreateHiveTable.
    I think we could implement both for this command.
    
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Since our native data source tables perform faster than the Hive serde tables, we expect our Spark users might prefer using data source tables. 
    
    Thanks for your work!


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137605940
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala ---
    @@ -0,0 +1,202 @@
    +/*
    + * 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.execution
    +
    +import java.io.{File, IOException}
    +import java.net.URI
    +import java.text.SimpleDateFormat
    +import java.util.{Date, Locale, Random}
    +
    +import scala.util.control.NonFatal
    +
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.exec.TaskRunner
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.hive.HiveExternalCatalog
    +import org.apache.spark.sql.hive.client.HiveVersion
    +
    +// Base trait for getting a temporary location for writing data
    +private[hive] trait HiveTmpPath extends Logging {
    +
    +  var createdTempDir: Option[Path] = None
    +
    +  def getExternalTmpPath(
    +      sparkSession: SparkSession,
    +      hadoopConf: Configuration,
    +      path: Path): Path = {
    +    import org.apache.spark.sql.hive.client.hive._
    +
    +    // Before Hive 1.1, when inserting into a table, Hive will create the staging directory under
    +    // a common scratch directory. After the writing is finished, Hive will simply empty the table
    +    // directory and move the staging directory to it.
    +    // After Hive 1.1, Hive will create the staging directory under the table directory, and when
    +    // moving staging directory to table directory, Hive will still empty the table directory, but
    +    // will exclude the staging directory there.
    +    // We have to follow the Hive behavior here, to avoid troubles. For example, if we create
    +    // staging directory under the table director for Hive prior to 1.1, the staging directory will
    +    // be removed by Hive when Hive is trying to empty the table directory.
    +    val hiveVersionsUsingOldExternalTempPath: Set[HiveVersion] = Set(v12, v13, v14, v1_0)
    +    val hiveVersionsUsingNewExternalTempPath: Set[HiveVersion] = Set(v1_1, v1_2, v2_0, v2_1)
    +
    +    // Ensure all the supported versions are considered here.
    +    assert(hiveVersionsUsingNewExternalTempPath ++ hiveVersionsUsingOldExternalTempPath ==
    +      allSupportedHiveVersions)
    +
    +    val externalCatalog = sparkSession.sharedState.externalCatalog
    +    val hiveVersion = externalCatalog.asInstanceOf[HiveExternalCatalog].client.version
    +    val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging")
    +    val scratchDir = hadoopConf.get("hive.exec.scratchdir", "/tmp/hive")
    +
    +    if (hiveVersionsUsingOldExternalTempPath.contains(hiveVersion)) {
    +      oldVersionExternalTempPath(path, hadoopConf, scratchDir)
    +    } else if (hiveVersionsUsingNewExternalTempPath.contains(hiveVersion)) {
    +      newVersionExternalTempPath(path, hadoopConf, stagingDir)
    +    } else {
    +      throw new IllegalStateException("Unsupported hive version: " + hiveVersion.fullVersion)
    +    }
    +  }
    +
    +  def deleteExternalTmpPath(hadoopConf: Configuration, stagingDir: String) : Unit = {
    --- End diff --
    
    updated.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/80959/
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137398869
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala ---
    @@ -360,6 +360,27 @@ case class InsertIntoTable(
     }
     
     /**
    + * Insert query result into a directory.
    + *
    + * @param isLocal Indicates whether the specified directory is local directory
    + * @param storage Info about output file, row and what serialization format
    + * @param provider Specifies what data source to use; only used for data source file.
    + * @param child The query to be executed
    + * @param overwrite If true, the existing directory will be overwritten
    + */
    +case class InsertIntoDir(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    provider: Option[String],
    +    child: LogicalPlan,
    +    overwrite: Boolean = true)
    +  extends LogicalPlan {
    +
    +  override def children: Seq[LogicalPlan] = child :: Nil
    +  override def output: Seq[Attribute] = Seq.empty
    --- End diff --
    
    updated


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134351219
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -2040,4 +2040,80 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
           assert(setOfPath.size() == pathSizeToDeleteOnExit)
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    import org.apache.spark.util.Utils
    +
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    checkAnswer(
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}' SELECT * FROM src where key < 10"),
    +      Seq.empty[Row])
    +
    +    checkAnswer(
    +      sql(s"""INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}'
    +             |STORED AS orc
    +             |SELECT * FROM src where key < 10""".stripMargin),
    +      Seq.empty[Row])
    +
    +    // use orc data source to check the data of path is right.
    +    sql(
    +      s"""CREATE TEMPORARY TABLE orc_source
    +         |USING org.apache.spark.sql.hive.orc
    +         |OPTIONS (
    +         |  PATH '${path.getCanonicalPath}'
    +         |)
    +       """.stripMargin)
    +    checkAnswer(
    +      sql("select * from orc_source"),
    +      sql("select * from src where key < 10").collect()
    +    )
    +
    +    Utils.deleteRecursively(path)
    +    dropTempTable("orc_source")
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    import org.apache.spark.util.Utils
    +
    +    sparkContext
    +      .parallelize(1 to 10)
    +      .map(i => TestData(i, i.toString))
    +      .toDF()
    +      .registerTempTable("test_insert_table")
    +
    +    val path = Utils.createTempDir()
    --- End diff --
    
    Please use `withTempDir`


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

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

    https://github.com/apache/spark/pull/18975#discussion_r134844461
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,67 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: Option[String],
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def innerChildren: Seq[LogicalPlan] = Seq(query)
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    assert(innerChildren.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +    assert(provider.isDefined)
    +
    +    // Create the relation based on the input logical plan: `data`.
    +    val pathOption = storage.locationUri.map("path" -> CatalogUtils.URIToString(_))
    +    val dataSource = DataSource(
    +      sparkSession,
    +      className = provider.get,
    +      options = storage.properties ++ pathOption,
    +      catalogTable = None)
    +
    +    val saveMode = if (overwrite) SaveMode.Overwrite else SaveMode.ErrorIfExists
    +    try {
    +      dataSource.writeAndRead(saveMode, query)
    --- End diff --
    
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134352626
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala ---
    @@ -142,10 +142,14 @@ object UnsupportedOperationChecker {
                 "Distinct aggregations are not supported on streaming DataFrames/Datasets. Consider " +
                   "using approx_count_distinct() instead.")
     
    +
    --- End diff --
    
    Nit: revert it back.


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

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

    https://github.com/apache/spark/pull/18975#discussion_r134351695
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -2040,4 +2040,80 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
           assert(setOfPath.size() == pathSizeToDeleteOnExit)
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    import org.apache.spark.util.Utils
    +
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    checkAnswer(
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}' SELECT * FROM src where key < 10"),
    +      Seq.empty[Row])
    +
    +    checkAnswer(
    +      sql(s"""INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}'
    +             |STORED AS orc
    +             |SELECT * FROM src where key < 10""".stripMargin),
    +      Seq.empty[Row])
    +
    +    // use orc data source to check the data of path is right.
    +    sql(
    +      s"""CREATE TEMPORARY TABLE orc_source
    +         |USING org.apache.spark.sql.hive.orc
    +         |OPTIONS (
    +         |  PATH '${path.getCanonicalPath}'
    +         |)
    +       """.stripMargin)
    +    checkAnswer(
    +      sql("select * from orc_source"),
    +      sql("select * from src where key < 10").collect()
    +    )
    +
    +    Utils.deleteRecursively(path)
    +    dropTempTable("orc_source")
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    import org.apache.spark.util.Utils
    +
    +    sparkContext
    +      .parallelize(1 to 10)
    +      .map(i => TestData(i, i.toString))
    +      .toDF()
    +      .registerTempTable("test_insert_table")
    --- End diff --
    
    Let us make it shorter.
    
    ```Scala
        spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_dir")
    ```


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

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

    https://github.com/apache/spark/pull/18975#discussion_r134842831
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1509,4 +1509,84 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path) match {
    --- End diff --
    
    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 issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Yeah. I am OK to use staging directories instead of directly writing to the final destination. 


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81338 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81338/testReport)** for PR 18975 at commit [`52350e8`](https://github.com/apache/spark/commit/52350e8adb7b07540f635c103f36e065c151f8cd).


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

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

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


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137460842
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,81 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + *
    + * @param storage storage format used to describe how the query result is stored.
    + * @param provider the data source type to be used
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: String,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def innerChildren: Seq[LogicalPlan] = Seq(query)
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    --- End diff --
    
    `run(sparkSession: SparkSession, children: Seq[SparkPlan])`


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137929284
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1512,4 +1512,81 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    if (ctx.LOCAL != null) {
    +      throw new ParseException(
    +        "LOCAL is not supported in INSERT OVERWRITE DIRECTORY to data source", ctx)
    --- End diff --
    
    Originally, LOCAL was not added.
    @gatorsmile had some comment that the parser might have some weird exception and he requested to add it.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Jenkin test 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Thanks for working on it! 
    
    We also need to add plan-parsing unit test cases in `DDLCommandSuite`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    nvm. I think you already said [renaming might break the atomicity](https://github.com/apache/spark/pull/18975#issuecomment-327017349). Users still can get the wrong results in the final destination. Hive is unable to guarantee `all or nothing`. Is my understanding right?


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134683419
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,67 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: Option[String],
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def innerChildren: Seq[LogicalPlan] = Seq(query)
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    assert(innerChildren.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +    assert(provider.isDefined)
    +
    +    // Create the relation based on the input logical plan: `data`.
    +    val pathOption = storage.locationUri.map("path" -> CatalogUtils.URIToString(_))
    +    val dataSource = DataSource(
    +      sparkSession,
    +      className = provider.get,
    +      options = storage.properties ++ pathOption,
    +      catalogTable = None)
    +
    +    val saveMode = if (overwrite) SaveMode.Overwrite else SaveMode.ErrorIfExists
    +    try {
    +      dataSource.writeAndRead(saveMode, query)
    --- End diff --
    
    Since we do not need to return BaseRelation, we can call `planForWriting` and execute the plan


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81297/
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137918792
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -192,12 +245,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
             "partitions with value: " + dynamicPartitionKeys.keys.mkString("[", ",", "]"), ctx)
         }
     
    -    InsertIntoTable(
    -      UnresolvedRelation(tableIdent),
    -      partitionKeys,
    -      query,
    -      ctx.OVERWRITE != null,
    -      ctx.EXISTS != null)
    +    (tableIdent, partitionKeys, ctx.OVERWRITE() != null, ctx.EXISTS() != null)
    --- End diff --
    
    You have `InsertOverwriteTableContext` and `InsertIntoTableContext` actually, that said you don't need an `overwrite` in `InsertTableParams`. You already know whether to overwrite before visiting them.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137918931
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala ---
    @@ -360,6 +360,31 @@ case class InsertIntoTable(
     }
     
     /**
    + * Insert query result into a directory.
    + *
    + * @param isLocal Indicates whether the specified directory is local directory
    + * @param storage Info about output file, row and what serialization format
    + * @param provider Specifies what data source to use; only used for data source file.
    + * @param child The query to be executed
    + * @param overwrite If true, the existing directory will be overwritten
    + *
    + * Note that this plan is unresolved and has to be replaced by the concrete implementations
    + * during analysis.
    + */
    +case class InsertIntoDir(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    provider: Option[String],
    +    child: LogicalPlan,
    +    overwrite: Boolean = true)
    +  extends LogicalPlan {
    +
    +  override def children: Seq[LogicalPlan] = child :: Nil
    --- End diff --
    
    Nit: We can simply extend `UnaryNode` and remove `children`.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134684564
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -2040,4 +2040,83 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
           assert(setOfPath.size() == pathSizeToDeleteOnExit)
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      checkAnswer(
    +        sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10"),
    +        Seq.empty[Row])
    +
    +      checkAnswer(
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |STORED AS orc
    +             |SELECT * FROM src where key < 10
    +          """.stripMargin),
    +        Seq.empty[Row])
    +
    +      // use orc data source to check the data of path is right.
    +      sql(
    +        s"""
    +           |CREATE TEMPORARY TABLE orc_source
    +           |USING org.apache.spark.sql.hive.orc
    +           |OPTIONS (
    +           |  PATH '${dir.getCanonicalPath}'
    +           |)
    +        """.stripMargin)
    +
    +      checkAnswer(
    +        sql("select * from orc_source"),
    +        sql("select * from src where key < 10").collect())
    +
    +      dropTempTable("orc_source")
    +    }
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val path = dir.toURI.getPath
    +
    +        checkAnswer(
    +          sql(
    +            s"""
    +               |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +               |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +               |SELECT * FROM test_insert_table
    +             """.stripMargin),
    +          Seq.empty[Row])
    +
    +        checkAnswer(
    +          sql(
    +            s"""
    +               |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +               |STORED AS orc
    +               |SELECT * FROM test_insert_table
    +             """.stripMargin),
    +          Seq.empty[Row])
    +
    +        // use orc data source to check the data of path is right.
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY TABLE orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from test_insert_table").collect())
    +
    +        dropTempTable("orc_source")
    --- End diff --
    
    Also put it into `withTempView `


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

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

    https://github.com/apache/spark/pull/18975#discussion_r134381632
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala ---
    @@ -359,6 +359,17 @@ case class InsertIntoTable(
       override lazy val resolved: Boolean = false
     }
     
    +case class InsertIntoDir(
    --- End diff --
    
    ok. 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 pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137929025
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -192,12 +245,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
             "partitions with value: " + dynamicPartitionKeys.keys.mkString("[", ",", "]"), ctx)
         }
     
    -    InsertIntoTable(
    -      UnresolvedRelation(tableIdent),
    -      partitionKeys,
    -      query,
    -      ctx.OVERWRITE != null,
    -      ctx.EXISTS != null)
    +    (tableIdent, partitionKeys, ctx.OVERWRITE() != null, ctx.EXISTS() != null)
    +  }
    +
    +  /**
    +   * Write to a file, returning a [[InsertIntoDir]] logical plan.
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    throw new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx)
    +  }
    +
    +  /**
    +   * Write to a file, returning a [[InsertIntoDir]] logical plan.
    --- End diff --
    
    updated


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137929022
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -192,12 +245,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
             "partitions with value: " + dynamicPartitionKeys.keys.mkString("[", ",", "]"), ctx)
         }
     
    -    InsertIntoTable(
    -      UnresolvedRelation(tableIdent),
    -      partitionKeys,
    -      query,
    -      ctx.OVERWRITE != null,
    -      ctx.EXISTS != null)
    +    (tableIdent, partitionKeys, ctx.OVERWRITE() != null, ctx.EXISTS() != null)
    +  }
    +
    +  /**
    +   * Write to a file, returning a [[InsertIntoDir]] logical plan.
    --- End diff --
    
    updated.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    @gatorsmile plan-pasring unittests are already added in DDLCommandSuite.


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

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

    https://github.com/apache/spark/pull/18975#discussion_r136502487
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,115 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10")
    +
    +      sql(
    +        s"""
    +           |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +           |STORED AS orc
    +           |SELECT * FROM src where key < 10
    +         """.stripMargin)
    +
    +      // use orc data source to check the data of path is right.
    +      withTempView("orc_source") {
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY VIEW orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from src where key < 10").collect())
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to local dir from temp table") {
    --- End diff --
    
    In addition, also need to add negative test cases. For example, the path is illegal. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81050/
    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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81050 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81050/testReport)** for PR 18975 at commit [`64f37f4`](https://github.com/apache/spark/commit/64f37f4a2517fb841a47d6fb105e01f10e700fc3).
     * 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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137461450
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,150 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10")
    +
    +      sql(
    +        s"""
    +           |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +           |STORED AS orc
    +           |SELECT * FROM src where key < 10
    +         """.stripMargin)
    +
    +      // use orc data source to check the data of path is right.
    +      withTempView("orc_source") {
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY VIEW orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from src where key < 10"))
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to local dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val path = dir.toURI.getPath
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        checkAnswer(
    +          spark.read.orc(dir.getCanonicalPath),
    +          sql("select * from test_insert_table"))
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val pathUri = dir.toURI
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        checkAnswer(
    +          spark.read.orc(dir.getCanonicalPath),
    +          sql("select * from test_insert_table"))
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to dir to illegal path") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      val e = intercept[IllegalArgumentException] {
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY 'abc://a'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +      }.getMessage
    +
    +      assert(e.contains("Wrong FS: abc://a, expected: file:///"))
    +    }
    +  }
    +
    +  test("insert overwrite to dir with mixed syntax") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      val e = intercept[ParseException] {
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY 'file://tmp'
    +             |USING json
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +      }.getMessage
    +
    +      assert(e.contains("mismatched input 'ROW'"))
    +    }
    +  }
    +
    +  test("insert overwrite to dir with multi inserts") {
    --- End diff --
    
    ```
    Hive extension (multiple inserts):
    FROM from_statement
    INSERT OVERWRITE [LOCAL] DIRECTORY directory1 select_statement1
    [INSERT OVERWRITE [LOCAL] DIRECTORY directory2 select_statement2] ...
    ```
    
    Could you add a test case like this?


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136421034
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,65 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: Option[String],
    --- End diff --
    
    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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136739964
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,132 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10")
    +
    +      sql(
    +        s"""
    +           |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +           |STORED AS orc
    +           |SELECT * FROM src where key < 10
    +         """.stripMargin)
    +
    +      // use orc data source to check the data of path is right.
    +      withTempView("orc_source") {
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY VIEW orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from src where key < 10").collect())
    --- End diff --
    
    Nit: `.collect()` can be 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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136439466
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,83 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      checkAnswer(
    +        sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10"),
    +        Seq.empty[Row])
    +
    +      checkAnswer(
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |STORED AS orc
    +             |SELECT * FROM src where key < 10
    +          """.stripMargin),
    +        Seq.empty[Row])
    +
    +      // use orc data source to check the data of path is right.
    +      withTempView("orc_source") {
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY VIEW orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from src where key < 10").collect())
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val path = dir.toURI.getPath
    +
    +        checkAnswer(
    +          sql(
    +            s"""
    +               |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    --- End diff --
    
    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 pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137063179
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,132 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10")
    +
    +      sql(
    +        s"""
    +           |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +           |STORED AS orc
    +           |SELECT * FROM src where key < 10
    +         """.stripMargin)
    +
    +      // use orc data source to check the data of path is right.
    +      withTempView("orc_source") {
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY VIEW orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from src where key < 10").collect())
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to local dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val path = dir.toURI.getPath
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        withTempView("orc_source") {
    +          sql(
    +            s"""
    +               |CREATE TEMPORARY VIEW orc_source
    +               |USING org.apache.spark.sql.hive.orc
    +               |OPTIONS (
    +               |  PATH '${dir.getCanonicalPath}'
    +               |)
    +             """.stripMargin)
    +
    +          checkAnswer(
    +            sql("select * from orc_source"),
    +            sql("select * from test_insert_table").collect())
    +        }
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val pathUri = dir.toURI
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        withTempView("orc_source") {
    +          sql(
    +            s"""
    +               |CREATE TEMPORARY VIEW orc_source
    +               |USING org.apache.spark.sql.hive.orc
    +               |OPTIONS (
    +               |  PATH '${dir.getCanonicalPath}'
    +               |)
    +             """.stripMargin)
    +
    +          checkAnswer(
    +            sql("select * from orc_source"),
    +            sql("select * from test_insert_table").collect())
    --- End diff --
    
    removed


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134352209
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,65 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: Option[String],
    +    query: LogicalPlan) extends RunnableCommand {
    +
    +  override def innerChildren: Seq[LogicalPlan] = Seq(query)
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    assert(innerChildren.length == 1)
    +    assert(!storage.locationUri.isEmpty)
    --- End diff --
    
    `assert(storage.locationUri.nonEmpty)`


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137605602
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala ---
    @@ -366,4 +367,43 @@ class InsertSuite extends DataSourceTest with SharedSQLContext {
             Row(Array(1, 2), Array("a", "b")))
         }
       }
    +
    +  test("insert overwrite directory") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      val v1 =
    +        s"""
    +           | INSERT OVERWRITE DIRECTORY '${path}'
    +           | USING json
    +           | OPTIONS (a 1, b 0.1, c TRUE)
    --- End diff --
    
    added


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    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 pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134383033
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1509,4 +1509,84 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path) match {
    +      case Some(s) => string(s)
    +      case None => ""
    +    }
    +
    +    if (!path.isEmpty && storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " +
    +          "you can only specify one of them.", ctx)
    +    }
    +    if (path.isEmpty && !storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "You need to specify directory path or 'path' in OPTIONS, but not both", ctx)
    +    }
    +
    +    if (!path.isEmpty) {
    +      val customLocation = Some(CatalogUtils.stringToURI(path))
    +      storage = storage.copy(locationUri = customLocation)
    +    }
    +
    +    val provider = ctx.tableProvider.qualifiedName.getText
    +
    +    (false, storage, Some(provider))
    +  }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   path
    --- End diff --
    
    added [LOCAL]


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    @gatorsmile: If the query fails in the middle (eg. tasks are OOMing), hive would have written data to the staging location and not the final output location. So users wont see this partial data. 
    
    Over here, we are directly telling tasks to write to the final output location. So if there are few tasks completed, their output is in the final output location. If the remaining tasks hit issues which lead to job failure, then users are left with partial output.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137398776
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.execution
    +
    +import java.io.{File, IOException}
    +import java.net.URI
    +import java.text.SimpleDateFormat
    +import java.util.{Date, Locale, Random}
    +
    +import scala.util.control.NonFatal
    +
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.exec.TaskRunner
    +
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.execution.command.RunnableCommand
    +import org.apache.spark.sql.hive.HiveExternalCatalog
    +import org.apache.spark.sql.hive.client.HiveVersion
    +
    +// Base trait for getting a temporary location for writing data
    +private[hive] trait HiveTmpPath extends RunnableCommand {
    --- End diff --
    
    removed RunnableCommand, the classes that exten HiveTmpPath already extends RunnableCommand


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134350779
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.execution
    +
    +import java.util.Properties
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.util.Utils
    +
    +
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    --- End diff --
    
    also add a parameter `overwrite: Boolean,`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    ok to test


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137605509
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala ---
    @@ -140,6 +143,14 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast
             parts, query, overwrite, false) if parts.isEmpty =>
           InsertIntoDataSourceCommand(l, query, overwrite)
     
    +    case InsertIntoDir(isLocal, storage, provider, query, overwrite)
    --- End diff --
    
    updated.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137401845
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala ---
    @@ -0,0 +1,78 @@
    +/*
    + * 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.execution
    +
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.internal.io.FileCommitProtocol
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.catalyst.catalog.BucketSpec
    +import org.apache.spark.sql.catalyst.expressions.Attribute
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.execution.command.DataWritingCommand
    +import org.apache.spark.sql.execution.datasources.FileFormatWriter
    +import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc}
    +
    +// Base trait from which all hive insert statement physical execution extends.
    +private[hive] trait SaveAsHiveFile extends DataWritingCommand {
    +
    +  protected def saveAsHiveFile(
    +      sparkSession: SparkSession,
    +      plan: SparkPlan,
    +      hadoopConf: Configuration,
    +      fileSinkConf: FileSinkDesc,
    +      outputLocation: String,
    +      partitionAttributes: Seq[Attribute] = Nil,
    +      bucketSpec: Option[BucketSpec] = None,
    +      options: Map[String, String] = Map.empty): Unit = {
    --- End diff --
    
    remove `bucketSpec ` and `options `? Add them back only when we need it?


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137057860
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -178,11 +179,50 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider).
    +   */
    +  type InsertDirParams = (Boolean, CatalogStorageFormat, Option[String])
    +
    +  /**
    +   * Add an
    +   *   INSERT INTO [TABLE] or
    +   *   INSERT OVERWRITE TABLE or
    +   *   INSERT OVERWRITE [LOCAL] DIRECTORY
    +   * operation to logical plan
        */
       private def withInsertInto(
           ctx: InsertIntoContext,
           query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    assert(ctx.children.size == 1)
    --- End diff --
    
    @gatorsmile, yes, that is more concise. Updated the code.
    @tejasapatil added testcases.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136680384
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,115 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10")
    +
    +      sql(
    +        s"""
    +           |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +           |STORED AS orc
    +           |SELECT * FROM src where key < 10
    +         """.stripMargin)
    +
    +      // use orc data source to check the data of path is right.
    +      withTempView("orc_source") {
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY VIEW orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from src where key < 10").collect())
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to local dir from temp table") {
    --- End diff --
    
    Yes, we have tested path in HDFS.  Added a negative cases.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137397811
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala ---
    @@ -178,11 +179,60 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
       }
     
       /**
    -   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   * Parameters used for writing query to a table:
    +   *   (tableIdentifier, partitionKeys, overwrite, exists).
    +   */
    +  type InsertTableParams = (TableIdentifier, Map[String, Option[String]], Boolean, Boolean)
    +
    +  /**
    +   * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider).
    +   */
    +  type InsertDirParams = (Boolean, CatalogStorageFormat, Option[String])
    +
    +  /**
    +   * Add an
    +   *   INSERT INTO [TABLE] or
    +   *   INSERT OVERWRITE TABLE or
    +   *   INSERT OVERWRITE [LOCAL] DIRECTORY
    --- End diff --
    
    added


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #80874 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80874/testReport)** for PR 18975 at commit [`7f5664d`](https://github.com/apache/spark/commit/7f5664dcf5e55f3cc2ad91a000bb2c8ab58f596a).
     * 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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134381489
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -740,6 +750,7 @@ nonReserved
         | AND | CASE | CAST | DISTINCT | DIV | ELSE | END | FUNCTION | INTERVAL | MACRO | OR | STRATIFY | THEN
         | UNBOUNDED | WHEN
         | DATABASE | SELECT | FROM | WHERE | HAVING | TO | TABLE | WITH | NOT | CURRENT_DATE | CURRENT_TIMESTAMP
    +    | DIRECTORY
    --- End diff --
    
    it is already in TableIdentifierParserSuite


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

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

    https://github.com/apache/spark/pull/18975#discussion_r134382822
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -2040,4 +2040,80 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
           assert(setOfPath.size() == pathSizeToDeleteOnExit)
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    import org.apache.spark.util.Utils
    +
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    checkAnswer(
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}' SELECT * FROM src where key < 10"),
    +      Seq.empty[Row])
    +
    +    checkAnswer(
    +      sql(s"""INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}'
    +             |STORED AS orc
    +             |SELECT * FROM src where key < 10""".stripMargin),
    +      Seq.empty[Row])
    +
    +    // use orc data source to check the data of path is right.
    +    sql(
    +      s"""CREATE TEMPORARY TABLE orc_source
    +         |USING org.apache.spark.sql.hive.orc
    +         |OPTIONS (
    +         |  PATH '${path.getCanonicalPath}'
    +         |)
    +       """.stripMargin)
    +    checkAnswer(
    +      sql("select * from orc_source"),
    +      sql("select * from src where key < 10").collect()
    +    )
    +
    +    Utils.deleteRecursively(path)
    +    dropTempTable("orc_source")
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    import org.apache.spark.util.Utils
    +
    +    sparkContext
    +      .parallelize(1 to 10)
    +      .map(i => TestData(i, i.toString))
    +      .toDF()
    +      .registerTempTable("test_insert_table")
    +
    +    val path = Utils.createTempDir()
    --- End diff --
    
    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 issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Will review it tonight.


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

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

    https://github.com/apache/spark/pull/18975#discussion_r134680856
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1509,4 +1509,84 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path) match {
    --- End diff --
    
    Use `.map` and `.getOrElse` ?


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

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

    https://github.com/apache/spark/pull/18975#discussion_r138681313
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whether overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile with HiveTmpPath {
    --- End diff --
    
    @cloud-fan and gatorsmile, I will merge them together and submit a PR.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137929328
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whether overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile with HiveTmpPath {
    +
    +  override def children: Seq[LogicalPlan] = query :: Nil
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +
    +    val hiveTable = HiveClientImpl.toHiveTable(CatalogTable(
    +      identifier = TableIdentifier(storage.locationUri.get.toString, Some("default")),
    +      tableType = org.apache.spark.sql.catalyst.catalog.CatalogTableType.VIEW,
    +      storage = storage,
    +      schema = query.schema
    +    ))
    +    hiveTable.getMetadata.put(serdeConstants.SERIALIZATION_LIB,
    +      storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
    +
    +    val tableDesc = new TableDesc(
    +      hiveTable.getInputFormatClass,
    +      hiveTable.getOutputFormatClass,
    +      hiveTable.getMetadata
    +    )
    +
    +    val hadoopConf = sparkSession.sessionState.newHadoopConf()
    +    val jobConf = new JobConf(hadoopConf)
    +
    +    val targetPath = new Path(storage.locationUri.get)
    +    val writeToPath =
    +      if (isLocal) {
    +        val localFileSystem = FileSystem.getLocal(jobConf)
    +        localFileSystem.makeQualified(targetPath)
    +      } else {
    +        val qualifiedPath = FileUtils.makeQualified(targetPath, hadoopConf)
    +        val dfs = qualifiedPath.getFileSystem(jobConf)
    +        if (!dfs.exists(qualifiedPath)) {
    +          dfs.mkdirs(qualifiedPath.getParent)
    +        }
    +        qualifiedPath
    +      }
    +
    +    val tmpPath = getExternalTmpPath(sparkSession, hadoopConf, writeToPath)
    +    val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc(
    +      tmpPath.toString, tableDesc, false)
    +
    +    try {
    +      saveAsHiveFile(
    +        sparkSession = sparkSession,
    +        plan = children.head,
    +        hadoopConf = hadoopConf,
    +        fileSinkConf = fileSinkConf,
    +        outputLocation = tmpPath.toString)
    +
    +      val fs = writeToPath.getFileSystem(hadoopConf)
    +      if (overwrite && fs.exists(writeToPath)) {
    +        fs.listStatus(writeToPath).foreach { existFile =>
    +          if (Option(existFile.getPath) != createdTempDir) fs.delete(existFile.getPath, true)
    +        }
    +      }
    +
    +      fs.listStatus(tmpPath).foreach {
    +        tmpFile => fs.rename(tmpFile.getPath, writeToPath)
    +      }
    +
    +      deleteExternalTmpPath(hadoopConf)
    --- End diff --
    
    good point. updated.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81333 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81333/testReport)** for PR 18975 at commit [`7ccbde4`](https://github.com/apache/spark/commit/7ccbde47fffba7bef8eceba4993bbd70eeb84845).
     * 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 issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #80917 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80917/testReport)** for PR 18975 at commit [`068662a`](https://github.com/apache/spark/commit/068662a5abaaa693529320bb855b7a3323915bf8).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class AnalyzePartitionCommand(`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81338/
    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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137449077
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala ---
    @@ -524,6 +525,50 @@ class DDLParserSuite extends PlanTest with SharedSQLContext {
         assert(e.message.contains("you can only specify one of them."))
       }
     
    +  test("insert overwrite directory") {
    +    val v1 = "INSERT OVERWRITE DIRECTORY '/tmp/file' USING parquet SELECT 1 as a"
    +    parser.parsePlan(v1) match {
    +      case InsertIntoDir(_, storage, provider, query, overwrite) =>
    +        assert(storage.locationUri != None && storage.locationUri.get.toString == "/tmp/file")
    --- End diff --
    
    Nit:
    ```Scala
    assert(storage.locationUri.isDefined && storage.locationUri.get.toString == "/tmp/file")
    ```


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134382831
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -2040,4 +2040,80 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
           assert(setOfPath.size() == pathSizeToDeleteOnExit)
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    import org.apache.spark.util.Utils
    +
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    checkAnswer(
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}' SELECT * FROM src where key < 10"),
    +      Seq.empty[Row])
    +
    +    checkAnswer(
    +      sql(s"""INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}'
    +             |STORED AS orc
    +             |SELECT * FROM src where key < 10""".stripMargin),
    +      Seq.empty[Row])
    +
    +    // use orc data source to check the data of path is right.
    +    sql(
    +      s"""CREATE TEMPORARY TABLE orc_source
    +         |USING org.apache.spark.sql.hive.orc
    +         |OPTIONS (
    +         |  PATH '${path.getCanonicalPath}'
    +         |)
    +       """.stripMargin)
    +    checkAnswer(
    +      sql("select * from orc_source"),
    +      sql("select * from src where key < 10").collect()
    +    )
    +
    +    Utils.deleteRecursively(path)
    +    dropTempTable("orc_source")
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    import org.apache.spark.util.Utils
    +
    +    sparkContext
    +      .parallelize(1 to 10)
    +      .map(i => TestData(i, i.toString))
    +      .toDF()
    +      .registerTempTable("test_insert_table")
    +
    +    val path = Utils.createTempDir()
    +    path.delete()
    +    checkAnswer(
    +      sql(
    +        s"""
    +           |INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}'
    +           |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +           |SELECT * FROM test_insert_table
    +         """.stripMargin),
    +      Seq.empty[Row])
    +
    +    checkAnswer(
    +      sql(s"""
    +        INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table""".stripMargin),
    +      Seq.empty[Row])
    +
    +    // use orc data source to check the data of path is right.
    +    sql(
    +      s"""CREATE TEMPORARY TABLE orc_source
    +         |USING org.apache.spark.sql.hive.orc
    +         |OPTIONS (
    +         |  PATH '${path.getCanonicalPath}'
    +         |)
    +       """.stripMargin)
    +    checkAnswer(
    +      sql("select * from orc_source"),
    +      sql("select * from test_insert_table").collect()
    +    )
    +    Utils.deleteRecursively(path)
    +    dropTempTable("test_insert_table")
    --- End diff --
    
    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 issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #80959 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80959/testReport)** for PR 18975 at commit [`0882dd1`](https://github.com/apache/spark/commit/0882dd1f3c300f832d731b69a0d57ef461e55038).


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

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

    https://github.com/apache/spark/pull/18975#discussion_r134350813
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,65 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: Option[String],
    --- End diff --
    
    also add a parameter `overwrite: Boolean,`.


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

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

    https://github.com/apache/spark/pull/18975#discussion_r136739990
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,132 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10")
    +
    +      sql(
    +        s"""
    +           |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +           |STORED AS orc
    +           |SELECT * FROM src where key < 10
    +         """.stripMargin)
    +
    +      // use orc data source to check the data of path is right.
    +      withTempView("orc_source") {
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY VIEW orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from src where key < 10").collect())
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to local dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val path = dir.toURI.getPath
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        withTempView("orc_source") {
    +          sql(
    +            s"""
    +               |CREATE TEMPORARY VIEW orc_source
    +               |USING org.apache.spark.sql.hive.orc
    +               |OPTIONS (
    +               |  PATH '${dir.getCanonicalPath}'
    +               |)
    +             """.stripMargin)
    +
    +          checkAnswer(
    +            sql("select * from orc_source"),
    +            sql("select * from test_insert_table").collect())
    +        }
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val pathUri = dir.toURI
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        withTempView("orc_source") {
    +          sql(
    +            s"""
    +               |CREATE TEMPORARY VIEW orc_source
    +               |USING org.apache.spark.sql.hive.orc
    +               |OPTIONS (
    +               |  PATH '${dir.getCanonicalPath}'
    +               |)
    +             """.stripMargin)
    +
    +          checkAnswer(
    +            sql("select * from orc_source"),
    +            sql("select * from test_insert_table").collect())
    --- 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 issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Looks pretty good! Will do the final pass after addressing the above comments. 
    
    Thanks for your work!


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134030622
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1499,4 +1500,34 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE or INSERT INTO [LOCAL] DIRECOTRY
    +   * operation to the logical plan.
    +   */
    +  protected override def withInsertInto(ctx: InsertIntoContext,
    +                                        query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    --- End diff --
    
    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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Just left a comment : https://github.com/apache/spark/pull/18975#discussion_r136256422


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137452336
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala ---
    @@ -524,6 +525,50 @@ class DDLParserSuite extends PlanTest with SharedSQLContext {
         assert(e.message.contains("you can only specify one of them."))
       }
     
    +  test("insert overwrite directory") {
    +    val v1 = "INSERT OVERWRITE DIRECTORY '/tmp/file' USING parquet SELECT 1 as a"
    +    parser.parsePlan(v1) match {
    +      case InsertIntoDir(_, storage, provider, query, overwrite) =>
    +        assert(storage.locationUri != None && storage.locationUri.get.toString == "/tmp/file")
    +      case other =>
    +        fail(s"Expected to parse ${classOf[InsertIntoDataSourceDirCommand].getClass.getName}" +
    +          " from query," + s"got ${other.getClass.getName}: $v1")
    +    }
    +
    +    val v2 = "INSERT OVERWRITE DIRECTORY USING parquet SELECT 1 as a"
    +    val e2 = intercept[ParseException] {
    +      parser.parsePlan(v2)
    +    }
    +    assert(e2.message.contains(
    +      "Directory path and 'path' in OPTIONS should be specified one, but not both"))
    +
    +    val v3 =
    +      """
    +        | INSERT OVERWRITE DIRECTORY USING json
    +        | OPTIONS ('path' '/tmp/file', a 1, b 0.1, c TRUE)
    +        | SELECT 1 as a
    +      """.stripMargin
    +    parser.parsePlan(v3) match {
    +      case InsertIntoDir(_, storage, provider, query, overwrite) =>
    +        assert(storage.locationUri != None && provider == Some("json"))
    --- End diff --
    
    Nit: the same here


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    LGTM pending Jenkins
    
    Thanks again!


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81049 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81049/testReport)** for PR 18975 at commit [`f36e933`](https://github.com/apache/spark/commit/f36e933f47163b0f9040087e4aeee582e0ccd9a4).
     * 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136256422
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1509,4 +1509,84 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path) match {
    +      case Some(s) => string(s)
    +      case None => ""
    +    }
    +
    +    if (!path.isEmpty && storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " +
    +          "you can only specify one of them.", ctx)
    +    }
    +    if (path.isEmpty && !storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "You need to specify directory path or 'path' in OPTIONS, but not both", ctx)
    +    }
    +
    +    if (!path.isEmpty) {
    +      val customLocation = Some(CatalogUtils.stringToURI(path))
    +      storage = storage.copy(locationUri = customLocation)
    +    }
    +
    +    val provider = ctx.tableProvider.qualifiedName.getText
    +
    +    (false, storage, Some(provider))
    +  }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE [LOCAL] DIRECTORY
    +   *   path
    +   *   [ROW FORMAT row_format]
    +   *   [STORED AS file_format]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteHiveDir(
    +      ctx: InsertOverwriteHiveDirContext): InsertDirParams = withOrigin(ctx) {
    +    validateRowFormatFileFormat(ctx.rowFormat, ctx.createFileFormat, ctx)
    +    val rowStorage = Option(ctx.rowFormat).map(visitRowFormat)
    +      .getOrElse(CatalogStorageFormat.empty)
    +    val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat)
    +      .getOrElse(CatalogStorageFormat.empty)
    +
    +    val path = string(ctx.path)
    +    // The path field is required
    +    if (path.isEmpty) {
    +      operationNotAllowed("INSERT OVERWRITE DIRECTORY must be accompanied by path", ctx)
    +    }
    +
    +    val defaultStorage = HiveSerDe.getDefaultStorage(conf)
    +
    +    val storage = CatalogStorageFormat(
    +      locationUri = Some(CatalogUtils.stringToURI(path)),
    +      inputFormat = fileStorage.inputFormat.orElse(defaultStorage.inputFormat),
    +      outputFormat = fileStorage.outputFormat.orElse(defaultStorage.outputFormat),
    +      serde = rowStorage.serde.orElse(fileStorage.serde).orElse(defaultStorage.serde),
    +      compressed = false,
    +      properties = rowStorage.properties ++ fileStorage.properties)
    +
    +    (ctx.LOCAL != null, storage, None)
    --- End diff --
    
    This command should be equivalent to the above command with the `USING hive` clause. 
    
    We need to make these two methods interchangeable for Hive commands. See what we did in the following example: https://github.com/apache/spark/blob/master/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala#L1440-L1448
    



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

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

    https://github.com/apache/spark/pull/18975#discussion_r134905838
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1509,4 +1509,84 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path) match {
    +      case Some(s) => string(s)
    +      case None => ""
    +    }
    +
    +    if (!path.isEmpty && storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " +
    +          "you can only specify one of them.", ctx)
    +    }
    +    if (path.isEmpty && !storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "You need to specify directory path or 'path' in OPTIONS, but not both", ctx)
    +    }
    +
    +    if (!path.isEmpty) {
    +      val customLocation = Some(CatalogUtils.stringToURI(path))
    +      storage = storage.copy(locationUri = customLocation)
    +    }
    +
    +    val provider = ctx.tableProvider.qualifiedName.getText
    +
    +    (false, storage, Some(provider))
    +  }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE [LOCAL] DIRECTORY
    +   *   path
    +   *   [ROW FORMAT row_format]
    +   *   [STORED AS file_format]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteHiveDir(
    +      ctx: InsertOverwriteHiveDirContext): InsertDirParams = withOrigin(ctx) {
    +    validateRowFormatFileFormat(ctx.rowFormat, ctx.createFileFormat, ctx)
    +    val rowStorage = Option(ctx.rowFormat).map(visitRowFormat)
    +      .getOrElse(CatalogStorageFormat.empty)
    +    val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat)
    +      .getOrElse(CatalogStorageFormat.empty)
    +
    +    val path = string(ctx.path)
    +    // The path field is required
    +    if (path.isEmpty) {
    +      operationNotAllowed("INSERT OVERWRITE DIRECTORY must be accompanied by path", ctx)
    +    }
    +
    +    val defaultStorage = HiveSerDe.getDefaultStorage(conf)
    +
    +    val storage = CatalogStorageFormat(
    +      locationUri = Some(CatalogUtils.stringToURI(path)),
    +      inputFormat = fileStorage.inputFormat.orElse(defaultStorage.inputFormat),
    +      outputFormat = fileStorage.outputFormat.orElse(defaultStorage.outputFormat),
    +      serde = rowStorage.serde.orElse(fileStorage.serde).orElse(defaultStorage.serde),
    +      compressed = false,
    +      properties = rowStorage.properties ++ fileStorage.properties)
    +
    +    (ctx.LOCAL != null, storage, None)
    --- End diff --
    
    I see. But we use provide == None to detect it is a hive command. Only when USING hive is specified, we using data source command.



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

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

    https://github.com/apache/spark/pull/18975#discussion_r137680545
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,176 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10")
    +
    +      sql(
    +        s"""
    +           |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +           |STORED AS orc
    +           |SELECT * FROM src where key < 10
    +         """.stripMargin)
    +
    +      // use orc data source to check the data of path is right.
    +      withTempView("orc_source") {
    +        sql(
    +          s"""
    +             |CREATE TEMPORARY VIEW orc_source
    +             |USING org.apache.spark.sql.hive.orc
    +             |OPTIONS (
    +             |  PATH '${dir.getCanonicalPath}'
    +             |)
    +           """.stripMargin)
    +
    +        checkAnswer(
    +          sql("select * from orc_source"),
    +          sql("select * from src where key < 10"))
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to local dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val path = dir.toURI.getPath
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        checkAnswer(
    +          spark.read.orc(dir.getCanonicalPath),
    +          sql("select * from test_insert_table"))
    +      }
    +    }
    +  }
    +
    +  test("insert overwrite to dir from temp table") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val pathUri = dir.toURI
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        sql(
    +          s"""
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |STORED AS orc
    +             |SELECT * FROM test_insert_table
    +           """.stripMargin)
    +
    +        // use orc data source to check the data of path is right.
    +        checkAnswer(
    +          spark.read.orc(dir.getCanonicalPath),
    +          sql("select * from test_insert_table"))
    +      }
    +    }
    +  }
    +
    +  test("multi insert overwrite to dir") {
    +    withTempView("test_insert_table") {
    +      spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table")
    +
    +      withTempDir { dir =>
    +        val pathUri = dir.toURI
    +
    +        sql(
    +          s"""
    +             |FROM test_insert_table
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    +             |STORED AS orc
    +             |SELECT id
    +             |INSERT OVERWRITE DIRECTORY '${pathUri}'
    --- End diff --
    
    To test multi-insert, we need to use different paths and then verify both are successful or not. 


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137452938
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala ---
    @@ -140,6 +143,14 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast
             parts, query, overwrite, false) if parts.isEmpty =>
           InsertIntoDataSourceCommand(l, query, overwrite)
     
    +    case InsertIntoDir(isLocal, storage, provider, query, overwrite)
    --- End diff --
    
    Nit: `isLocal ` -> `_`


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136436585
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
    @@ -534,4 +534,83 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
           }
         }
       }
    +
    +  test("insert overwrite to dir from hive metastore table") {
    +    withTempDir { dir =>
    +      val path = dir.toURI.getPath
    +
    +      checkAnswer(
    +        sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10"),
    +        Seq.empty[Row])
    --- End diff --
    
    ok. 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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134842420
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala ---
    @@ -359,6 +359,18 @@ case class InsertIntoTable(
       override lazy val resolved: Boolean = false
     }
     
    +case class InsertIntoDir(
    --- End diff --
    
    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 pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r137919207
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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.execution
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.hive.client.HiveClientImpl
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whether overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile with HiveTmpPath {
    +
    +  override def children: Seq[LogicalPlan] = query :: Nil
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +
    +    val hiveTable = HiveClientImpl.toHiveTable(CatalogTable(
    +      identifier = TableIdentifier(storage.locationUri.get.toString, Some("default")),
    +      tableType = org.apache.spark.sql.catalyst.catalog.CatalogTableType.VIEW,
    +      storage = storage,
    +      schema = query.schema
    +    ))
    +    hiveTable.getMetadata.put(serdeConstants.SERIALIZATION_LIB,
    +      storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
    +
    +    val tableDesc = new TableDesc(
    +      hiveTable.getInputFormatClass,
    +      hiveTable.getOutputFormatClass,
    +      hiveTable.getMetadata
    +    )
    +
    +    val hadoopConf = sparkSession.sessionState.newHadoopConf()
    +    val jobConf = new JobConf(hadoopConf)
    +
    +    val targetPath = new Path(storage.locationUri.get)
    +    val writeToPath =
    +      if (isLocal) {
    +        val localFileSystem = FileSystem.getLocal(jobConf)
    +        localFileSystem.makeQualified(targetPath)
    +      } else {
    +        val qualifiedPath = FileUtils.makeQualified(targetPath, hadoopConf)
    +        val dfs = qualifiedPath.getFileSystem(jobConf)
    +        if (!dfs.exists(qualifiedPath)) {
    +          dfs.mkdirs(qualifiedPath.getParent)
    +        }
    +        qualifiedPath
    +      }
    +
    +    val tmpPath = getExternalTmpPath(sparkSession, hadoopConf, writeToPath)
    +    val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc(
    +      tmpPath.toString, tableDesc, false)
    +
    +    try {
    +      saveAsHiveFile(
    +        sparkSession = sparkSession,
    +        plan = children.head,
    +        hadoopConf = hadoopConf,
    +        fileSinkConf = fileSinkConf,
    +        outputLocation = tmpPath.toString)
    +
    +      val fs = writeToPath.getFileSystem(hadoopConf)
    +      if (overwrite && fs.exists(writeToPath)) {
    +        fs.listStatus(writeToPath).foreach { existFile =>
    +          if (Option(existFile.getPath) != createdTempDir) fs.delete(existFile.getPath, true)
    +        }
    +      }
    +
    +      fs.listStatus(tmpPath).foreach {
    +        tmpFile => fs.rename(tmpFile.getPath, writeToPath)
    +      }
    +
    +      deleteExternalTmpPath(hadoopConf)
    --- End diff --
    
    We should also try to remove the external tmp path when an exception happens.


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81346 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81346/testReport)** for PR 18975 at commit [`52350e8`](https://github.com/apache/spark/commit/52350e8adb7b07540f635c103f36e065c151f8cd).
     * 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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134845628
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -1509,4 +1509,84 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
           query: LogicalPlan): LogicalPlan = {
         RepartitionByExpression(expressions, query, conf.numShufflePartitions)
       }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE DIRECTORY
    +   *   [path]
    +   *   [OPTIONS table_property_list]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteDir(
    +      ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
    +    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
    +    var storage = DataSource.buildStorageFormatFromOptions(options)
    +
    +    val path = Option(ctx.path) match {
    +      case Some(s) => string(s)
    +      case None => ""
    +    }
    +
    +    if (!path.isEmpty && storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " +
    +          "you can only specify one of them.", ctx)
    +    }
    +    if (path.isEmpty && !storage.locationUri.isDefined) {
    +      throw new ParseException(
    +        "You need to specify directory path or 'path' in OPTIONS, but not both", ctx)
    +    }
    +
    +    if (!path.isEmpty) {
    +      val customLocation = Some(CatalogUtils.stringToURI(path))
    +      storage = storage.copy(locationUri = customLocation)
    +    }
    +
    +    val provider = ctx.tableProvider.qualifiedName.getText
    +
    +    (false, storage, Some(provider))
    +  }
    +
    +  /**
    +   * Return the parameters for [[InsertIntoDir]] logical plan.
    +   *
    +   * Expected format:
    +   * {{{
    +   *   INSERT OVERWRITE [LOCAL] DIRECTORY
    +   *   path
    +   *   [ROW FORMAT row_format]
    +   *   [STORED AS file_format]
    +   *   select_statement;
    +   * }}}
    +   */
    +  override def visitInsertOverwriteHiveDir(
    +      ctx: InsertOverwriteHiveDirContext): InsertDirParams = withOrigin(ctx) {
    +    validateRowFormatFileFormat(ctx.rowFormat, ctx.createFileFormat, ctx)
    +    val rowStorage = Option(ctx.rowFormat).map(visitRowFormat)
    +      .getOrElse(CatalogStorageFormat.empty)
    +    val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat)
    +      .getOrElse(CatalogStorageFormat.empty)
    +
    +    val path = string(ctx.path)
    +    // The path field is required
    +    if (path.isEmpty) {
    +      operationNotAllowed("INSERT OVERWRITE DIRECTORY must be accompanied by path", ctx)
    +    }
    +
    +    val defaultStorage = HiveSerDe.getDefaultStorage(conf)
    +
    +    val storage = CatalogStorageFormat(
    +      locationUri = Some(CatalogUtils.stringToURI(path)),
    +      inputFormat = fileStorage.inputFormat.orElse(defaultStorage.inputFormat),
    +      outputFormat = fileStorage.outputFormat.orElse(defaultStorage.outputFormat),
    +      serde = rowStorage.serde.orElse(fileStorage.serde).orElse(defaultStorage.serde),
    +      compressed = false,
    +      properties = rowStorage.properties ++ fileStorage.properties)
    +
    +    (ctx.LOCAL != null, storage, None)
    --- End diff --
    
    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 issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #81366 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81366/testReport)** for PR 18975 at commit [`2ec9947`](https://github.com/apache/spark/commit/2ec9947af571691966e979e17aec12d3d683decf).
     * 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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136443263
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: Option[String],
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def innerChildren: Seq[LogicalPlan] = Seq(query)
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    assert(innerChildren.length == 1)
    +    assert(storage.locationUri.nonEmpty, "Directory path is required")
    +    assert(provider.isDefined, "Data source is required")
    +
    +    // Create the relation based on the input logical plan: `data`.
    +    val pathOption = storage.locationUri.map("path" -> CatalogUtils.URIToString(_))
    +    val dataSource = DataSource(
    --- End diff --
    
    @gatorsmile I am not familiar with data source. Is it possible that you can give me some hints how to limit this to only "FileFormat"?


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137460756
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,81 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + *
    + * @param storage storage format used to describe how the query result is stored.
    + * @param provider the data source type to be used
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: String,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def innerChildren: Seq[LogicalPlan] = Seq(query)
    --- End diff --
    
    `innerChildren ` -> `children`


---

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


[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #80959 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80959/testReport)** for PR 18975 at commit [`0882dd1`](https://github.com/apache/spark/commit/0882dd1f3c300f832d731b69a0d57ef461e55038).
     * This patch **fails due to an unknown error code, -9**.
     * 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 #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r134348994
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -740,6 +750,7 @@ nonReserved
         | AND | CASE | CAST | DISTINCT | DIV | ELSE | END | FUNCTION | INTERVAL | MACRO | OR | STRATIFY | THEN
         | UNBOUNDED | WHEN
         | DATABASE | SELECT | FROM | WHERE | HAVING | TO | TABLE | WITH | NOT | CURRENT_DATE | CURRENT_TIMESTAMP
    +    | DIRECTORY
    --- End diff --
    
    Could you add it to `TableIdentifierParserSuite`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


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

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

    https://github.com/apache/spark/pull/18975#discussion_r137929108
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,82 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + *
    + * @param storage storage format used to describe how the query result is stored.
    + * @param provider the data source type to be used
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: String,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends RunnableCommand {
    +
    +  override def children: Seq[LogicalPlan] = Seq(query)
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty, "Directory path is required")
    +    assert(!provider.isEmpty, "Data source is required")
    --- End diff --
    
    updated.


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136259136
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala ---
    @@ -0,0 +1,65 @@
    +/*
    + * 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.execution.command
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * A command used to write the result of a query to a directory.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE DIRECTORY (path=STRING)?
    + *   USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...])
    + *   SELECT ...
    + * }}}
    + */
    +case class InsertIntoDataSourceDirCommand(
    +    storage: CatalogStorageFormat,
    +    provider: Option[String],
    --- End diff --
    
    Let us change it to `provider: String`


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

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

    https://github.com/apache/spark/pull/18975#discussion_r134350387
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala ---
    @@ -359,6 +359,17 @@ case class InsertIntoTable(
       override lazy val resolved: Boolean = false
     }
     
    +case class InsertIntoDir(
    --- End diff --
    
    This should be `InsertOverwriteDir`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

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


---

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


[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

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

    https://github.com/apache/spark/pull/18975#discussion_r136681131
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala ---
    @@ -0,0 +1,134 @@
    +/*
    + * 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.execution
    +
    +import java.util.Properties
    +
    +import scala.language.existentials
    +
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.hive.common.FileUtils
    +import org.apache.hadoop.hive.ql.plan.TableDesc
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +import org.apache.hadoop.mapred._
    +
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkPlan
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Command for writing the results of `query` to file system.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   INSERT OVERWRITE [LOCAL] DIRECTORY
    + *   path
    + *   [ROW FORMAT row_format]
    + *   [STORED AS file_format]
    + *   SELECT ...
    + * }}}
    + *
    + * @param isLocal whether the path specified in `storage` is a local directory
    + * @param storage storage format used to describe how the query result is stored.
    + * @param query the logical plan representing data to write to
    + * @param overwrite whthere overwrites existing directory
    + */
    +case class InsertIntoHiveDirCommand(
    +    isLocal: Boolean,
    +    storage: CatalogStorageFormat,
    +    query: LogicalPlan,
    +    overwrite: Boolean) extends SaveAsHiveFile {
    +
    +  override def children: Seq[LogicalPlan] = query :: Nil
    +
    +  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = {
    +    assert(children.length == 1)
    +    assert(storage.locationUri.nonEmpty)
    +
    +    val Array(cols, types) = children.head.output.foldLeft(Array("", "")) { case (r, a) =>
    +      r(0) = r(0) + a.name + ","
    +      r(1) = r(1) + a.dataType.catalogString + ":"
    +      r
    +    }
    +
    +    val properties = new Properties()
    +    properties.put("columns", cols.dropRight(1))
    +    properties.put("columns.types", types.dropRight(1))
    +
    +    val sqlContext = sparkSession.sqlContext
    +
    +    properties.put(serdeConstants.SERIALIZATION_LIB,
    +      storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
    +
    +    import scala.collection.JavaConverters._
    +    properties.putAll(storage.properties.asJava)
    +
    +    var tableDesc = new TableDesc(
    --- End diff --
    
    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 issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    **[Test build #80944 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80944/testReport)** for PR 18975 at commit [`8261b39`](https://github.com/apache/spark/commit/8261b39aa4a0ad31a62ebcdb33c718a121f9fbbc).
     * 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 #18975: [SPARK-4131] Support "Writing data into the filesystem f...

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

    https://github.com/apache/spark/pull/18975
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81291/
    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