You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by liancheng <gi...@git.apache.org> on 2016/04/26 18:29:14 UTC

[GitHub] spark pull request: [SPARK-14445][SQL] Supports native execution o...

GitHub user liancheng opened a pull request:

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

    [SPARK-14445][SQL] Supports native execution of SHOW COLUMNS and SHOW PARTITIONS

    (This PR is mostly a rebased version of PR #12222 by @dilipbiswal with all pending PR review comments addressed. Committer who merges this PR please attribute it to @dilipbiswal.)
    
    ## What changes were proposed in this pull request?
    
    This PR adds Native execution of `SHOW COLUMNS` and `SHOW PARTITION` commands.
    
    Command Syntax:
    
    ```sql
    SHOW COLUMNS (FROM | IN) table_identifier [(FROM | IN) database]
    
    SHOW PARTITIONS [db_name.]table_name [PARTITION(partition_spec)]
    ```
    
    ## How was this patch tested?
    
    Added test cases in HiveCommandSuite to verify execution and DDLCommandSuite to verify plans.

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

    $ git pull https://github.com/liancheng/spark spark-14445-show-columns-partitions

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

    https://github.com/apache/spark/pull/12703.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 #12703
    
----
commit b0edb3dd09bac2d8c44235f91bc360ffa831e074
Author: Dilip Biswal <db...@us.ibm.com>
Date:   2016-04-01T17:25:41Z

    [SPARK-14445] Support native execution of SHOW COLUMNS and SHOW PARTITIONS

commit a13db610805f9ec7817370d393a16260bfabd543
Author: Dilip Biswal <db...@us.ibm.com>
Date:   2016-04-14T22:14:18Z

    Andrew's comments

commit c0985a8a326e6c4af13bad9fb02e7b8f2b2171d8
Author: Dilip Biswal <db...@us.ibm.com>
Date:   2016-04-15T06:19:20Z

    test fix

commit 6cf6d954a799b9690a7f4a1b4a2a706207ea03e4
Author: Cheng Lian <li...@databricks.com>
Date:   2016-04-26T15:26:08Z

    Fixes issues introduced while rebasing

----


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

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


[GitHub] spark pull request: [SPARK-14445][SQL] Supports native execution o...

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

    https://github.com/apache/spark/pull/12703#discussion_r61180561
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -112,3 +116,107 @@ case class ExplainCommand(
         ("Error occurred during query planning: \n" + cause.getMessage).split("\n").map(Row(_))
       }
     }
    +
    +/**
    + * A command that lists column names of a given table.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   SHOW COLUMNS (FROM | IN) table_identifier [(FROM | IN) database];
    + * }}}
    + */
    +case class ShowColumnsCommand(table: TableIdentifier) extends RunnableCommand {
    +  // The result of SHOW COLUMNS has one column called 'result'
    +  override val output: Seq[Attribute] = {
    +    AttributeReference("result", StringType, nullable = false)() :: Nil
    +  }
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    sparkSession.sessionState.catalog.getTableMetadata(table).schema.map(c => Row(c.name))
    +  }
    +}
    +
    +/**
    + * A command that lists the partition names of a given table. If the partition spec is specified,
    + * partitions that match the spec are returned. [[AnalysisException]] is thrown under the following
    + * conditions:
    + *
    + *  1. If the command is called for a non partitioned table.
    + *  2. If the partition spec refers to the columns that are not defined as partitioning columns.
    + *
    + * This function creates a [[ShowPartitionsCommand]] logical plan
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   SHOW PARTITIONS [db_name.]table_name [PARTITION(partition_spec)]
    + * }}}
    + */
    +case class ShowPartitionsCommand(
    +    table: TableIdentifier,
    +    spec: Option[TablePartitionSpec]) extends RunnableCommand {
    +  // The result of SHOW PARTITIONS has one column called 'result'
    +  override val output: Seq[Attribute] = {
    +    AttributeReference("result", StringType, nullable = false)() :: Nil
    +  }
    +
    +  private def getPartName(spec: TablePartitionSpec, partitionColumns: Seq[String]): String = {
    +    partitionColumns.map { col =>
    +      PartitioningUtils.escapePathName(col) + "=" + PartitioningUtils.escapePathName(spec(col))
    +    }.mkString("/")
    --- End diff --
    
    @yhuai I was trying to remember why i changed it as i had it use "/" before :-). I think i looked at the function makePartName in Warehouse.java in hive and had decided to change the code. Just FYI.


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

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


[GitHub] spark pull request: [SPARK-14445][SQL] Supports native execution o...

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

    https://github.com/apache/spark/pull/12703#discussion_r61179510
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -112,3 +116,107 @@ case class ExplainCommand(
         ("Error occurred during query planning: \n" + cause.getMessage).split("\n").map(Row(_))
       }
     }
    +
    +/**
    + * A command that lists column names of a given table.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   SHOW COLUMNS (FROM | IN) table_identifier [(FROM | IN) database];
    + * }}}
    + */
    +case class ShowColumnsCommand(table: TableIdentifier) extends RunnableCommand {
    +  // The result of SHOW COLUMNS has one column called 'result'
    +  override val output: Seq[Attribute] = {
    +    AttributeReference("result", StringType, nullable = false)() :: Nil
    +  }
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    sparkSession.sessionState.catalog.getTableMetadata(table).schema.map(c => Row(c.name))
    +  }
    +}
    +
    +/**
    + * A command that lists the partition names of a given table. If the partition spec is specified,
    + * partitions that match the spec are returned. [[AnalysisException]] is thrown under the following
    + * conditions:
    + *
    + *  1. If the command is called for a non partitioned table.
    + *  2. If the partition spec refers to the columns that are not defined as partitioning columns.
    + *
    + * This function creates a [[ShowPartitionsCommand]] logical plan
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   SHOW PARTITIONS [db_name.]table_name [PARTITION(partition_spec)]
    + * }}}
    + */
    +case class ShowPartitionsCommand(
    +    table: TableIdentifier,
    +    spec: Option[TablePartitionSpec]) extends RunnableCommand {
    +  // The result of SHOW PARTITIONS has one column called 'result'
    +  override val output: Seq[Attribute] = {
    +    AttributeReference("result", StringType, nullable = false)() :: Nil
    +  }
    +
    +  private def getPartName(spec: TablePartitionSpec, partitionColumns: Seq[String]): String = {
    +    partitionColumns.map { col =>
    +      PartitioningUtils.escapePathName(col) + "=" + PartitioningUtils.escapePathName(spec(col))
    +    }.mkString("/")
    --- End diff --
    
    @yhuai Got it. 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 pull request: [SPARK-14445][SQL] Supports native execution o...

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

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


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

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


[GitHub] spark pull request: [SPARK-14445][SQL] Supports native execution o...

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

    https://github.com/apache/spark/pull/12703#issuecomment-214822872
  
    **[Test build #57005 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57005/consoleFull)** for PR 12703 at commit [`6cf6d95`](https://github.com/apache/spark/commit/6cf6d954a799b9690a7f4a1b4a2a706207ea03e4).
     * 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: [SPARK-14445][SQL] Supports native execution o...

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

    https://github.com/apache/spark/pull/12703#issuecomment-214938531
  
    @dilipbiswal also updated #12222 and passed Jenkins. So I merged it. Closing this one.


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

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


[GitHub] spark pull request: [SPARK-14445][SQL] Supports native execution o...

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

    https://github.com/apache/spark/pull/12703#discussion_r61176555
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -112,3 +116,107 @@ case class ExplainCommand(
         ("Error occurred during query planning: \n" + cause.getMessage).split("\n").map(Row(_))
       }
     }
    +
    +/**
    + * A command that lists column names of a given table.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   SHOW COLUMNS (FROM | IN) table_identifier [(FROM | IN) database];
    + * }}}
    + */
    +case class ShowColumnsCommand(table: TableIdentifier) extends RunnableCommand {
    +  // The result of SHOW COLUMNS has one column called 'result'
    +  override val output: Seq[Attribute] = {
    +    AttributeReference("result", StringType, nullable = false)() :: Nil
    +  }
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    sparkSession.sessionState.catalog.getTableMetadata(table).schema.map(c => Row(c.name))
    +  }
    +}
    +
    +/**
    + * A command that lists the partition names of a given table. If the partition spec is specified,
    + * partitions that match the spec are returned. [[AnalysisException]] is thrown under the following
    + * conditions:
    + *
    + *  1. If the command is called for a non partitioned table.
    + *  2. If the partition spec refers to the columns that are not defined as partitioning columns.
    + *
    + * This function creates a [[ShowPartitionsCommand]] logical plan
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   SHOW PARTITIONS [db_name.]table_name [PARTITION(partition_spec)]
    + * }}}
    + */
    +case class ShowPartitionsCommand(
    +    table: TableIdentifier,
    +    spec: Option[TablePartitionSpec]) extends RunnableCommand {
    +  // The result of SHOW PARTITIONS has one column called 'result'
    +  override val output: Seq[Attribute] = {
    +    AttributeReference("result", StringType, nullable = false)() :: Nil
    +  }
    +
    +  private def getPartName(spec: TablePartitionSpec, partitionColumns: Seq[String]): String = {
    +    partitionColumns.map { col =>
    +      PartitioningUtils.escapePathName(col) + "=" + PartitioningUtils.escapePathName(spec(col))
    +    }.mkString("/")
    --- End diff --
    
    I think this should be always `/`. The separator used by the hadoop fs may be different from `File.separator`. Also, this is returning the string representation for the display purpose. Even if hadoop fs is using a different separator, I am not sure we have to match that.


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

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


[GitHub] spark pull request: [SPARK-14445][SQL] Supports native execution o...

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

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


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

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


[GitHub] spark pull request: [SPARK-14445][SQL] Supports native execution o...

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

    https://github.com/apache/spark/pull/12703#issuecomment-214803508
  
    **[Test build #57005 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57005/consoleFull)** for PR 12703 at commit [`6cf6d95`](https://github.com/apache/spark/commit/6cf6d954a799b9690a7f4a1b4a2a706207ea03e4).


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

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


[GitHub] spark pull request: [SPARK-14445][SQL] Supports native execution o...

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

    https://github.com/apache/spark/pull/12703#issuecomment-214823091
  
    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: [SPARK-14445][SQL] Supports native execution o...

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

    https://github.com/apache/spark/pull/12703#discussion_r61126850
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala ---
    @@ -247,4 +259,105 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto
           hiveContext.sessionState.hadoopConf.set("fs.default.name", originalFsName)
         }
       }
    +
    +  test("show columns") {
    +    checkAnswer(
    +      sql("SHOW COLUMNS IN parquet_tab3"),
    +      Row("col1") :: Row("col 2") :: Nil)
    +
    +    checkAnswer(
    +      sql("SHOW COLUMNS IN default.parquet_tab3"),
    +      Row("col1") :: Row("col 2") :: Nil)
    +
    +    checkAnswer(
    +      sql("SHOW COLUMNS IN parquet_tab3 FROM default"),
    +      Row("col1") :: Row("col 2") :: Nil)
    +
    +    checkAnswer(
    +      sql("SHOW COLUMNS IN parquet_tab4 IN default"),
    +      Row("price") :: Row("qty") :: Row("year") :: Row("month") :: Nil)
    +
    +    val message = intercept[NoSuchTableException] {
    +      sql("SHOW COLUMNS IN badtable FROM default")
    +    }.getMessage
    +    assert(message.contains("badtable not found in database"))
    +  }
    +
    +  test("show partitions - show everything") {
    +    checkAnswer(
    +      sql("show partitions parquet_tab4"),
    +      Row("year=2015/month=1") ::
    +        Row("year=2015/month=2") ::
    +        Row("year=2016/month=2") ::
    +        Row("year=2016/month=3") :: Nil)
    +
    +    checkAnswer(
    +      sql("show partitions default.parquet_tab4"),
    +      Row("year=2015/month=1") ::
    +        Row("year=2015/month=2") ::
    +        Row("year=2016/month=2") ::
    +        Row("year=2016/month=3") :: Nil)
    +  }
    +
    --- End diff --
    
    @liancheng , i had added an extra test to test > 5 partition columns like following:
     test("show partitions - show everything more than 5 part keys") {
        checkAnswer(
          sql("show partitions parquet_tab5"),
          Row("year=2016/month=3/hour=10/minute=10/sec=10/extra=1") ::
            Row("year=2016/month=4/hour=10/minute=10/sec=10/extra=1") :: Nil)
      }
    
    with a table definition in beforeAll()
    sql(
          """
            |CREATE TABLE parquet_tab5 (price int, qty int)
            |PARTITIONED BY (year int, month int, hour int, minute int, sec int, extra int)
          """.stripMargin)
        sql(
          """
            |INSERT INTO parquet_tab5
            |PARTITION(year = 2016, month = 3, hour = 10, minute = 10, sec = 10, extra = 1) SELECT 3, 3
          """.stripMargin)
        sql(
          """
            |INSERT INTO parquet_tab5
            |PARTITION(year = 2016, month = 4, hour = 10, minute = 10, sec = 10, extra = 1) SELECT 3, 3
          """.stripMargin)
    
    Should we add it here ?


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

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


[GitHub] spark pull request: [SPARK-14445][SQL] Supports native execution o...

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

    https://github.com/apache/spark/pull/12703#discussion_r61185701
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala ---
    @@ -247,4 +259,105 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto
           hiveContext.sessionState.hadoopConf.set("fs.default.name", originalFsName)
         }
       }
    +
    +  test("show columns") {
    +    checkAnswer(
    +      sql("SHOW COLUMNS IN parquet_tab3"),
    +      Row("col1") :: Row("col 2") :: Nil)
    +
    +    checkAnswer(
    +      sql("SHOW COLUMNS IN default.parquet_tab3"),
    +      Row("col1") :: Row("col 2") :: Nil)
    +
    +    checkAnswer(
    +      sql("SHOW COLUMNS IN parquet_tab3 FROM default"),
    +      Row("col1") :: Row("col 2") :: Nil)
    +
    +    checkAnswer(
    +      sql("SHOW COLUMNS IN parquet_tab4 IN default"),
    +      Row("price") :: Row("qty") :: Row("year") :: Row("month") :: Nil)
    +
    +    val message = intercept[NoSuchTableException] {
    +      sql("SHOW COLUMNS IN badtable FROM default")
    +    }.getMessage
    +    assert(message.contains("badtable not found in database"))
    +  }
    +
    +  test("show partitions - show everything") {
    +    checkAnswer(
    +      sql("show partitions parquet_tab4"),
    +      Row("year=2015/month=1") ::
    +        Row("year=2015/month=2") ::
    +        Row("year=2016/month=2") ::
    +        Row("year=2016/month=3") :: Nil)
    +
    +    checkAnswer(
    +      sql("show partitions default.parquet_tab4"),
    +      Row("year=2015/month=1") ::
    +        Row("year=2015/month=2") ::
    +        Row("year=2016/month=2") ::
    +        Row("year=2016/month=3") :: Nil)
    +  }
    +
    --- End diff --
    
    Oh yea, I should add that. 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 pull request: [SPARK-14445][SQL] Supports native execution o...

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

    https://github.com/apache/spark/pull/12703#discussion_r61129992
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -112,3 +116,107 @@ case class ExplainCommand(
         ("Error occurred during query planning: \n" + cause.getMessage).split("\n").map(Row(_))
       }
     }
    +
    +/**
    + * A command that lists column names of a given table.
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   SHOW COLUMNS (FROM | IN) table_identifier [(FROM | IN) database];
    + * }}}
    + */
    +case class ShowColumnsCommand(table: TableIdentifier) extends RunnableCommand {
    +  // The result of SHOW COLUMNS has one column called 'result'
    +  override val output: Seq[Attribute] = {
    +    AttributeReference("result", StringType, nullable = false)() :: Nil
    +  }
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    sparkSession.sessionState.catalog.getTableMetadata(table).schema.map(c => Row(c.name))
    +  }
    +}
    +
    +/**
    + * A command that lists the partition names of a given table. If the partition spec is specified,
    + * partitions that match the spec are returned. [[AnalysisException]] is thrown under the following
    + * conditions:
    + *
    + *  1. If the command is called for a non partitioned table.
    + *  2. If the partition spec refers to the columns that are not defined as partitioning columns.
    + *
    + * This function creates a [[ShowPartitionsCommand]] logical plan
    + *
    + * The syntax of using this command in SQL is:
    + * {{{
    + *   SHOW PARTITIONS [db_name.]table_name [PARTITION(partition_spec)]
    + * }}}
    + */
    +case class ShowPartitionsCommand(
    +    table: TableIdentifier,
    +    spec: Option[TablePartitionSpec]) extends RunnableCommand {
    +  // The result of SHOW PARTITIONS has one column called 'result'
    +  override val output: Seq[Attribute] = {
    +    AttributeReference("result", StringType, nullable = false)() :: Nil
    +  }
    +
    +  private def getPartName(spec: TablePartitionSpec, partitionColumns: Seq[String]): String = {
    +    partitionColumns.map { col =>
    +      PartitioningUtils.escapePathName(col) + "=" + PartitioningUtils.escapePathName(spec(col))
    +    }.mkString("/")
    --- End diff --
    
    I was using File.separator here. Isn't that more portable ? Or its always guaranteed to be a back slash ?


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

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