[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request: [SPARK-13139][SQL][WIP] Create native DDL comm...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[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 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org