You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by viirya <gi...@git.apache.org> on 2016/02/03 11:09:47 UTC

[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...

GitHub user viirya opened a pull request:

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

    [SPARK-13139][SQL][WIP] Create native DDL commands

    JIRA: https://issues.apache.org/jira/browse/SPARK-13139
    
    From JIRA: We currently delegate most DDLs directly to Hive, through NativePlaceholder in HiveQl.scala. In Spark 2.0, we want to provide native implementations for DDLs for both SQLContext and HiveContext.
    
    This PR will do the first step to parse DDL commands and create logical commands that encapsulate them. Actual implementations still delegate to HiveNativeCommand now.


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

    $ git pull https://github.com/viirya/spark-1 native-ddl

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

    https://github.com/apache/spark/pull/11048.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 #11048
    
----
commit 77252af5b83d026030c5dd1270ecd4d0788af779
Author: Liang-Chi Hsieh <vi...@gmail.com>
Date:   2016-02-03T10:00:02Z

    init import: create database and create 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: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-189118075
  
    **[Test build #52016 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52016/consoleFull)** for PR 11048 at commit [`53d02f0`](https://github.com/apache/spark/commit/53d02f06d4b6c019eb03873f1ea1697a78b414dd).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53571306
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala ---
    @@ -0,0 +1,652 @@
    +/*
    + * 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
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.plans.PlanTest
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources.BucketSpec
    +import org.apache.spark.sql.types._
    +
    +class SparkQlSuite extends PlanTest {
    --- End diff --
    
    why is this named SparkQlSuite?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187624865
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51757/
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54797924
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/DDLCommandSuite.scala ---
    @@ -0,0 +1,652 @@
    +/*
    + * 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
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.plans.PlanTest
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources.BucketSpec
    +import org.apache.spark.sql.types._
    +
    +class DDLCommandSuite extends PlanTest {
    --- End diff --
    
    this should probably be in the `command` package 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 pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-181365000
  
    @rxin I've added alter table command support. As this command and corresponding change is big, I think we should let this PR only cover these three commands and do other commands in other PRs. How do you think?


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187626323
  
    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: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53593980
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    --- End diff --
    
    Yes. But it is optional.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187548746
  
    **[Test build #51725 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51725/consoleFull)** for PR 11048 at commit [`54c0578`](https://github.com/apache/spark/commit/54c0578db8bb19439723b392d91b4cd5c20da962).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53568504
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    --- End diff --
    
    Can there be more than one location? You could do a direct pattern match, i.e:
    
        val Seq(Token("TOK_SKEWED_LOCATIONS", Token("TOK_SKEWED_LOCATION_LIST", locationMaps) :: Nil)) = args


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-181437574
  
    **[Test build #50922 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50922/consoleFull)** for PR 11048 at commit [`eb1fab7`](https://github.com/apache/spark/commit/eb1fab78319432184e3eed34721adae6da230188).
     * 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569257
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    --- End diff --
    
    we know that we are getting two parameters here: bucketCols & sortCols.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187671556
  
    @hvanhovell I've addressed your comments. Please review this if it is appropriate now. Let me know if you have other comments. 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54817960
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala ---
    @@ -16,18 +16,38 @@
      */
     package org.apache.spark.sql.execution
     
    +import scala.collection.mutable.ArrayBuffer
    +
     import org.apache.spark.sql.{AnalysisException, SaveMode}
    -import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier}
    +import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    -import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, CatalystQl, ParserConf, ParserSupport, SimpleParserConf}
     import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
     import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
     import org.apache.spark.sql.execution.datasources._
     import org.apache.spark.sql.types.StructType
     
     private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends CatalystQl(conf) {
    +  import ParserSupport._
    +
       /** Check if a command should not be explained. */
    -  protected def isNoExplainCommand(command: String): Boolean = "TOK_DESCTABLE" == command
    +  protected def isNoExplainCommand(command: String): Boolean =
    +    "TOK_DESCTABLE" == command || "TOK_ALTERTABLE" == command
    +
    +  protected def extractProps(
    +      node: ASTNode,
    +      firstLevelNodeStr: String,
    +      secondLevelNodeStr: String): Seq[(String, String)] = node match {
    +    case Token(firstLevelNodeStr, options) =>
    --- End diff --
    
    I'm not sure if this works. Because `firstLevelNodeStr` is a variable, not a literal, you might have to do this:
    ```
    case Token(x, options) if x == firstLevelNodeStr =>
    ```
    same in L45.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-186370350
  
    I'll take another round this weekend.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569202
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    --- End diff --
    
    We can split out the serdeClassName already.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54821276
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/ddl.scala ---
    @@ -0,0 +1,189 @@
    +/*
    + * 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.commands
    +
    +import java.util.NoSuchElementException
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.errors.TreeNodeException
    +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution._
    +import org.apache.spark.sql.execution.datasources.BucketSpec
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +
    +abstract class NativeDDLCommands(val sql: String) extends RunnableCommand {
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    sqlContext.catalog.runNativeCommand(sql)
    +  }
    +
    +  override val output: Seq[Attribute] =
    +    Seq(AttributeReference("result", StringType, nullable = false)())
    +}
    +
    +case class CreateDataBase(
    +    databaseName: String,
    +    allowExisting: Boolean,
    +    path: Option[String],
    +    comment: Option[String],
    +    props: Map[String, String])(sql: String) extends NativeDDLCommands(sql) with Logging
    +
    +case class CreateFunction(
    +    functionName: String,
    +    asName: String,
    --- End diff --
    
    iiuc, you need two things. one is the fully qualified name for source code of the function, and the other is the name of the function.
    
    the current naming is confusing.



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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54797155
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/ddl.scala ---
    @@ -0,0 +1,189 @@
    +/*
    --- End diff --
    
    and then maybe for other things like `ExplainCommand` or `ClearCacheCommand` I would just leave them in `commands.scala`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54822707
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala ---
    @@ -73,349 +73,3 @@ private[sql] case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan
     
    --- End diff --
    
    Now that you have the `command` package, would it make sense to move `RunnableCommand` there? Right now it's confusing how there's a `o.a.s.sql.execution.commands.scala` and a `o.a.s.sql.execution.command.commands.scala`. We should only have the latter and move everything here there.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54802393
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/CatalystQl.scala ---
    @@ -90,92 +85,13 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) extends
         }
       }
     
    -  protected def getClauses(
    -      clauseNames: Seq[String],
    -      nodeList: Seq[ASTNode]): Seq[Option[ASTNode]] = {
    -    var remainingNodes = nodeList
    -    val clauses = clauseNames.map { clauseName =>
    -      val (matches, nonMatches) = remainingNodes.partition(_.text.toUpperCase == clauseName)
    -      remainingNodes = nonMatches ++ (if (matches.nonEmpty) matches.tail else Nil)
    -      matches.headOption
    -    }
    -
    -    if (remainingNodes.nonEmpty) {
    -      sys.error(
    -        s"""Unhandled clauses: ${remainingNodes.map(_.treeString).mkString("\n")}.
    -            |You are likely trying to use an unsupported Hive feature."""".stripMargin)
    -    }
    -    clauses
    -  }
    -
    -  protected def getClause(clauseName: String, nodeList: Seq[ASTNode]): ASTNode =
    -    getClauseOption(clauseName, nodeList).getOrElse(sys.error(
    -      s"Expected clause $clauseName missing from ${nodeList.map(_.treeString).mkString("\n")}"))
    -
    -  protected def getClauseOption(clauseName: String, nodeList: Seq[ASTNode]): Option[ASTNode] = {
    -    nodeList.filter { case ast: ASTNode => ast.text == clauseName } match {
    -      case Seq(oneMatch) => Some(oneMatch)
    -      case Seq() => None
    -      case _ => sys.error(s"Found multiple instances of clause $clauseName")
    -    }
    -  }
    -
    -  protected def nodeToAttribute(node: ASTNode): Attribute = node match {
    +  def nodeToAttribute(node: ASTNode): Attribute = node match {
    --- End diff --
    
    actually, the only usage is `parseDdl`, which is not used anywhere. Would you mind deleting this method and `parseDdl` to simplify the code?


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

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


[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-179150071
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/50658/
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569610
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    --- End diff --
    
    Is TOK_IFNOTEXISTS the first of the children when it is defined? Is so we could just pattern match this.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-184747842
  
    **[Test build #51365 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51365/consoleFull)** for PR 11048 at commit [`3c2fc25`](https://github.com/apache/spark/commit/3c2fc25ecb1fdfa77b15db5d163c6fab46c9c8e8).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-184686197
  
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53567674
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala ---
    @@ -30,17 +30,20 @@ import org.apache.spark.sql.types._
     import org.apache.spark.unsafe.types.CalendarInterval
     import org.apache.spark.util.random.RandomSampler
     
    +abstract class BaseParser(val conf: ParserConf) extends ParserInterface with ParserBase {
    --- End diff --
    
    Let's move CatalystQl into the parser package.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569704
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    +        case t @ Token("TOK_PARTSPEC", partArgs) =>
    +          if (currentPart != null) {
    +            partitions += ((currentPart, None))
    +          }
    +          currentPart = parsePartitionSpec(t).get
    +        case Token("TOK_PARTITIONLOCATION", loc :: Nil) =>
    +          val location = unquoteString(loc.text)
    +          if (currentPart != null) {
    +            partitions += ((currentPart, Some(location)))
    +            currentPart = null
    +          } else {
    +            // We should not reach here
    +            throw new AnalysisException("Partition location must follow a partition spec.")
    +          }
    +      }
    +
    +      if (currentPart != null) {
    +        partitions += ((currentPart, None))
    +      }
    +      AlterTableAddPartition(tableIdent, partitions, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest =>
    --- End diff --
    
    Make the match more concise. You are expecting only 1 argument.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53568236
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala ---
    @@ -0,0 +1,533 @@
    +/*
    + * 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.commands
    +
    +import java.util.NoSuchElementException
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row, SQLConf, SQLContext}
    +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.errors.TreeNodeException
    +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution._
    +import org.apache.spark.sql.execution.datasources.BucketSpec
    +import org.apache.spark.sql.types._
    +
    +abstract class NativeDDLCommands(val sql: String) extends RunnableCommand {
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    sqlContext.catalog.runNativeCommand(sql)
    +  }
    +
    +  override val output: Seq[Attribute] =
    +    Seq(AttributeReference("result", StringType, nullable = false)())
    +}
    +
    +case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableCommand with Logging {
    +
    +  private def keyValueOutput: Seq[Attribute] = {
    +    val schema = StructType(
    +      StructField("key", StringType, false) ::
    +        StructField("value", StringType, false) :: Nil)
    +    schema.toAttributes
    +  }
    +
    +  private val (_output, runFunc): (Seq[Attribute], SQLContext => Seq[Row]) = kv match {
    +    // Configures the deprecated "mapred.reduce.tasks" property.
    +    case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, Some(value))) =>
    +      val runFunc = (sqlContext: SQLContext) => {
    +        logWarning(
    +          s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " +
    +            s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS.key} instead.")
    +        if (value.toInt < 1) {
    +          val msg =
    +            s"Setting negative ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} for automatically " +
    +              "determining the number of reducers is not supported."
    +          throw new IllegalArgumentException(msg)
    +        } else {
    +          sqlContext.setConf(SQLConf.SHUFFLE_PARTITIONS.key, value)
    +          Seq(Row(SQLConf.SHUFFLE_PARTITIONS.key, value))
    +        }
    +      }
    +      (keyValueOutput, runFunc)
    +
    +    case Some((SQLConf.Deprecated.EXTERNAL_SORT, Some(value))) =>
    --- End diff --
    
    A lot of duplicate code here. Why not put it in a collection and do a lookup?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-181370933
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/50919/
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54795827
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/ddl.scala ---
    @@ -0,0 +1,189 @@
    +/*
    + * 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.commands
    +
    +import java.util.NoSuchElementException
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.errors.TreeNodeException
    +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution._
    +import org.apache.spark.sql.execution.datasources.BucketSpec
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +
    +abstract class NativeDDLCommands(val sql: String) extends RunnableCommand {
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    sqlContext.catalog.runNativeCommand(sql)
    --- End diff --
    
    I see, because we want the command to be in the sql package it shouldn't know anything about Hive. In the future these commands will no longer be passed to Hive directly as a string so they shouldn't be "native" anymore. For now, I would create a temporary method in `SQLContext`:
    ```
    // TODO: remove this once we call specific operations in the catalog instead
    protected[sql] def runDDLCommand(text: String): Seq[Row] = {
      throw new UnsupportedOperationException
    }
    
    // In HiveContext.scala
    protected[sql] override def runDDLCommand(text: String): Seq[Row] = {
      runHiveSql(text).map(Row(_))
    }
    ```
    even though it's temporary I still think it's cleaner than doing it in the catalog.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-179249273
  
    **[Test build #50660 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50660/consoleFull)** for PR 11048 at commit [`2b38d11`](https://github.com/apache/spark/commit/2b38d1197bfaeef2f56691543ece01051e9cbb3e).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54821023
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/ddl.scala ---
    @@ -0,0 +1,189 @@
    +/*
    + * 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.commands
    +
    +import java.util.NoSuchElementException
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.errors.TreeNodeException
    +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution._
    +import org.apache.spark.sql.execution.datasources.BucketSpec
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +
    +abstract class NativeDDLCommands(val sql: String) extends RunnableCommand {
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    sqlContext.catalog.runNativeCommand(sql)
    +  }
    +
    +  override val output: Seq[Attribute] =
    +    Seq(AttributeReference("result", StringType, nullable = false)())
    +}
    +
    +case class CreateDataBase(
    +    databaseName: String,
    +    allowExisting: Boolean,
    +    path: Option[String],
    +    comment: Option[String],
    +    props: Map[String, String])(sql: String) extends NativeDDLCommands(sql) with Logging
    +
    +case class CreateFunction(
    +    functionName: String,
    +    asName: String,
    --- End diff --
    
    What do you mean by `asName`? Maybe this should be called `alias`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54804281
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserSupport.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.catalyst.parser
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.trees.CurrentOrigin
    +import org.apache.spark.sql.types._
    +
    +object ParserSupport {
    +  protected val escapedIdentifier = "`(.+)`".r
    +  protected val doubleQuotedString = "\"([^\"]+)\"".r
    +  protected val singleQuotedString = "'([^']+)'".r
    +
    +  private[sql] def unquoteString(str: String): String = str match {
    +    case singleQuotedString(s) => s
    +    case doubleQuotedString(s) => s
    +    case other => other
    +  }
    +
    +  /** Strips backticks from ident if present */
    +  private[sql] def cleanIdentifier(ident: String): String = ident match {
    +    case escapedIdentifier(i) => i
    +    case plainIdent => plainIdent
    +  }
    +
    +  private[sql] def cleanAndUnquoteString(str: String): String = {
    +    cleanIdentifier(unquoteString(str))
    +  }
    +
    +  object Token {
    --- End diff --
    
    can you move this to the top so it's not hidden among other methods?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187670296
  
    **[Test build #51759 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51759/consoleFull)** for PR 11048 at commit [`0eb1687`](https://github.com/apache/spark/commit/0eb16871d95c5214ed1923a809d9dcae8618674d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-184686201
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51361/
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187514903
  
    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: [SPARK-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-179149432
  
    **[Test build #50658 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50658/consoleFull)** for PR 11048 at commit [`77252af`](https://github.com/apache/spark/commit/77252af5b83d026030c5dd1270ecd4d0788af779).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569457
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    --- End diff --
    
    Doesn't the existence of skewedArgs imply the value of notStoredAsDirs?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54793856
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/ddl.scala ---
    @@ -0,0 +1,189 @@
    +/*
    + * 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.commands
    +
    +import java.util.NoSuchElementException
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.errors.TreeNodeException
    +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution._
    +import org.apache.spark.sql.execution.datasources.BucketSpec
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +
    +abstract class NativeDDLCommands(val sql: String) extends RunnableCommand {
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    sqlContext.catalog.runNativeCommand(sql)
    +  }
    +
    +  override val output: Seq[Attribute] =
    +    Seq(AttributeReference("result", StringType, nullable = false)())
    +}
    +
    +case class CreateDataBase(
    --- End diff --
    
    `CreateDatabase`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53593184
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    --- End diff --
    
    Some alter table commands have partition spec defined. We should parse it at one place instead of doing this parsing at many places.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#discussion_r52612493
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala ---
    @@ -52,7 +56,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly
               getClauses(Seq("TOK_CREATETABLE", "FORMATTED", "EXTENDED"), explainArgs)
             ExplainCommand(nodeToPlan(crtTbl), extended = extended.isDefined)
     
    -      case Token("TOK_EXPLAIN", explainArgs) =>
    +      case Token("TOK_EXPLAIN", explainArgs) if "TOK_QUERY" == explainArgs.head.text =>
    --- End diff --
    
    Why not `Token("TOK_EXPLAIN", Token("TOK_QUERY", query) :: explainArgs) =>` ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53593420
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    --- End diff --
    
    Yeah, as above. I will update this.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53593393
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    --- End diff --
    
    `rest` here is not alway Nil, it is possibly partition spec for alter table commands. This optional partition spec is after alter table command. Actually as we parse it before here, we don't need to use `rest` to catch it and can ignore it. I will update this.
    
    For example,
    
        TOK_ALTERTABLE 2, 1, 22, 12 
        :- TOK_TABNAME 2, 5, 5, 12 
        :  +- table_name 2, 5, 5, 12 
        :- TOK_ALTERTABLE_COMPACT 3, 20, 22, 8 
        :  +- 'MAJOR' 3, 22, 22, 8 
        +- TOK_PARTSPEC 2, 7, 18, 34 
           :- TOK_PARTVAL 2, 10, 12, 34 
           :  :- dt 2, 10, 10, 34 
           :  +- '2008-08-08' 2, 12, 12, 37 
           +- TOK_PARTVAL 2, 15, 17, 51 
              :- country 2, 15, 15, 51 
              +- 'us' 2, 17, 17, 59 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-182784626
  
    btw @viirya can we create a execution.commands package for this?



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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53570023
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    +        case t @ Token("TOK_PARTSPEC", partArgs) =>
    +          if (currentPart != null) {
    +            partitions += ((currentPart, None))
    +          }
    +          currentPart = parsePartitionSpec(t).get
    +        case Token("TOK_PARTITIONLOCATION", loc :: Nil) =>
    +          val location = unquoteString(loc.text)
    +          if (currentPart != null) {
    +            partitions += ((currentPart, Some(location)))
    +            currentPart = null
    +          } else {
    +            // We should not reach here
    +            throw new AnalysisException("Partition location must follow a partition spec.")
    +          }
    +      }
    +
    +      if (currentPart != null) {
    +        partitions += ((currentPart, None))
    +      }
    +      AlterTableAddPartition(tableIdent, partitions, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest =>
    +      val newPartition = parsePartitionSpec(args(0))
    +      AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", args) :: rest =>
    +      val Seq(Some(partSpec), Some(fromTable)) =
    +        base.getClauses(Seq("TOK_PARTSPEC", "TOK_TABNAME"), args)
    +      val partition = parsePartitionSpec(partSpec).get
    +      val fromTableIdent = base.extractTableIdent(fromTable)
    +      AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPARTS", args) :: rest =>
    +      val parts = args.collect {
    +        case Token("TOK_PARTSPEC", partitions) =>
    +          partitions.map {
    +            case Token("TOK_PARTVAL", ident :: op :: constant :: Nil) =>
    +              (unquoteString(cleanIdentifier(ident.text)),
    +                op.text, unquoteString(cleanIdentifier(constant.text)))
    +          }
    +      }
    +
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      val purge = base.getClauseOption("PURGE", args)
    +
    +      val replication = base.getClauseOption("TOK_REPLICATION", args).map {
    +        case Token("TOK_REPLICATION", replId :: metadata :: Nil) =>
    +          (unquoteString(cleanIdentifier(replId.text)), true)
    +        case Token("TOK_REPLICATION", replId :: Nil) =>
    +          (unquoteString(cleanIdentifier(replId.text)), false)
    +      }
    +
    +      AlterTableDropPartition(
    +        tableIdent,
    +        parts,
    +        allowExisting.isDefined,
    +        purge.isDefined,
    +        replication)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ARCHIVE", args) :: rest =>
    +      val partition = parsePartitionSpec(args(0)).get
    --- End diff --
    
    Make the match more concise. You are expecting only 1 argument.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569784
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    +        case t @ Token("TOK_PARTSPEC", partArgs) =>
    +          if (currentPart != null) {
    +            partitions += ((currentPart, None))
    +          }
    +          currentPart = parsePartitionSpec(t).get
    +        case Token("TOK_PARTITIONLOCATION", loc :: Nil) =>
    +          val location = unquoteString(loc.text)
    +          if (currentPart != null) {
    +            partitions += ((currentPart, Some(location)))
    +            currentPart = null
    +          } else {
    +            // We should not reach here
    +            throw new AnalysisException("Partition location must follow a partition spec.")
    +          }
    +      }
    +
    +      if (currentPart != null) {
    +        partitions += ((currentPart, None))
    +      }
    +      AlterTableAddPartition(tableIdent, partitions, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest =>
    +      val newPartition = parsePartitionSpec(args(0))
    +      AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", args) :: rest =>
    +      val Seq(Some(partSpec), Some(fromTable)) =
    +        base.getClauses(Seq("TOK_PARTSPEC", "TOK_TABNAME"), args)
    +      val partition = parsePartitionSpec(partSpec).get
    +      val fromTableIdent = base.extractTableIdent(fromTable)
    +      AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPARTS", args) :: rest =>
    +      val parts = args.collect {
    --- End diff --
    
    Why collect? What else can we expect in DROPPARTS?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53594561
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    --- End diff --
    
    Yes. It looks like `Token("TOK_PARTSPEC", _) :: Token("TOK_PARTITIONLOCATION", _) :: Token("TOK_PARTSPEC", _) :: Token("TOK_PARTITIONLOCATION", _) :: ... `.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-191533739
  
    @viirya Thanks for working on this. The overall approach is very reasonable but a few things can be improved:
    - many files need to be moved to the new packages
    - many unused imports in all the files
    - many fields and methods can be marked as private, but are not
    - the splitting of commands into 2 files is kind of arbitrary. IIUC all native commands we currently pass to Hive directly today are grouped under `ddl.scala`, and the rest are grouped under `commands.scala`. In the future, however, we would like to handle all the DDLs ourselves, so this division won't make sense anymore.
    - general lack of comments in this area of the code makes things difficult to follow (not your fault)
    
    Additionally I think the reason why this patch is so big is because we moved a lot of files. The moving itself can be done in a separate PR, which would reduce the diff significantly and make the patch easier to review.
    
    Addressing all the outstanding comments will likely take a long time. Would you mind that I take this over? I'll be sure to include you as the original author in the final commit.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

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


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

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


[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-181370236
  
    **[Test build #50919 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50919/consoleFull)** for PR 11048 at commit [`f9c1397`](https://github.com/apache/spark/commit/f9c13976132febea2a6d8443aa5d144a2a107648).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53770770
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala ---
    @@ -0,0 +1,533 @@
    +/*
    --- End diff --
    
    ok. I've split it to two files.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-184609994
  
    **[Test build #51356 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51356/consoleFull)** for PR 11048 at commit [`170bd77`](https://github.com/apache/spark/commit/170bd771655f09337c078f48ace9f18b6d667c9e).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `abstract class BaseParser(val conf: ParserConf) extends ParserInterface with ParserBase `
      * `trait ParserBase `
      * `abstract class NativeDDLCommands(val sql: String) extends RunnableCommand `
      * `case class AlterTableSetProperties(`
      * `case class AlterTableDropProperties(`
      * `case class AlterTableCommandParser(base: CatalystQl) extends PlanParser `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54793483
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala ---
    @@ -23,6 +23,7 @@ import scala.util.parsing.combinator.syntactical.StandardTokenParsers
     import scala.util.parsing.combinator.PackratParsers
     import scala.util.parsing.input.CharArrayReader.EofCh
     
    +import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical._
     
     private[sql] abstract class AbstractSparkSQLParser
    --- End diff --
    
    This class seems to belong to your new parser package. I would move it there instead of changing the import here. Also, `CatalystQlSuite` needs to be moved since `CatalystQl` is now part of the parser package.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-181394362
  
    **[Test build #50922 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50922/consoleFull)** for PR 11048 at commit [`eb1fab7`](https://github.com/apache/spark/commit/eb1fab78319432184e3eed34721adae6da230188).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569083
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    --- End diff --
    
    use `rest`? Is it possible to defer partition parsing until later on?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

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


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

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


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

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


[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-182633680
  
    @viirya  yes we can do this incrementally. Let's just create subtasks under https://issues.apache.org/jira/browse/SPARK-13139


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

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


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53595807
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    +        case t @ Token("TOK_PARTSPEC", partArgs) =>
    +          if (currentPart != null) {
    +            partitions += ((currentPart, None))
    +          }
    +          currentPart = parsePartitionSpec(t).get
    +        case Token("TOK_PARTITIONLOCATION", loc :: Nil) =>
    +          val location = unquoteString(loc.text)
    +          if (currentPart != null) {
    +            partitions += ((currentPart, Some(location)))
    +            currentPart = null
    +          } else {
    +            // We should not reach here
    +            throw new AnalysisException("Partition location must follow a partition spec.")
    +          }
    +      }
    +
    +      if (currentPart != null) {
    +        partitions += ((currentPart, None))
    +      }
    +      AlterTableAddPartition(tableIdent, partitions, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest =>
    +      val newPartition = parsePartitionSpec(args(0))
    +      AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", args) :: rest =>
    +      val Seq(Some(partSpec), Some(fromTable)) =
    +        base.getClauses(Seq("TOK_PARTSPEC", "TOK_TABNAME"), args)
    +      val partition = parsePartitionSpec(partSpec).get
    +      val fromTableIdent = base.extractTableIdent(fromTable)
    +      AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPARTS", args) :: rest =>
    +      val parts = args.collect {
    --- End diff --
    
    Yes. After a sequence of `TOK_PARTSPEC`, there is an optional `TOK_IFEXISTS`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-179150064
  
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569740
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    +        case t @ Token("TOK_PARTSPEC", partArgs) =>
    +          if (currentPart != null) {
    +            partitions += ((currentPart, None))
    +          }
    +          currentPart = parsePartitionSpec(t).get
    +        case Token("TOK_PARTITIONLOCATION", loc :: Nil) =>
    +          val location = unquoteString(loc.text)
    +          if (currentPart != null) {
    +            partitions += ((currentPart, Some(location)))
    +            currentPart = null
    +          } else {
    +            // We should not reach here
    +            throw new AnalysisException("Partition location must follow a partition spec.")
    +          }
    +      }
    +
    +      if (currentPart != null) {
    +        partitions += ((currentPart, None))
    +      }
    +      AlterTableAddPartition(tableIdent, partitions, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest =>
    +      val newPartition = parsePartitionSpec(args(0))
    +      AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", args) :: rest =>
    +      val Seq(Some(partSpec), Some(fromTable)) =
    +        base.getClauses(Seq("TOK_PARTSPEC", "TOK_TABNAME"), args)
    +      val partition = parsePartitionSpec(partSpec).get
    --- End diff --
    
    val Some(partition) = parsePartitionSpec(partSpec)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-188669802
  
    **[Test build #51951 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51951/consoleFull)** for PR 11048 at commit [`5f8e70d`](https://github.com/apache/spark/commit/5f8e70d7dfd7c7c3efb640210e9a0798dffad964).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187396766
  
    **[Test build #51681 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51681/consoleFull)** for PR 11048 at commit [`0c57651`](https://github.com/apache/spark/commit/0c57651f1be2b6a5e04532d49ad504d2b8038ddd).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54801945
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/CatalystQl.scala ---
    @@ -90,92 +85,13 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) extends
         }
       }
     
    -  protected def getClauses(
    -      clauseNames: Seq[String],
    -      nodeList: Seq[ASTNode]): Seq[Option[ASTNode]] = {
    -    var remainingNodes = nodeList
    -    val clauses = clauseNames.map { clauseName =>
    -      val (matches, nonMatches) = remainingNodes.partition(_.text.toUpperCase == clauseName)
    -      remainingNodes = nonMatches ++ (if (matches.nonEmpty) matches.tail else Nil)
    -      matches.headOption
    -    }
    -
    -    if (remainingNodes.nonEmpty) {
    -      sys.error(
    -        s"""Unhandled clauses: ${remainingNodes.map(_.treeString).mkString("\n")}.
    -            |You are likely trying to use an unsupported Hive feature."""".stripMargin)
    -    }
    -    clauses
    -  }
    -
    -  protected def getClause(clauseName: String, nodeList: Seq[ASTNode]): ASTNode =
    -    getClauseOption(clauseName, nodeList).getOrElse(sys.error(
    -      s"Expected clause $clauseName missing from ${nodeList.map(_.treeString).mkString("\n")}"))
    -
    -  protected def getClauseOption(clauseName: String, nodeList: Seq[ASTNode]): Option[ASTNode] = {
    -    nodeList.filter { case ast: ASTNode => ast.text == clauseName } match {
    -      case Seq(oneMatch) => Some(oneMatch)
    -      case Seq() => None
    -      case _ => sys.error(s"Found multiple instances of clause $clauseName")
    -    }
    -  }
    -
    -  protected def nodeToAttribute(node: ASTNode): Attribute = node match {
    +  def nodeToAttribute(node: ASTNode): Attribute = node match {
    --- End diff --
    
    only used in 1 place. Can you make it private?


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

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


[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-181370915
  
    **[Test build #50919 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50919/consoleFull)** for PR 11048 at commit [`f9c1397`](https://github.com/apache/spark/commit/f9c13976132febea2a6d8443aa5d144a2a107648).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54823428
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,391 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, CatalystQl, ParserConf, ParserSupport, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +object AlterTableCommandParser {
    --- End diff --
    
    by the way, almost everything in this class can be private. This will make the code a lot more readable.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-181818773
  
    **[Test build #50965 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50965/consoleFull)** for PR 11048 at commit [`3db2e1d`](https://github.com/apache/spark/commit/3db2e1d867b5750eb157c751a303c542bdad110e).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54820517
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala ---
    @@ -16,18 +16,38 @@
      */
     package org.apache.spark.sql.execution
     
    +import scala.collection.mutable.ArrayBuffer
    +
     import org.apache.spark.sql.{AnalysisException, SaveMode}
    -import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier}
    +import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    -import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, CatalystQl, ParserConf, ParserSupport, SimpleParserConf}
     import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
     import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
     import org.apache.spark.sql.execution.datasources._
     import org.apache.spark.sql.types.StructType
     
     private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends CatalystQl(conf) {
    +  import ParserSupport._
    +
       /** Check if a command should not be explained. */
    -  protected def isNoExplainCommand(command: String): Boolean = "TOK_DESCTABLE" == command
    +  protected def isNoExplainCommand(command: String): Boolean =
    +    "TOK_DESCTABLE" == command || "TOK_ALTERTABLE" == command
    +
    +  protected def extractProps(
    --- End diff --
    
    also it would be good to add a test for it. As I pointed out elsewhere I don't think this is actually working.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187397336
  
    **[Test build #51681 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51681/consoleFull)** for PR 11048 at commit [`0c57651`](https://github.com/apache/spark/commit/0c57651f1be2b6a5e04532d49ad504d2b8038ddd).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#discussion_r52644696
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala ---
    @@ -62,6 +66,458 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly
             val tableIdent = extractTableIdent(nameParts)
             RefreshTable(tableIdent)
     
    +      case Token("TOK_CREATEDATABASE", Token(databaseName, Nil) :: createDatabaseArgs) =>
    --- End diff --
    
    The main problem is LogicalPlan parsing, which we can split up in command/ddl/query parsing. We could use partial functions (or something like that) to implement the different parts of the parsing logic.
    
    For instance:
    
        abstract class BaseParser(val conf: ParserConf) extends ParserInterface {
          val planParsers: Seq[PlanParser]
    
          lazy val planParser = planParsers.reduce(_.orElse(_))
    
          def nodeToPlan(node: ASTNode): LogicalPlan = {
            planParser.applyOrElse(node, throw new NotImplementedError(node.text))
          }
        }
    
        abstract class PlanParser extends PartialFunction[ASTNode, LogicalPlan]
    
        case class ExplainCommandParser(base: BaseParser) extends PlanParser {
          val catalystQl = new CatalystQl
          import catalystQl.Token
          
          override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_EXPLAIN"
    
          override def apply(v1: ASTNode): LogicalPlan = v1.children match {
            case (crtTbl @ Token("TOK_CREATETABLE" | "TOK_QUERY", _)) :: rest =>
              val extended = rest.exists(_.text.toUpperCase == "EXTENDED")
              ExplainCommand(base.nodeToPlan(crtTbl), extended)
          }
        }
    
        class SomeParser(conf: ParserConf) extends BaseParser {
          val planParsers: Seq[PlanParser] = Seq(
            ExplainCommandParser(this))
        }
    
    



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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187397340
  
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-188714991
  
    Merged build finished. Test PASSed.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-184609621
  
    **[Test build #51356 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51356/consoleFull)** for PR 11048 at commit [`170bd77`](https://github.com/apache/spark/commit/170bd771655f09337c078f48ace9f18b6d667c9e).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54820194
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala ---
    @@ -16,18 +16,38 @@
      */
     package org.apache.spark.sql.execution
     
    +import scala.collection.mutable.ArrayBuffer
    +
     import org.apache.spark.sql.{AnalysisException, SaveMode}
    -import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier}
    +import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    -import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, CatalystQl, ParserConf, ParserSupport, SimpleParserConf}
     import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
     import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
     import org.apache.spark.sql.execution.datasources._
     import org.apache.spark.sql.types.StructType
     
     private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends CatalystQl(conf) {
    +  import ParserSupport._
    +
       /** Check if a command should not be explained. */
    -  protected def isNoExplainCommand(command: String): Boolean = "TOK_DESCTABLE" == command
    +  protected def isNoExplainCommand(command: String): Boolean =
    +    "TOK_DESCTABLE" == command || "TOK_ALTERTABLE" == command
    +
    +  protected def extractProps(
    --- End diff --
    
    can you add a comment that shows an example of what a matching AST might look like?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#discussion_r52864016
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala ---
    @@ -35,7 +36,8 @@ import org.apache.spark.sql.types.StructType
     private[sql] case class BucketSpec(
         numBuckets: Int,
         bucketColumnNames: Seq[String],
    -    sortColumnNames: Seq[String])
    +    sortColumnNames: Seq[String],
    --- End diff --
    
    `BucketSpec` is used in other places. Because `sortDirections` is optional, I don't want to change current implementation a lot.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#discussion_r52544458
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala ---
    @@ -62,6 +66,458 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly
             val tableIdent = extractTableIdent(nameParts)
             RefreshTable(tableIdent)
     
    +      case Token("TOK_CREATEDATABASE", Token(databaseName, Nil) :: createDatabaseArgs) =>
    --- End diff --
    
    cc @hvanhovell 
    
    any suggestions on how we can make this file/function more modular? It is getting too long and we are about to add a lot more statements to it.



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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53568183
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala ---
    @@ -62,6 +87,57 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly
             val tableIdent = extractTableIdent(nameParts)
             RefreshTable(tableIdent)
     
    +      case Token("TOK_CREATEDATABASE", Token(databaseName, Nil) :: createDatabaseArgs) =>
    +        val Seq(
    +          allowExisting,
    +          dbLocation,
    +          databaseComment,
    +          dbprops) = getClauses(Seq(
    +          "TOK_IFNOTEXISTS",
    +          "TOK_DATABASELOCATION",
    +          "TOK_DATABASECOMMENT",
    +          "TOK_DATABASEPROPERTIES"), createDatabaseArgs)
    +
    +        val location = dbLocation.map {
    +          case Token("TOK_DATABASELOCATION", Token(loc, Nil) :: Nil) => unquoteString(loc)
    +        }
    +        val comment = databaseComment.map {
    +          case Token("TOK_DATABASECOMMENT", Token(comment, Nil) :: Nil) => unquoteString(comment)
    +        }
    +        val props: Map[String, String] = dbprops.toSeq.flatMap {
    +          case Token("TOK_DATABASEPROPERTIES", propList) =>
    +            propList.flatMap(extractProps)
    +        }.toMap
    +
    +        CreateDataBase(databaseName, allowExisting.isDefined, location, comment, props)(node.source)
    +
    +      case Token("TOK_CREATEFUNCTION", func :: as :: createFuncArgs) =>
    +        val funcName = func.map(x => unquoteString(x.text)).mkString(".")
    +        val asName = unquoteString(as.text)
    +        val Seq(
    +          rList,
    +          temp) = getClauses(Seq(
    +          "TOK_RESOURCE_LIST",
    +          "TOK_TEMPORARY"), createFuncArgs)
    +
    +        val resourcesMap: Map[String, String] = rList.toSeq.flatMap {
    +          case Token("TOK_RESOURCE_LIST", resources) =>
    +            resources.map {
    +              case Token("TOK_RESOURCE_URI", rType :: Token(rPath, Nil) :: Nil) =>
    +                val resourceType = rType match {
    +                  case Token("TOK_JAR", Nil) => "jar"
    +                  case Token("TOK_FILE", Nil) => "file"
    +                  case Token("TOK_ARCHIVE", Nil) => "archive"
    +                }
    +                (resourceType, unquoteString(rPath))
    +            }
    +        }.toMap
    +        CreateFunction(funcName, asName, resourcesMap, temp.isDefined)(node.source)
    +
    +      case Token("TOK_ALTERTABLE", alterTableArgs) =>
    +        AlterTableCommandParser(this).applyOrElse(node,
    --- End diff --
    
    The idea of the `PartialFunction` design was that we factor out all Parsing functionality into a bunch of PartialFunctions each taking case of a specific parsing area. This is nice because this allows us to compose a parser.
    
    The code here just calls into the AlterTableCommandParser and treats it like a regular function. This deviates from the design. There are few options here:
    
    - follow the proposed design;
    - present a composable alternative and implement that;
    - create a function in SparkQl which handles AlterTable commands, drop the created infrastructure, and we will split the parsers in a followup PR.
    
    I currently favor the third option.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-184696062
  
    **[Test build #51365 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51365/consoleFull)** for PR 11048 at commit [`3c2fc25`](https://github.com/apache/spark/commit/3c2fc25ecb1fdfa77b15db5d163c6fab46c9c8e8).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53568197
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala ---
    @@ -0,0 +1,533 @@
    +/*
    --- End diff --
    
    Shouldn't we split this file into a number of smaller units? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569223
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    --- End diff --
    
    This seems redundant. We can do this in the initial match.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569174
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    --- End diff --
    
    Map over serdeArgs.tail.headOption?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-189093128
  
    **[Test build #52016 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52016/consoleFull)** for PR 11048 at commit [`53d02f0`](https://github.com/apache/spark/commit/53d02f06d4b6c019eb03873f1ea1697a78b414dd).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187670589
  
    Merged build finished. Test PASSed.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-179479022
  
    Thanks - this looks pretty good as a start. We will need to add many other ddls, including alter/drop table, etc.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53570001
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    +        case t @ Token("TOK_PARTSPEC", partArgs) =>
    +          if (currentPart != null) {
    +            partitions += ((currentPart, None))
    +          }
    +          currentPart = parsePartitionSpec(t).get
    +        case Token("TOK_PARTITIONLOCATION", loc :: Nil) =>
    +          val location = unquoteString(loc.text)
    +          if (currentPart != null) {
    +            partitions += ((currentPart, Some(location)))
    +            currentPart = null
    +          } else {
    +            // We should not reach here
    +            throw new AnalysisException("Partition location must follow a partition spec.")
    +          }
    +      }
    +
    +      if (currentPart != null) {
    +        partitions += ((currentPart, None))
    +      }
    +      AlterTableAddPartition(tableIdent, partitions, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest =>
    +      val newPartition = parsePartitionSpec(args(0))
    +      AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", args) :: rest =>
    +      val Seq(Some(partSpec), Some(fromTable)) =
    +        base.getClauses(Seq("TOK_PARTSPEC", "TOK_TABNAME"), args)
    +      val partition = parsePartitionSpec(partSpec).get
    +      val fromTableIdent = base.extractTableIdent(fromTable)
    +      AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPARTS", args) :: rest =>
    +      val parts = args.collect {
    +        case Token("TOK_PARTSPEC", partitions) =>
    +          partitions.map {
    +            case Token("TOK_PARTVAL", ident :: op :: constant :: Nil) =>
    +              (unquoteString(cleanIdentifier(ident.text)),
    +                op.text, unquoteString(cleanIdentifier(constant.text)))
    +          }
    +      }
    +
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      val purge = base.getClauseOption("PURGE", args)
    +
    +      val replication = base.getClauseOption("TOK_REPLICATION", args).map {
    --- End diff --
    
    We could simplify the pattern match, i.e.: 
    
        case Token("TOK_REPLICATION", replId :: metadata) =>
          (unquoteString(cleanIdentifier(replId.text)), metadata.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: [SPARK-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#discussion_r52622161
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala ---
    @@ -62,6 +66,458 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly
             val tableIdent = extractTableIdent(nameParts)
             RefreshTable(tableIdent)
     
    +      case Token("TOK_CREATEDATABASE", Token(databaseName, Nil) :: createDatabaseArgs) =>
    +        val Seq(
    +          allowExisting,
    +          dbLocation,
    +          databaseComment,
    +          dbprops) = getClauses(Seq(
    +          "TOK_IFNOTEXISTS",
    +          "TOK_DATABASELOCATION",
    +          "TOK_DATABASECOMMENT",
    +          "TOK_DATABASEPROPERTIES"), createDatabaseArgs)
    +
    +        val location = dbLocation.map {
    +          case Token("TOK_DATABASELOCATION", Token(loc, Nil) :: Nil) => unquoteString(loc)
    +        }
    +        val comment = databaseComment.map {
    +          case Token("TOK_DATABASECOMMENT", Token(comment, Nil) :: Nil) => unquoteString(comment)
    +        }
    +        val props: Map[String, String] = dbprops.toSeq.flatMap {
    +          case Token("TOK_DATABASEPROPERTIES", propList) =>
    +            propList.flatMap {
    +              case Token("TOK_DBPROPLIST", props) =>
    +                props.map {
    +                  case Token("TOK_TABLEPROPERTY", keysAndValue) =>
    +                    val key = keysAndValue.init.map(x => unquoteString(x.text)).mkString(".")
    +                    val value = unquoteString(keysAndValue.last.text)
    +                    (key, value)
    +                }
    +            }
    +        }.toMap
    +
    +        CreateDataBase(databaseName, allowExisting.isDefined, location, comment, props)(node.source)
    +
    +      case Token("TOK_CREATEFUNCTION", func :: as :: createFuncArgs) =>
    +        val funcName = func.map(x => unquoteString(x.text)).mkString(".")
    +        val asName = unquoteString(as.text)
    +        val Seq(
    +          rList,
    +          temp) = getClauses(Seq(
    +          "TOK_RESOURCE_LIST",
    +          "TOK_TEMPORARY"), createFuncArgs)
    +
    +        val resourcesMap: Map[String, String] = rList.toSeq.flatMap {
    +          case Token("TOK_RESOURCE_LIST", resources) =>
    +            resources.map {
    +              case Token("TOK_RESOURCE_URI", rType :: Token(rPath, Nil) :: Nil) =>
    +                val resourceType = rType match {
    +                  case Token("TOK_JAR", Nil) => "jar"
    +                  case Token("TOK_FILE", Nil) => "file"
    +                  case Token("TOK_ARCHIVE", Nil) => "archive"
    +                }
    +                (resourceType, unquoteString(rPath))
    +            }
    +        }.toMap
    +        CreateFunction(funcName, asName, resourcesMap, temp.isDefined)(node.source)
    +
    +      case Token("TOK_ALTERTABLE", alterTableArgs) =>
    --- End diff --
    
    There is not one single `ALTER TABLE` command, there are many (stopped counting at 20). The parser gives us pretty good trees to match on. For instance: `ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 'test')` gives us:
    
        TOK_ALTERTABLE 1, 0, 15, 12
        :- TOK_TABNAME 1, 4, 4, 12
        :  +- table_name 1, 4, 4, 12
        +- TOK_ALTERTABLE_DROPPROPERTIES 1, 6, 15, 44
           +- TOK_TABLEPROPERTIES 1, 10, 15, 44
              +- TOK_TABLEPROPLIST 1, 11, 14, 44
                 :- TOK_TABLEPROPERTY 1, 11, 11, 44
                 :  :- 'comment' 1, 11, 11, 44
                 :  +- TOK_NULL 0, -1, -1, 0
                 +- TOK_TABLEPROPERTY 1, 14, 14, 55
                    :- 'test' 1, 14, 14, 55
                    +- TOK_NULL 0, -1, -1, 0
    
    Lets split this code by matching on the `TOK_ALTERTABLE_*` tokens. The result should be alot easier to understand.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#discussion_r52617996
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala ---
    @@ -62,6 +66,458 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly
             val tableIdent = extractTableIdent(nameParts)
             RefreshTable(tableIdent)
     
    +      case Token("TOK_CREATEDATABASE", Token(databaseName, Nil) :: createDatabaseArgs) =>
    +        val Seq(
    +          allowExisting,
    +          dbLocation,
    +          databaseComment,
    +          dbprops) = getClauses(Seq(
    +          "TOK_IFNOTEXISTS",
    +          "TOK_DATABASELOCATION",
    +          "TOK_DATABASECOMMENT",
    +          "TOK_DATABASEPROPERTIES"), createDatabaseArgs)
    +
    +        val location = dbLocation.map {
    +          case Token("TOK_DATABASELOCATION", Token(loc, Nil) :: Nil) => unquoteString(loc)
    +        }
    +        val comment = databaseComment.map {
    +          case Token("TOK_DATABASECOMMENT", Token(comment, Nil) :: Nil) => unquoteString(comment)
    +        }
    +        val props: Map[String, String] = dbprops.toSeq.flatMap {
    +          case Token("TOK_DATABASEPROPERTIES", propList) =>
    --- End diff --
    
    This looks very similar to code in SparkQl https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala#L91-L99.
    
    We could also try to integrate the concepts of TOK_DBPROPLIST/TOK_TABLEOPTIONS and TOK_TABLEPROPERTY/TOK_TABLEOPTION.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53567861
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/PlanParser.scala ---
    @@ -0,0 +1,53 @@
    +/*
    + * 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.catalyst
    +
    +import org.apache.spark.sql.catalyst.parser.ASTNode
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.trees.CurrentOrigin
    +
    +trait ParserBase {
    --- End diff --
    
    `ParserBase` and `BaseParser` are a bit confusing. Maybe call this `ParserSupport`?
    
    Most of the functions in here do not rely on any form of state. We could also make this an object and import that into the parsers.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54795996
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/ddl.scala ---
    @@ -0,0 +1,189 @@
    +/*
    + * 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.commands
    +
    +import java.util.NoSuchElementException
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.errors.TreeNodeException
    --- End diff --
    
    by the way there are a lot of unused imports in this file and other files. Please take the time to remove the ones that are not used.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569392
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    --- End diff --
    
    Maybe it is easier to separate the skewed/not-skewed cases here... Why not extract the children, and do a match on those?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-190584474
  
    **[Test build #52217 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52217/consoleFull)** for PR 11048 at commit [`6032268`](https://github.com/apache/spark/commit/603226830dc8aee52ca957c60f15cb164f10fb90).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-190548517
  
    **[Test build #52217 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52217/consoleFull)** for PR 11048 at commit [`6032268`](https://github.com/apache/spark/commit/603226830dc8aee52ca957c60f15cb164f10fb90).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-179298832
  
    Merged build finished. Test PASSed.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

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


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-179150053
  
    **[Test build #50658 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50658/consoleFull)** for PR 11048 at commit [`77252af`](https://github.com/apache/spark/commit/77252af5b83d026030c5dd1270ecd4d0788af779).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class CreateDataBase(`
      * `case class CreateFunction(`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-181774944
  
    **[Test build #50965 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50965/consoleFull)** for PR 11048 at commit [`3db2e1d`](https://github.com/apache/spark/commit/3db2e1d867b5750eb157c751a303c542bdad110e).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53567755
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala ---
    @@ -90,7 +93,7 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) extends
         }
       }
     
    -  protected def getClauses(
    +  def getClauses(
    --- End diff --
    
    Shouldn't we move all `getClauseX` methods into the `ParserBase`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187550242
  
    Merged build finished. Test PASSed.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569109
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    --- End diff --
    
    Rest is Nil? Same goes for every other rest instance....


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#discussion_r52616927
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala ---
    @@ -418,3 +419,312 @@ case class SetDatabaseCommand(databaseName: String) extends RunnableCommand {
     
       override val output: Seq[Attribute] = Seq.empty
     }
    +
    +case class CreateDataBase(
    +    databaseName: String,
    +    allowExisting: Boolean,
    +    path: Option[String],
    +    comment: Option[String],
    +    props: Map[String, String])(sql: String) extends RunnableCommand with Logging {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    --- End diff --
    
    `override def run(sqlContext: SQLContext)` and `override val output: Seq[Attribute]` are the same is every command. Lets introduce a class which implements this for us.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187468647
  
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187624863
  
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53568012
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala ---
    @@ -16,18 +16,43 @@
      */
     package org.apache.spark.sql.execution
     
    +import scala.collection.mutable.ArrayBuffer
    +
     import org.apache.spark.sql.{AnalysisException, SaveMode}
     import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier}
     import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
     import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
     import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
     import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
     import org.apache.spark.sql.execution.datasources._
     import org.apache.spark.sql.types.StructType
     
     private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends CatalystQl(conf) {
       /** Check if a command should not be explained. */
    -  protected def isNoExplainCommand(command: String): Boolean = "TOK_DESCTABLE" == command
    +  protected def isNoExplainCommand(command: String): Boolean =
    +    "TOK_DESCTABLE" == command || "TOK_ALTERTABLE" == command
    +
    +  protected def extractProps(node: ASTNode): Seq[(String, String)] = node match {
    --- End diff --
    
    We could make this more DRYly by passing the Token strings and processing function (or just unquote the TOK_TABLEOPTION as well).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53571196
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    +        case t @ Token("TOK_PARTSPEC", partArgs) =>
    +          if (currentPart != null) {
    +            partitions += ((currentPart, None))
    +          }
    +          currentPart = parsePartitionSpec(t).get
    +        case Token("TOK_PARTITIONLOCATION", loc :: Nil) =>
    +          val location = unquoteString(loc.text)
    +          if (currentPart != null) {
    +            partitions += ((currentPart, Some(location)))
    +            currentPart = null
    +          } else {
    +            // We should not reach here
    +            throw new AnalysisException("Partition location must follow a partition spec.")
    +          }
    +      }
    +
    +      if (currentPart != null) {
    +        partitions += ((currentPart, None))
    +      }
    +      AlterTableAddPartition(tableIdent, partitions, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest =>
    +      val newPartition = parsePartitionSpec(args(0))
    +      AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", args) :: rest =>
    +      val Seq(Some(partSpec), Some(fromTable)) =
    +        base.getClauses(Seq("TOK_PARTSPEC", "TOK_TABNAME"), args)
    +      val partition = parsePartitionSpec(partSpec).get
    +      val fromTableIdent = base.extractTableIdent(fromTable)
    +      AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPARTS", args) :: rest =>
    +      val parts = args.collect {
    +        case Token("TOK_PARTSPEC", partitions) =>
    +          partitions.map {
    +            case Token("TOK_PARTVAL", ident :: op :: constant :: Nil) =>
    +              (unquoteString(cleanIdentifier(ident.text)),
    +                op.text, unquoteString(cleanIdentifier(constant.text)))
    +          }
    +      }
    +
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      val purge = base.getClauseOption("PURGE", args)
    +
    +      val replication = base.getClauseOption("TOK_REPLICATION", args).map {
    +        case Token("TOK_REPLICATION", replId :: metadata :: Nil) =>
    +          (unquoteString(cleanIdentifier(replId.text)), true)
    +        case Token("TOK_REPLICATION", replId :: Nil) =>
    +          (unquoteString(cleanIdentifier(replId.text)), false)
    +      }
    +
    +      AlterTableDropPartition(
    +        tableIdent,
    +        parts,
    +        allowExisting.isDefined,
    +        purge.isDefined,
    +        replication)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ARCHIVE", args) :: rest =>
    +      val partition = parsePartitionSpec(args(0)).get
    +      AlterTableArchivePartition(tableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_UNARCHIVE", args) :: rest =>
    +      val partition = parsePartitionSpec(args(0)).get
    +      AlterTableUnarchivePartition(tableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_FILEFORMAT", args) :: rest =>
    +      val Seq(fileFormat, genericFormat) =
    +        base.getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"),
    +          args)
    +      val fFormat = fileFormat.map(_.children.map(n => unquoteString(cleanIdentifier(n.text))))
    +      val gFormat = genericFormat.map(f => unquoteString(cleanIdentifier(f.children(0).text)))
    +      AlterTableSetFileFormat(tableIdent, partition, fFormat, gFormat)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_LOCATION", Token(loc, Nil) :: Nil) :: rest =>
    +      AlterTableSetLocation(tableIdent, partition, unquoteString(cleanIdentifier(loc)))(node.source)
    +
    +    case Token("TOK_ALTERTABLE_TOUCH", args) :: rest =>
    +      val part = base.getClauseOption("TOK_PARTSPEC", args).flatMap(parsePartitionSpec)
    +      AlterTableTouch(tableIdent, part)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_COMPACT", Token(compactType, Nil) :: Nil) :: rest =>
    +      AlterTableCompact(tableIdent, partition,
    +        unquoteString(cleanIdentifier(compactType)))(node.source)
    +
    +    case Token("TOK_ALTERTABLE_MERGEFILES", _) :: rest =>
    +      AlterTableMerge(tableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMECOL", args) :: rest =>
    +      val oldName = args(0).text
    +      val newName = args(1).text
    +      val dataType = base.nodeToDataType(args(2))
    +      val afterPos =
    +        base.getClauseOption("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", args)
    +      val afterPosCol = afterPos.map { ap =>
    +        ap.children match {
    +          case Token(col, Nil) :: Nil => col
    +          case _ => null
    +        }
    +      }
    +
    +      val restrict = base.getClauseOption("TOK_RESTRICT", args)
    --- End diff --
    
    This code is repeated a few times. We could move this into a separate method.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187164566
  
    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: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569098
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    --- End diff --
    
    Don't need rename variable. Rest is Nil?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-181819018
  
    Merged build finished. Test PASSed.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53568615
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    --- End diff --
    
    The pattern `unquoteString(cleanIdentifier(...))` is used often. Create a method?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-191571708
  
    @andrewor14 Thanks for reviewing. I don't mind if you want to take this over. Thanks for the credit!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-184684879
  
    **[Test build #51361 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51361/consoleFull)** for PR 11048 at commit [`4a96331`](https://github.com/apache/spark/commit/4a963317165877036469abe9bc45eebfc725edf1).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54803696
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserSupport.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.catalyst.parser
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.trees.CurrentOrigin
    +import org.apache.spark.sql.types._
    +
    +object ParserSupport {
    --- End diff --
    
    this is more like `ParserUtils`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187632553
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51761/
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187632547
  
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-185536867
  
    @hvanhovell Thanks for commenting this. I've addressed them. Please take a look if the updates are appropriate for you.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53567662
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/PlanParser.scala ---
    @@ -0,0 +1,53 @@
    +/*
    + * 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.catalyst
    +
    +import org.apache.spark.sql.catalyst.parser.ASTNode
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.trees.CurrentOrigin
    +
    +trait ParserBase {
    +  object Token {
    +    def unapply(node: ASTNode): Some[(String, List[ASTNode])] = {
    +      CurrentOrigin.setPosition(node.line, node.positionInLine)
    +      node.pattern
    +    }
    +  }
    +
    +  protected val escapedIdentifier = "`(.+)`".r
    --- End diff --
    
    We could put these in a separate object. No need to create then every time we create a parser.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-184610004
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51356/
    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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-179297684
  
    **[Test build #50660 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50660/consoleFull)** for PR 11048 at commit [`2b38d11`](https://github.com/apache/spark/commit/2b38d1197bfaeef2f56691543ece01051e9cbb3e).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

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


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

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


[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-181437761
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/50922/
    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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#discussion_r52620118
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala ---
    @@ -0,0 +1,149 @@
    +/*
    + * 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
    +
    +import org.apache.spark.sql.catalyst.plans.PlanTest
    +
    +class SparkQlSuite extends PlanTest {
    +  val parser = new SparkQl()
    +
    +  test("create database") {
    +    parser.parsePlan("CREATE DATABASE IF NOT EXISTS database_name " +
    --- End diff --
    
    Why don't we use raw strings? This seems/looks painfull.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54803632
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserSupport.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.catalyst.parser
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.trees.CurrentOrigin
    +import org.apache.spark.sql.types._
    +
    +object ParserSupport {
    +  protected val escapedIdentifier = "`(.+)`".r
    +  protected val doubleQuotedString = "\"([^\"]+)\"".r
    +  protected val singleQuotedString = "'([^']+)'".r
    --- End diff --
    
    these can all be private


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-184748514
  
    Merged build finished. Test PASSed.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-184610002
  
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187627230
  
    **[Test build #51759 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51759/consoleFull)** for PR 11048 at commit [`0eb1687`](https://github.com/apache/spark/commit/0eb16871d95c5214ed1923a809d9dcae8618674d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-181437759
  
    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-13139][SQL] Create native DDL commands

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

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


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

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


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-189118219
  
    Merged build finished. Test PASSed.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54794921
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/ddl.scala ---
    @@ -0,0 +1,189 @@
    +/*
    + * 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.commands
    +
    +import java.util.NoSuchElementException
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.errors.TreeNodeException
    +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution._
    +import org.apache.spark.sql.execution.datasources.BucketSpec
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types._
    +
    +abstract class NativeDDLCommands(val sql: String) extends RunnableCommand {
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    sqlContext.catalog.runNativeCommand(sql)
    +  }
    +
    +  override val output: Seq[Attribute] =
    +    Seq(AttributeReference("result", StringType, nullable = false)())
    +}
    +
    +case class CreateDataBase(
    +    databaseName: String,
    +    allowExisting: Boolean,
    +    path: Option[String],
    +    comment: Option[String],
    +    props: Map[String, String])(sql: String) extends NativeDDLCommands(sql) with Logging
    +
    +case class CreateFunction(
    +    functionName: String,
    +    asName: String,
    --- End diff --
    
    what do you mean `asName`? Just say `functionName` or `name`


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569677
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    --- End diff --
    
    You are using side effects, foreach is preferred. Is the problem you are trying to solve that partition locations are not nested?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569754
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    +        case t @ Token("TOK_PARTSPEC", partArgs) =>
    +          if (currentPart != null) {
    +            partitions += ((currentPart, None))
    +          }
    +          currentPart = parsePartitionSpec(t).get
    +        case Token("TOK_PARTITIONLOCATION", loc :: Nil) =>
    +          val location = unquoteString(loc.text)
    +          if (currentPart != null) {
    +            partitions += ((currentPart, Some(location)))
    +            currentPart = null
    +          } else {
    +            // We should not reach here
    +            throw new AnalysisException("Partition location must follow a partition spec.")
    +          }
    +      }
    +
    +      if (currentPart != null) {
    +        partitions += ((currentPart, None))
    +      }
    +      AlterTableAddPartition(tableIdent, partitions, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest =>
    +      val newPartition = parsePartitionSpec(args(0))
    +      AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", args) :: rest =>
    +      val Seq(Some(partSpec), Some(fromTable)) =
    --- End diff --
    
    Do we know the sequence 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-181370928
  
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53570110
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    +        case t @ Token("TOK_PARTSPEC", partArgs) =>
    +          if (currentPart != null) {
    +            partitions += ((currentPart, None))
    +          }
    +          currentPart = parsePartitionSpec(t).get
    +        case Token("TOK_PARTITIONLOCATION", loc :: Nil) =>
    +          val location = unquoteString(loc.text)
    +          if (currentPart != null) {
    +            partitions += ((currentPart, Some(location)))
    +            currentPart = null
    +          } else {
    +            // We should not reach here
    +            throw new AnalysisException("Partition location must follow a partition spec.")
    +          }
    +      }
    +
    +      if (currentPart != null) {
    +        partitions += ((currentPart, None))
    +      }
    +      AlterTableAddPartition(tableIdent, partitions, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest =>
    +      val newPartition = parsePartitionSpec(args(0))
    +      AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", args) :: rest =>
    +      val Seq(Some(partSpec), Some(fromTable)) =
    +        base.getClauses(Seq("TOK_PARTSPEC", "TOK_TABNAME"), args)
    +      val partition = parsePartitionSpec(partSpec).get
    +      val fromTableIdent = base.extractTableIdent(fromTable)
    +      AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPARTS", args) :: rest =>
    +      val parts = args.collect {
    +        case Token("TOK_PARTSPEC", partitions) =>
    +          partitions.map {
    +            case Token("TOK_PARTVAL", ident :: op :: constant :: Nil) =>
    +              (unquoteString(cleanIdentifier(ident.text)),
    +                op.text, unquoteString(cleanIdentifier(constant.text)))
    +          }
    +      }
    +
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      val purge = base.getClauseOption("PURGE", args)
    +
    +      val replication = base.getClauseOption("TOK_REPLICATION", args).map {
    +        case Token("TOK_REPLICATION", replId :: metadata :: Nil) =>
    +          (unquoteString(cleanIdentifier(replId.text)), true)
    +        case Token("TOK_REPLICATION", replId :: Nil) =>
    +          (unquoteString(cleanIdentifier(replId.text)), false)
    +      }
    +
    +      AlterTableDropPartition(
    +        tableIdent,
    +        parts,
    +        allowExisting.isDefined,
    +        purge.isDefined,
    +        replication)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ARCHIVE", args) :: rest =>
    +      val partition = parsePartitionSpec(args(0)).get
    +      AlterTableArchivePartition(tableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_UNARCHIVE", args) :: rest =>
    +      val partition = parsePartitionSpec(args(0)).get
    +      AlterTableUnarchivePartition(tableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_FILEFORMAT", args) :: rest =>
    +      val Seq(fileFormat, genericFormat) =
    +        base.getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"),
    +          args)
    +      val fFormat = fileFormat.map(_.children.map(n => unquoteString(cleanIdentifier(n.text))))
    +      val gFormat = genericFormat.map(f => unquoteString(cleanIdentifier(f.children(0).text)))
    +      AlterTableSetFileFormat(tableIdent, partition, fFormat, gFormat)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_LOCATION", Token(loc, Nil) :: Nil) :: rest =>
    +      AlterTableSetLocation(tableIdent, partition, unquoteString(cleanIdentifier(loc)))(node.source)
    +
    +    case Token("TOK_ALTERTABLE_TOUCH", args) :: rest =>
    +      val part = base.getClauseOption("TOK_PARTSPEC", args).flatMap(parsePartitionSpec)
    --- End diff --
    
    Are we only expecting a single optional PartSpec?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#issuecomment-183012248
  
    @viirya I have made an initial pass. This PR is large enough as it is, lets not more commands to it.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#discussion_r52616686
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala ---
    @@ -35,7 +36,8 @@ import org.apache.spark.sql.types.StructType
     private[sql] case class BucketSpec(
         numBuckets: Int,
         bucketColumnNames: Seq[String],
    -    sortColumnNames: Seq[String])
    +    sortColumnNames: Seq[String],
    --- End diff --
    
    No Tuples? (String, SortOrder)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-190584967
  
    Merged build finished. Test PASSed.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54796821
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/ddl.scala ---
    @@ -0,0 +1,189 @@
    +/*
    --- End diff --
    
    by the way, this division of commands is kind of arbitrary. The things in `commands.scala` are *also* DDLs, so dividing that file into two smaller files this way doesn't really make sense. Instead maybe I would put all the table ones into 1 file (maybe `tableDDL.scala`), all the database ones into another (`databaseDDL.scala`) and all the function ones into yet another (`functionDDL.scala`).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54819242
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala ---
    @@ -16,18 +16,38 @@
      */
     package org.apache.spark.sql.execution
     
    +import scala.collection.mutable.ArrayBuffer
    +
     import org.apache.spark.sql.{AnalysisException, SaveMode}
    -import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier}
    +import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    -import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, CatalystQl, ParserConf, ParserSupport, SimpleParserConf}
     import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
     import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
     import org.apache.spark.sql.execution.datasources._
     import org.apache.spark.sql.types.StructType
     
     private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends CatalystQl(conf) {
    +  import ParserSupport._
    +
       /** Check if a command should not be explained. */
    -  protected def isNoExplainCommand(command: String): Boolean = "TOK_DESCTABLE" == command
    +  protected def isNoExplainCommand(command: String): Boolean =
    +    "TOK_DESCTABLE" == command || "TOK_ALTERTABLE" == command
    +
    +  protected def extractProps(
    +      node: ASTNode,
    +      firstLevelNodeStr: String,
    +      secondLevelNodeStr: String): Seq[(String, String)] = node match {
    +    case Token(firstLevelNodeStr, options) =>
    --- End diff --
    
    e.g.
    ```
    scala> object Token { def unapply(x: String): Some[(String, String)] = Some((x, x)) }
    scala> val matchString = "abc"
    scala> "def" match {
         |   case Token(matchString, _) => true
         |   case _ => false
         | }
    res8: Boolean = true
    ```
    It should have returned false because `def` doesn't match `abc`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54821488
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,391 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, CatalystQl, ParserConf, ParserSupport, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +object AlterTableCommandParser {
    --- End diff --
    
    this should probably be under o.a.s.sql.catalyst.parser`. Also this is a big enough object for the file to be called `AlterTableCommandParser.scala` instead of just `parsers.scala`. You only have 1 parser anyway.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#discussion_r52623977
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala ---
    @@ -52,7 +56,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly
               getClauses(Seq("TOK_CREATETABLE", "FORMATTED", "EXTENDED"), explainArgs)
             ExplainCommand(nodeToPlan(crtTbl), extended = extended.isDefined)
     
    -      case Token("TOK_EXPLAIN", explainArgs) =>
    +      case Token("TOK_EXPLAIN", explainArgs) if "TOK_QUERY" == explainArgs.head.text =>
    --- End diff --
    
    Are there other `EXPLAIN` cases possible?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53569287
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    --- End diff --
    
    cols unzip?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL][WIP] Create native DDL comm...

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

    https://github.com/apache/spark/pull/11048#discussion_r52611305
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala ---
    @@ -0,0 +1,149 @@
    +/*
    + * 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
    +
    +import org.apache.spark.sql.catalyst.plans.PlanTest
    +
    +class SparkQlSuite extends PlanTest {
    --- End diff --
    
    We really should test the resulting plans here, and not wait for an `AnalysisException` to be thrown. I know this is a PITA, but it will save us a lot of headaches in the future.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53570024
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    +        case t @ Token("TOK_PARTSPEC", partArgs) =>
    +          if (currentPart != null) {
    +            partitions += ((currentPart, None))
    +          }
    +          currentPart = parsePartitionSpec(t).get
    +        case Token("TOK_PARTITIONLOCATION", loc :: Nil) =>
    +          val location = unquoteString(loc.text)
    +          if (currentPart != null) {
    +            partitions += ((currentPart, Some(location)))
    +            currentPart = null
    +          } else {
    +            // We should not reach here
    +            throw new AnalysisException("Partition location must follow a partition spec.")
    +          }
    +      }
    +
    +      if (currentPart != null) {
    +        partitions += ((currentPart, None))
    +      }
    +      AlterTableAddPartition(tableIdent, partitions, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest =>
    +      val newPartition = parsePartitionSpec(args(0))
    +      AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", args) :: rest =>
    +      val Seq(Some(partSpec), Some(fromTable)) =
    +        base.getClauses(Seq("TOK_PARTSPEC", "TOK_TABNAME"), args)
    +      val partition = parsePartitionSpec(partSpec).get
    +      val fromTableIdent = base.extractTableIdent(fromTable)
    +      AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPARTS", args) :: rest =>
    +      val parts = args.collect {
    +        case Token("TOK_PARTSPEC", partitions) =>
    +          partitions.map {
    +            case Token("TOK_PARTVAL", ident :: op :: constant :: Nil) =>
    +              (unquoteString(cleanIdentifier(ident.text)),
    +                op.text, unquoteString(cleanIdentifier(constant.text)))
    +          }
    +      }
    +
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      val purge = base.getClauseOption("PURGE", args)
    +
    +      val replication = base.getClauseOption("TOK_REPLICATION", args).map {
    +        case Token("TOK_REPLICATION", replId :: metadata :: Nil) =>
    +          (unquoteString(cleanIdentifier(replId.text)), true)
    +        case Token("TOK_REPLICATION", replId :: Nil) =>
    +          (unquoteString(cleanIdentifier(replId.text)), false)
    +      }
    +
    +      AlterTableDropPartition(
    +        tableIdent,
    +        parts,
    +        allowExisting.isDefined,
    +        purge.isDefined,
    +        replication)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ARCHIVE", args) :: rest =>
    +      val partition = parsePartitionSpec(args(0)).get
    +      AlterTableArchivePartition(tableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_UNARCHIVE", args) :: rest =>
    +      val partition = parsePartitionSpec(args(0)).get
    --- End diff --
    
    Make the match more concise. You are expecting only 1 argument.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187397342
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51681/
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53594996
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    +        case t @ Token("TOK_PARTSPEC", partArgs) =>
    +          if (currentPart != null) {
    +            partitions += ((currentPart, None))
    +          }
    +          currentPart = parsePartitionSpec(t).get
    +        case Token("TOK_PARTITIONLOCATION", loc :: Nil) =>
    +          val location = unquoteString(loc.text)
    +          if (currentPart != null) {
    +            partitions += ((currentPart, Some(location)))
    +            currentPart = null
    +          } else {
    +            // We should not reach here
    +            throw new AnalysisException("Partition location must follow a partition spec.")
    +          }
    +      }
    +
    +      if (currentPart != null) {
    +        partitions += ((currentPart, None))
    +      }
    +      AlterTableAddPartition(tableIdent, partitions, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest =>
    +      val newPartition = parsePartitionSpec(args(0))
    +      AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", args) :: rest =>
    +      val Seq(Some(partSpec), Some(fromTable)) =
    --- End diff --
    
    Yes. I will use pattern match 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53567668
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/PlanParser.scala ---
    @@ -0,0 +1,53 @@
    +/*
    + * 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.catalyst
    --- End diff --
    
    Put this class in the parser subpackage.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53571180
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    +        case t @ Token("TOK_PARTSPEC", partArgs) =>
    +          if (currentPart != null) {
    +            partitions += ((currentPart, None))
    +          }
    +          currentPart = parsePartitionSpec(t).get
    +        case Token("TOK_PARTITIONLOCATION", loc :: Nil) =>
    +          val location = unquoteString(loc.text)
    +          if (currentPart != null) {
    +            partitions += ((currentPart, Some(location)))
    +            currentPart = null
    +          } else {
    +            // We should not reach here
    +            throw new AnalysisException("Partition location must follow a partition spec.")
    +          }
    +      }
    +
    +      if (currentPart != null) {
    +        partitions += ((currentPart, None))
    +      }
    +      AlterTableAddPartition(tableIdent, partitions, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest =>
    +      val newPartition = parsePartitionSpec(args(0))
    +      AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", args) :: rest =>
    +      val Seq(Some(partSpec), Some(fromTable)) =
    +        base.getClauses(Seq("TOK_PARTSPEC", "TOK_TABNAME"), args)
    +      val partition = parsePartitionSpec(partSpec).get
    +      val fromTableIdent = base.extractTableIdent(fromTable)
    +      AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPARTS", args) :: rest =>
    +      val parts = args.collect {
    +        case Token("TOK_PARTSPEC", partitions) =>
    +          partitions.map {
    +            case Token("TOK_PARTVAL", ident :: op :: constant :: Nil) =>
    +              (unquoteString(cleanIdentifier(ident.text)),
    +                op.text, unquoteString(cleanIdentifier(constant.text)))
    +          }
    +      }
    +
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      val purge = base.getClauseOption("PURGE", args)
    +
    +      val replication = base.getClauseOption("TOK_REPLICATION", args).map {
    +        case Token("TOK_REPLICATION", replId :: metadata :: Nil) =>
    +          (unquoteString(cleanIdentifier(replId.text)), true)
    +        case Token("TOK_REPLICATION", replId :: Nil) =>
    +          (unquoteString(cleanIdentifier(replId.text)), false)
    +      }
    +
    +      AlterTableDropPartition(
    +        tableIdent,
    +        parts,
    +        allowExisting.isDefined,
    +        purge.isDefined,
    +        replication)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ARCHIVE", args) :: rest =>
    +      val partition = parsePartitionSpec(args(0)).get
    +      AlterTableArchivePartition(tableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_UNARCHIVE", args) :: rest =>
    +      val partition = parsePartitionSpec(args(0)).get
    +      AlterTableUnarchivePartition(tableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_FILEFORMAT", args) :: rest =>
    +      val Seq(fileFormat, genericFormat) =
    +        base.getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"),
    +          args)
    +      val fFormat = fileFormat.map(_.children.map(n => unquoteString(cleanIdentifier(n.text))))
    +      val gFormat = genericFormat.map(f => unquoteString(cleanIdentifier(f.children(0).text)))
    +      AlterTableSetFileFormat(tableIdent, partition, fFormat, gFormat)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_LOCATION", Token(loc, Nil) :: Nil) :: rest =>
    +      AlterTableSetLocation(tableIdent, partition, unquoteString(cleanIdentifier(loc)))(node.source)
    +
    +    case Token("TOK_ALTERTABLE_TOUCH", args) :: rest =>
    +      val part = base.getClauseOption("TOK_PARTSPEC", args).flatMap(parsePartitionSpec)
    +      AlterTableTouch(tableIdent, part)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_COMPACT", Token(compactType, Nil) :: Nil) :: rest =>
    +      AlterTableCompact(tableIdent, partition,
    +        unquoteString(cleanIdentifier(compactType)))(node.source)
    +
    +    case Token("TOK_ALTERTABLE_MERGEFILES", _) :: rest =>
    +      AlterTableMerge(tableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMECOL", args) :: rest =>
    --- End diff --
    
    We know that at least 3 arguments will be returned... Lets use a pattern match


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-188714057
  
    **[Test build #51951 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51951/consoleFull)** for PR 11048 at commit [`5f8e70d`](https://github.com/apache/spark/commit/5f8e70d7dfd7c7c3efb640210e9a0798dffad964).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StorageStatusListener(conf: SparkConf) extends SparkListener `
      * `class FPGrowthModel(JavaModelWrapper, JavaSaveable, JavaLoader):`
      * `class ExpressionSet protected(`
      * `public abstract class BufferedRowIterator `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-184686192
  
    **[Test build #51361 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51361/consoleFull)** for PR 11048 at commit [`4a96331`](https://github.com/apache/spark/commit/4a963317165877036469abe9bc45eebfc725edf1).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53592672
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    +                val bucketCols = bucketArgs.head.children.map(_.text)
    +
    +                val (sortCols, sortDirections, numBuckets) = {
    +                  if (bucketArgs(1).text == "TOK_TABCOLNAME") {
    +                    val cols = bucketArgs(1).children.map {
    +                      case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Ascending)
    +                      case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) =>
    +                        (colName, Descending)
    +                    }
    +                    (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt)
    +                  } else {
    +                    (Nil, Nil, bucketArgs(1).text.toInt)
    +                  }
    +                }
    +
    +                (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)),
    +                  false, false)
    +              case Token("TOK_NOT_CLUSTERED", Nil) =>
    +                (None, true, false)
    +              case Token("TOK_NOT_SORTED", Nil) =>
    +                (None, false, true)
    +          }
    +      }
    +
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        noClustered,
    +        noSorted)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest =>
    +      val num = bucketNum.toInt
    +      val buckets = Some(BucketSpec(num, Nil, Nil, Nil))
    +      AlterTableStoreProperties(
    +        tableIdent,
    +        buckets,
    +        false,
    +        false)(node.source)
    +
    +    case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest =>
    +      // Alter Table not skewed
    +      // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed.
    +      val notSkewed = if (tableSkewed.children.size == 0) {
    +        true
    +      } else {
    +        false
    +      }
    +
    +      val (notStoredAsDirs, skewedArgs) = tableSkewed match {
    +        case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) =>
    +          // Alter Table not stored as directories
    +          (true, None)
    +        case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) =>
    +          val (cols, values, storedAsDirs) = skewedArgs match {
    +            case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) =>
    +              val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text)))
    +              val values = skewedValues match {
    +                case Token("TOK_TABCOLVALUE", values) =>
    +                  Seq(values.map(n => unquoteString(cleanIdentifier(n.text))))
    +                case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
    +                  pairs.map {
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +              }
    +
    +              val storedAsDirs = stored match {
    +                case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
    +                case _ => false
    +              }
    +
    +              (cols, values, storedAsDirs)
    +          }
    +          (false, Some((cols, values, storedAsDirs)))
    +      }
    +
    +      if (skewedArgs.isDefined) {
    +        AlterTableSkewed(
    +          tableIdent,
    +          skewedArgs.get._1, /* cols */
    +          skewedArgs.get._2, /* values */
    +          skewedArgs.get._3, /* storedAsDirs */
    +          notSkewed, notStoredAsDirs)(node.source)
    +      } else {
    +        AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source)
    +      }
    +
    +    case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest =>
    +      val skewedMaps = args(0) match {
    +        case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) =>
    +          locationList match {
    +            case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) =>
    +              locationMaps.map {
    +                case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) =>
    +                  val k = key match {
    +                    case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const)))
    +                    case Token("TOK_TABCOLVALUES", values :: Nil) =>
    +                      values match {
    +                        case Token("TOK_TABCOLVALUE", vals) =>
    +                          vals.map(n => unquoteString(cleanIdentifier(n.text)))
    +                      }
    +                  }
    +                  (k, unquoteString(cleanIdentifier(value.text)))
    +              }.toMap
    +          }
    +      }
    +      AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest =>
    +      val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs)
    +      val parts = if (allowExisting.isDefined) {
    +        addPartsArgs.tail
    +      } else {
    +        addPartsArgs
    +      }
    +
    +      val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] =
    +        new ArrayBuffer()
    +      var currentPart: Map[String, Option[String]] = null
    +      parts.map {
    +        case t @ Token("TOK_PARTSPEC", partArgs) =>
    +          if (currentPart != null) {
    +            partitions += ((currentPart, None))
    +          }
    +          currentPart = parsePartitionSpec(t).get
    +        case Token("TOK_PARTITIONLOCATION", loc :: Nil) =>
    +          val location = unquoteString(loc.text)
    +          if (currentPart != null) {
    +            partitions += ((currentPart, Some(location)))
    +            currentPart = null
    +          } else {
    +            // We should not reach here
    +            throw new AnalysisException("Partition location must follow a partition spec.")
    +          }
    +      }
    +
    +      if (currentPart != null) {
    +        partitions += ((currentPart, None))
    +      }
    +      AlterTableAddPartition(tableIdent, partitions, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest =>
    +      val newPartition = parsePartitionSpec(args(0))
    +      AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", args) :: rest =>
    +      val Seq(Some(partSpec), Some(fromTable)) =
    +        base.getClauses(Seq("TOK_PARTSPEC", "TOK_TABNAME"), args)
    +      val partition = parsePartitionSpec(partSpec).get
    +      val fromTableIdent = base.extractTableIdent(fromTable)
    +      AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPARTS", args) :: rest =>
    +      val parts = args.collect {
    +        case Token("TOK_PARTSPEC", partitions) =>
    +          partitions.map {
    +            case Token("TOK_PARTVAL", ident :: op :: constant :: Nil) =>
    +              (unquoteString(cleanIdentifier(ident.text)),
    +                op.text, unquoteString(cleanIdentifier(constant.text)))
    +          }
    +      }
    +
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      val purge = base.getClauseOption("PURGE", args)
    +
    +      val replication = base.getClauseOption("TOK_REPLICATION", args).map {
    +        case Token("TOK_REPLICATION", replId :: metadata :: Nil) =>
    +          (unquoteString(cleanIdentifier(replId.text)), true)
    +        case Token("TOK_REPLICATION", replId :: Nil) =>
    +          (unquoteString(cleanIdentifier(replId.text)), false)
    +      }
    +
    +      AlterTableDropPartition(
    +        tableIdent,
    +        parts,
    +        allowExisting.isDefined,
    +        purge.isDefined,
    +        replication)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_ARCHIVE", args) :: rest =>
    +      val partition = parsePartitionSpec(args(0)).get
    +      AlterTableArchivePartition(tableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_UNARCHIVE", args) :: rest =>
    +      val partition = parsePartitionSpec(args(0)).get
    +      AlterTableUnarchivePartition(tableIdent, partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_FILEFORMAT", args) :: rest =>
    +      val Seq(fileFormat, genericFormat) =
    +        base.getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"),
    +          args)
    +      val fFormat = fileFormat.map(_.children.map(n => unquoteString(cleanIdentifier(n.text))))
    +      val gFormat = genericFormat.map(f => unquoteString(cleanIdentifier(f.children(0).text)))
    +      AlterTableSetFileFormat(tableIdent, partition, fFormat, gFormat)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_LOCATION", Token(loc, Nil) :: Nil) :: rest =>
    +      AlterTableSetLocation(tableIdent, partition, unquoteString(cleanIdentifier(loc)))(node.source)
    +
    +    case Token("TOK_ALTERTABLE_TOUCH", args) :: rest =>
    +      val part = base.getClauseOption("TOK_PARTSPEC", args).flatMap(parsePartitionSpec)
    --- End diff --
    
    Yes.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53571294
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala ---
    @@ -0,0 +1,533 @@
    +/*
    --- End diff --
    
    +1


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r53602175
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala ---
    @@ -0,0 +1,420 @@
    +/*
    + * 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.commands
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.{AnalysisException, SaveMode}
    +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
    +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.commands._
    +import org.apache.spark.sql.execution.datasources._
    +import org.apache.spark.sql.types.StructType
    +
    +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
    +
    +  def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = {
    +    node match {
    +      case Token("TOK_PARTSPEC", partitions) =>
    +        val spec = partitions.map {
    +          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)),
    +              Some(unquoteString(cleanIdentifier(constant.text))))
    +          case Token("TOK_PARTVAL", ident :: Nil) =>
    +            (unquoteString(cleanIdentifier(ident.text)), None)
    +        }.toMap
    +        Some(spec)
    +      case _ => None
    +    }
    +  }
    +
    +  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match {
    +    case Token("TOK_TABLEPROPERTIES", propsList) =>
    +      propsList.flatMap {
    +        case Token("TOK_TABLEPROPLIST", props) =>
    +          props.map {
    +            case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              (k, None)
    +            case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
    +              val k = unquoteString(cleanIdentifier(key.text))
    +              val v = unquoteString(cleanIdentifier(value.text))
    +              (k, Some(v))
    +          }
    +      }.toMap
    +  }
    +
    +  override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE"
    +
    +  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
    +    case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
    +      val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
    +      val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
    +      val partition = partitionSpec.flatMap(parsePartitionSpec)
    +      matchAlterTableCommands(v1, rest, tableIdent, partition)
    +    case _ =>
    +      throw new NotImplementedError(v1.text)
    +  }
    +
    +  def matchAlterTableCommands(
    +      node: ASTNode,
    +      nodes: Seq[ASTNode],
    +      tableIdent: TableIdentifier,
    +      partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match {
    +    case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
    +      val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
    +      val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable)
    +      AlterTableRename(tableIdent, renamedTableIdent)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
    +      val setTableProperties = extractTableProps(args.head)
    +      AlterTableSetProperties(
    +        tableIdent,
    +        setTableProperties)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
    +      val dropTableProperties = extractTableProps(args.head)
    +      val allowExisting = base.getClauseOption("TOK_IFEXISTS", args)
    +      AlterTableDropProperties(
    +        tableIdent,
    +        dropTableProperties, allowExisting.isDefined)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest =>
    +      val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text))
    +
    +      val serdeProperties: Option[Map[String, Option[String]]] = Option(
    +        // SET SERDE serde_classname WITH SERDEPROPERTIES
    +        if (serdeArgs.tail.isEmpty) {
    +          null
    +        } else {
    +          extractTableProps(serdeArgs.tail.head)
    +        }
    +      )
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        Some(serdeClassName),
    +        serdeProperties,
    +        partition)(node.source)
    +
    +    case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest =>
    +      val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head)
    +
    +      AlterTableSerDeProperties(
    +        tableIdent,
    +        None,
    +        Some(serdeProperties),
    +        partition)(node.source)
    +
    +    case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest =>
    +      val (buckets, noClustered, noSorted) = bucketSpec match {
    +        case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) =>
    +          clusterAndSoryByArgs match {
    +              case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) =>
    --- End diff --
    
    Because that rule `tableBuckets` is used in other places, I think we better don't touch it.


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

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


[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187515501
  
    **[Test build #51725 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51725/consoleFull)** for PR 11048 at commit [`54c0578`](https://github.com/apache/spark/commit/54c0578db8bb19439723b392d91b4cd5c20da962).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with 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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#issuecomment-187468650
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51708/
    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-13139][SQL] Create native DDL commands

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

    https://github.com/apache/spark/pull/11048#discussion_r54793638
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala ---
    @@ -41,6 +41,10 @@ trait Catalog {
         throw new UnsupportedOperationException
       }
     
    +  def runNativeCommand(sql: String): Seq[Row] = {
    --- End diff --
    
    Running commands shouldn't be part of the catalog interface. This is only used in 1 place anyway so I wouldn't bother making an abstraction 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