You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/09/30 00:49:52 UTC

[GitHub] [spark] huaxingao opened a new pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

huaxingao opened a new pull request #34148:
URL: https://github.com/apache/spark/pull/34148


   
   
   ### What changes were proposed in this pull request?
   This is the 2nd PR for DSv2 index support.
   
   This PR adds the following:
   
   - create index syntax support in parser and analyzer
   - `CreateIndex` logic node
   - `CreateIndexExec` physical node
   
   `CreateIndex` is not implemented yet in this PR. Calling `CreateIndex` will throw `SQLFeatureNotSupportedException`, and the parsed index information such as `IndexName` `indexType` `columns` and index properties will be included in the error message for now for testing purpose.
   
   ### Why are the changes needed?
   To support index in DSv2
   
   
   ### Does this PR introduce _any_ user-facing change?
   Yes, the create table syntax as the following:
   
   ```
   CREATE [index_type] INDEX [index_name] ON [TABLE] table_name (column_index_property_list)[OPTIONS indexPropertyList]
   
       column_index_property_list: column_name [OPTIONS(indexPropertyList)]  [ ,  . . . ]
       indexPropertyList: index_property_name = index_property_value [ ,  . . . ]
   ```
   
   ### How was this patch tested?
   add a UT


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] viirya commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r720784528



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -427,6 +427,11 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
       val table = a.table.asInstanceOf[ResolvedTable]
       AlterTableExec(table.catalog, table.identifier, a.changes) :: Nil
 
+    case CreateIndex(ResolvedDBObjectName(catalog, table),

Review comment:
       ditto. I think we cannot create index on arbitrary database object.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932856319


   **[Test build #143808 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143808/testReport)** for PR 34148 at commit [`8248e60`](https://github.com/apache/spark/commit/8248e601d3f0dca30a85906b6c0f7f527b83fbe5).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-931559396


   **[Test build #143771 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143771/testReport)** for PR 34148 at commit [`0024096`](https://github.com/apache/spark/commit/0024096e342ab08ab9ba9ccfe8c8576780a95fcb).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-943093169


   @cloud-fan I have changed `CREATE [index_type] INDEX ...` to `CREATE INDEX index_name ON [TABLE] table_name [USING index_type]`. Please take a look again when you have time. Thanks a lot!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-930645448


   **[Test build #143737 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143737/testReport)** for PR 34148 at commit [`f2d7e76`](https://github.com/apache/spark/commit/f2d7e76d73cca0ae91803736d34d1a26dba794c7).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-938063377


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48475/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-937712311


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48448/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-937419912


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48418/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946443504


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48869/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946143893


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48853/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r730987088



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.datasources.v2
+
+import java.util
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    table: SupportsIndex,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[(NamedReference, Map[String, String])],
+    properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+    try {
+      val colProperties: Array[util.Map[NamedReference, util.Map[String, String]]] = columns.map {
+        case (column, map) =>
+          val javaMap = new util.HashMap[String, String]
+          map.foreach { case (key, value) => javaMap.put(key, value) }
+          val propMap = new util.HashMap[NamedReference, util.Map[String, String]]
+          propMap.put(column, javaMap)
+          propMap
+      }.toArray
+      val indexProperties = new util.HashMap[String, String]
+      properties.foreach { case (key, value) => indexProperties.put(key, value) }
+      table match {
+        case _: SupportsIndex =>

Review comment:
       we don't need this pattern match now.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945272154


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48814/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-952217588


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/49098/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934808940


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143858/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932888908


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143808/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] viirya commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r720783018



##########
File path: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
##########
@@ -110,9 +110,9 @@ statement
     | CREATE namespace (IF NOT EXISTS)? multipartIdentifier
         (commentSpec |
          locationSpec |
-         (WITH (DBPROPERTIES | PROPERTIES) tablePropertyList))*        #createNamespace
+         (WITH (DBPROPERTIES | PROPERTIES) propertyList))*             #createNamespace

Review comment:
       Seems you add some additional spaces?

##########
File path: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
##########
@@ -110,9 +110,9 @@ statement
     | CREATE namespace (IF NOT EXISTS)? multipartIdentifier
         (commentSpec |
          locationSpec |
-         (WITH (DBPROPERTIES | PROPERTIES) tablePropertyList))*        #createNamespace
+         (WITH (DBPROPERTIES | PROPERTIES) propertyList))*             #createNamespace

Review comment:
       Seems you add some additional spaces? Can you remove them?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939379621


   **[Test build #144058 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144058/testReport)** for PR 34148 at commit [`9557c49`](https://github.com/apache/spark/commit/9557c494da6b4e20fa194707dc808d679da203b6).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945273547


   **[Test build #144338 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144338/testReport)** for PR 34148 at commit [`8c96ebc`](https://github.com/apache/spark/commit/8c96ebc0fff70a3e04ba23782e71d7d35978dc01).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946063918


   **[Test build #144379 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144379/testReport)** for PR 34148 at commit [`8eb052a`](https://github.com/apache/spark/commit/8eb052a5d2b17ed0dd07ca93514348357ffe400c).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946063918


   **[Test build #144379 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144379/testReport)** for PR 34148 at commit [`8eb052a`](https://github.com/apache/spark/commit/8eb052a5d2b17ed0dd07ca93514348357ffe400c).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r731511773



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/index/SupportsIndex.java
##########
@@ -38,7 +37,8 @@
    * Creates an index.
    *
    * @param indexName the name of the index to be created
-   * @param indexType the IndexType of the index to be created
+   * @param indexType the type of the index to be created. If this is not specified, Spark
+   *                  will use empty String.

Review comment:
       Will fix this in next PR when adding the DROP INDEX syntax




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945404075


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-942956384


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144220/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946340558


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144381/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-952184754


   **[Test build #144628 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144628/testReport)** for PR 34148 at commit [`fe0d4d3`](https://github.com/apache/spark/commit/fe0d4d34f77a1cc0129485a1e0c0608d7e0dc7a9).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-952245675


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/49098/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-930695892


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48248/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945048212






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-933002759


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48322/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932866566


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48320/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939407874


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144058/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945048216






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] c21 commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r722967709



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##########
@@ -4492,6 +4492,39 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
     CommentOnTable(createUnresolvedTable(ctx.multipartIdentifier, "COMMENT ON TABLE"), comment)
   }
 
+  /**
+   * Create an index, returning a [[CreateIndex]] logical plan.
+   * For example:
+   * {{{
+   * CREATE [index_type] INDEX index_name ON [TABLE] table_name (column_index_property_list)
+   *   [OPTIONS indexPropertyList]
+   *   column_index_property_list: column_name [OPTIONS(indexPropertyList)]  [ ,  . . . ]
+   *   indexPropertyList: index_property_name [= index_property_value] [ ,  . . . ]
+   * }}}
+   */
+  override def visitCreateIndex(ctx: CreateIndexContext): LogicalPlan = withOrigin(ctx) {
+    val (indexType, indexName) = if (ctx.identifier.size() == 1) {
+      (null, ctx.identifier(0).getText)

Review comment:
       Sorry just for my own understanding, why do we allow creating index without index type?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934808940


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143858/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-931559396


   **[Test build #143771 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143771/testReport)** for PR 34148 at commit [`0024096`](https://github.com/apache/spark/commit/0024096e342ab08ab9ba9ccfe8c8576780a95fcb).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] viirya commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r720784004



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##########
@@ -4492,6 +4492,37 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
     CommentOnTable(createUnresolvedTable(ctx.multipartIdentifier, "COMMENT ON TABLE"), comment)
   }
 
+  /**
+   * Create an index, returning a [[CreateIndex]] logical plan.
+   * For example:
+   * {{{
+   * CREATE [index_type] INDEX [index_name] ON [TABLE] table_name (column_index_property_list)
+   *   [OPTIONS indexPropertyList]
+   *   column_index_property_list: column_name [OPTIONS(indexPropertyList)]  [ ,  . . . ]
+   *   indexPropertyList: index_property_name = index_property_value [ ,  . . . ]

Review comment:
       Hmm, is `index_property_value` required or optional?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945229672


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48813/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945217151


   **[Test build #144335 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144335/testReport)** for PR 34148 at commit [`748157e`](https://github.com/apache/spark/commit/748157e8356c8a3920e7c8a9f54596d54fad3fab).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945273547


   **[Test build #144338 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144338/testReport)** for PR 34148 at commit [`8c96ebc`](https://github.com/apache/spark/commit/8c96ebc0fff70a3e04ba23782e71d7d35978dc01).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946002268


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48849/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] sunchao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r731187473



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/index/SupportsIndex.java
##########
@@ -38,7 +37,8 @@
    * Creates an index.
    *
    * @param indexName the name of the index to be created
-   * @param indexType the IndexType of the index to be created
+   * @param indexType the IndexType of the index to be created. If this is not specified, Spark

Review comment:
       nit: "the IndexType of the index to be created" -> "the type of the index to be created"?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.datasources.v2
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    table: SupportsIndex,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[(NamedReference, Map[String, String])],
+    properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+    try {
+      val colProperties: Array[util.Map[NamedReference, util.Map[String, String]]] = columns.map {
+        case (column, map) =>
+          val propMap = new util.HashMap[NamedReference, util.Map[String, String]]
+          propMap.put(column, map.asJava)
+          propMap
+      }.toArray
+      val (cols, _) = columns.unzip
+      table.createIndex(indexName, indexType, cols.toArray, colProperties, properties.asJava)
+    } catch {
+      case _: IndexAlreadyExistsException if ignoreIfExists =>
+        logWarning(s"Index ${indexName} already exists. Ignoring.")

Review comment:
       maybe print out the table name too

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.datasources.v2
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    table: SupportsIndex,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[(NamedReference, Map[String, String])],
+    properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+    try {
+      val colProperties: Array[util.Map[NamedReference, util.Map[String, String]]] = columns.map {
+        case (column, map) =>
+          val propMap = new util.HashMap[NamedReference, util.Map[String, String]]
+          propMap.put(column, map.asJava)
+          propMap

Review comment:
       hmm why we are creating a new `propMap` for every column?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946100051


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48853/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946195449


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144379/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946195449


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144379/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r731365809



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.datasources.v2
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    table: SupportsIndex,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[(NamedReference, Map[String, String])],
+    properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+    try {
+      val colProperties: Array[util.Map[NamedReference, util.Map[String, String]]] = columns.map {
+        case (column, map) =>
+          val propMap = new util.HashMap[NamedReference, util.Map[String, String]]
+          propMap.put(column, map.asJava)
+          propMap

Review comment:
       I see what you mean now. Changed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946247941


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48855/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946340558


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144381/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946385318


   **[Test build #144395 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144395/testReport)** for PR 34148 at commit [`57fbf67`](https://github.com/apache/spark/commit/57fbf67cb1808c0489872854e66a9abcd68881e6).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-942867218


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48699/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r720852466



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##########
@@ -4492,6 +4492,37 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
     CommentOnTable(createUnresolvedTable(ctx.multipartIdentifier, "COMMENT ON TABLE"), comment)
   }
 
+  /**
+   * Create an index, returning a [[CreateIndex]] logical plan.
+   * For example:
+   * {{{
+   * CREATE [index_type] INDEX [index_name] ON [TABLE] table_name (column_index_property_list)
+   *   [OPTIONS indexPropertyList]
+   *   column_index_property_list: column_name [OPTIONS(indexPropertyList)]  [ ,  . . . ]
+   *   indexPropertyList: index_property_name = index_property_value [ ,  . . . ]
+   * }}}
+   */
+  override def visitCreateIndex(ctx: CreateIndexContext): LogicalPlan = withOrigin(ctx) {
+    val indexName = ctx.identifier.getText
+    val indexType = if (ctx.indexType == null) "" else ctx.indexType.getText
+    val columns = ctx.columns.multipartIdentifierProperty.asScala
+      .map(_.multipartIdentifier.getText).toSeq
+    val columnsProperties = ctx.columns.multipartIdentifierProperty.asScala
+      .map(x => (Option(x.options).map(visitPropertyKeyValues).getOrElse(Map.empty))).toSeq
+    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
+
+    CreateIndex(
+      UnresolvedDBObjectName(
+        visitMultipartIdentifier(ctx.multipartIdentifier),
+        isNamespace = true),

Review comment:
       You are right. Should be `UnresolvedTable`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932866566


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48320/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-942884164


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48699/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-942884164


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48699/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-937680061


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48448/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-930695892


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48248/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945208234


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144334/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] sunchao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r731257013



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.datasources.v2
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    table: SupportsIndex,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[(NamedReference, Map[String, String])],
+    properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+    try {
+      val colProperties: Array[util.Map[NamedReference, util.Map[String, String]]] = columns.map {
+        case (column, map) =>
+          val propMap = new util.HashMap[NamedReference, util.Map[String, String]]
+          propMap.put(column, map.asJava)
+          propMap

Review comment:
       It's just a bit strange that we are creating a map with only one entry (column -> its property map) for each column, and then we pass them as an array to `createIndex`? why can't we just create a single map with type `Map[NamedReference, util.Map[String, String]]` instead?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946635089


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144395/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r730988347



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.datasources.v2
+
+import java.util
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    table: SupportsIndex,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[(NamedReference, Map[String, String])],
+    properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+    try {
+      val colProperties: Array[util.Map[NamedReference, util.Map[String, String]]] = columns.map {
+        case (column, map) =>
+          val javaMap = new util.HashMap[String, String]
+          map.foreach { case (key, value) => javaMap.put(key, value) }
+          val propMap = new util.HashMap[NamedReference, util.Map[String, String]]

Review comment:
       Can we use the scala convert to turn scala map to java map? import the implicits and simply do `map.asJava`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r730988930



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.datasources.v2
+
+import java.util
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    table: SupportsIndex,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[(NamedReference, Map[String, String])],
+    properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+    try {
+      val colProperties: Array[util.Map[NamedReference, util.Map[String, String]]] = columns.map {
+        case (column, map) =>
+          val javaMap = new util.HashMap[String, String]
+          map.foreach { case (key, value) => javaMap.put(key, value) }
+          val propMap = new util.HashMap[NamedReference, util.Map[String, String]]
+          propMap.put(column, javaMap)
+          propMap
+      }.toArray
+      val indexProperties = new util.HashMap[String, String]

Review comment:
       ditto




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946143340


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144375/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946269917


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48855/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946339873


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r731511373



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.datasources.v2
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    table: SupportsIndex,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[(NamedReference, Map[String, String])],
+    properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+    try {

Review comment:
       fixed

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
##########
@@ -178,6 +177,7 @@ private case object MySQLDialect extends JdbcDialect with SQLConfHelper {
       tableName: String,
       options: JDBCOptions): Array[TableIndex] = {
     val sql = s"SHOW INDEXES FROM $tableName"
+    Array.empty[util.Map[NamedReference, util.HashMap[String, String]]]

Review comment:
       Sorry. Removed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945332904


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48817/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945407474


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144338/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r723433243



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##########
@@ -4492,6 +4492,39 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
     CommentOnTable(createUnresolvedTable(ctx.multipartIdentifier, "COMMENT ON TABLE"), comment)
   }
 
+  /**
+   * Create an index, returning a [[CreateIndex]] logical plan.
+   * For example:
+   * {{{
+   * CREATE [index_type] INDEX index_name ON [TABLE] table_name (column_index_property_list)
+   *   [OPTIONS indexPropertyList]
+   *   column_index_property_list: column_name [OPTIONS(indexPropertyList)]  [ ,  . . . ]
+   *   indexPropertyList: index_property_name [= index_property_value] [ ,  . . . ]
+   * }}}
+   */
+  override def visitCreateIndex(ctx: CreateIndexContext): LogicalPlan = withOrigin(ctx) {
+    val (indexType, indexName) = if (ctx.identifier.size() == 1) {
+      (null, ctx.identifier(0).getText)

Review comment:
       If index type is not specified explicitly, the default value will be used. For example, MySQL uses BTREE as index type if  index type is not provided.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934673339


   **[Test build #143860 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143860/testReport)** for PR 34148 at commit [`cf1e821`](https://github.com/apache/spark/commit/cf1e821fff29ba7c2594e61d21a2e92304142b05).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-937716602


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48448/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r721875897



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.datasources.v2
+
+import java.sql.SQLFeatureNotSupportedException
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog}
+import org.apache.spark.sql.connector.expressions.FieldReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    catalog: TableCatalog,
+    ident: Identifier,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[FieldReference],
+    columnProperties: Seq[Map[String, String]],
+    private var properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+
+    try {
+      // Todo: replace this with the createIndex

Review comment:
       The actual implementation is as simple as calling v2 `createIndex` API, right? I think we should throw exception in the testing v2 source instead.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-930691640


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48248/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934806194


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934801362


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48373/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-930744501


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143737/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939380754


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48535/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945048216






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939407874


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144058/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939391467






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r728855880



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -429,6 +429,11 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
       val table = a.table.asInstanceOf[ResolvedTable]
       AlterTableExec(table.catalog, table.identifier, a.changes) :: Nil
 
+    case CreateIndex(ResolvedTable(catalog, identifier, _, _),
+        indexName, indexType, ifNotExists, columns, columnProperties, properties) =>
+      CreateIndexExec(catalog, identifier, indexName, indexType, ifNotExists, columns,

Review comment:
       can't we pass `ResolvedTable.table` to the physical plan?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-937716602


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48448/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-930662486


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48248/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932376633


   @HyukjinKwon Thanks for taking a look! I am implementing this in JDBC right now, so I can test out the new code in index support.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-931774842


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143771/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r728856511



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import java.sql.SQLFeatureNotSupportedException
+import java.util
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog}
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    catalog: TableCatalog,
+    ident: Identifier,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[NamedReference],
+    columnProperties: Seq[Map[String, String]],
+    private var properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+    try {
+      val table = catalog.loadTable(ident)
+      val colProperties: Array[util.Map[NamedReference, util.Properties]] =
+        columns.zip(columnProperties).map {
+        case (column, map) =>
+          val props = new util.Properties
+          map.foreach { case (key, value) => props.setProperty(key, value) }
+          val propMap = new util.HashMap[NamedReference, util.Properties]()
+          propMap.put(column, props)
+          propMap
+      }.toArray
+      val indexProperties = new util.Properties
+      properties.foreach { case (key, value) => indexProperties.setProperty(key, value) }
+      table match {
+        case _: SupportsIndex =>
+          table.asInstanceOf[SupportsIndex].createIndex(indexName, indexType, columns.toArray,
+            colProperties, indexProperties)
+        case _ => throw new SQLFeatureNotSupportedException(s"CreateIndex not supported yet." +
+          s" IndexName $indexName indexType $indexType columns $columns" +
+          s" columnProperties $columnProperties properties $properties")

Review comment:
       can we move this check to the planner rule? then at least we can fail at query compilation time




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934625470


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48371/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934625470


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48371/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-930645448


   **[Test build #143737 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143737/testReport)** for PR 34148 at commit [`f2d7e76`](https://github.com/apache/spark/commit/f2d7e76d73cca0ae91803736d34d1a26dba794c7).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-931634507


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48282/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r720852441



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##########
@@ -4492,6 +4492,37 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
     CommentOnTable(createUnresolvedTable(ctx.multipartIdentifier, "COMMENT ON TABLE"), comment)
   }
 
+  /**
+   * Create an index, returning a [[CreateIndex]] logical plan.
+   * For example:
+   * {{{
+   * CREATE [index_type] INDEX [index_name] ON [TABLE] table_name (column_index_property_list)
+   *   [OPTIONS indexPropertyList]
+   *   column_index_property_list: column_name [OPTIONS(indexPropertyList)]  [ ,  . . . ]
+   *   indexPropertyList: index_property_name = index_property_value [ ,  . . . ]

Review comment:
       It's optional. I forgot to put this in `[]`.
   I double checked oracle and MySQL reference, the index properties could have name without value.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] viirya commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r720784317



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##########
@@ -4492,6 +4492,37 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
     CommentOnTable(createUnresolvedTable(ctx.multipartIdentifier, "COMMENT ON TABLE"), comment)
   }
 
+  /**
+   * Create an index, returning a [[CreateIndex]] logical plan.
+   * For example:
+   * {{{
+   * CREATE [index_type] INDEX [index_name] ON [TABLE] table_name (column_index_property_list)
+   *   [OPTIONS indexPropertyList]
+   *   column_index_property_list: column_name [OPTIONS(indexPropertyList)]  [ ,  . . . ]
+   *   indexPropertyList: index_property_name = index_property_value [ ,  . . . ]
+   * }}}
+   */
+  override def visitCreateIndex(ctx: CreateIndexContext): LogicalPlan = withOrigin(ctx) {
+    val indexName = ctx.identifier.getText
+    val indexType = if (ctx.indexType == null) "" else ctx.indexType.getText
+    val columns = ctx.columns.multipartIdentifierProperty.asScala
+      .map(_.multipartIdentifier.getText).toSeq
+    val columnsProperties = ctx.columns.multipartIdentifierProperty.asScala
+      .map(x => (Option(x.options).map(visitPropertyKeyValues).getOrElse(Map.empty))).toSeq
+    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
+
+    CreateIndex(
+      UnresolvedDBObjectName(
+        visitMultipartIdentifier(ctx.multipartIdentifier),
+        isNamespace = true),

Review comment:
       We only can create index on a table, right? If so, this sounds to be `UnresolvedTable` and we can use `createUnresolvedTable(...)`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r728855322



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import java.sql.SQLFeatureNotSupportedException
+import java.util
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog}
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    catalog: TableCatalog,
+    ident: Identifier,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[NamedReference],
+    columnProperties: Seq[Map[String, String]],
+    private var properties: Map[String, String])
+  extends LeafV2CommandExec {

Review comment:
       ```suggestion
       private var properties: Map[String, String]) extends LeafV2CommandExec {
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932856319


   **[Test build #143808 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143808/testReport)** for PR 34148 at commit [`8248e60`](https://github.com/apache/spark/commit/8248e601d3f0dca30a85906b6c0f7f527b83fbe5).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945338931


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946403957


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48869/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946265116


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48855/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] sunchao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r731188993



##########
File path: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
##########
@@ -247,6 +247,10 @@ statement
     | SET .*?                                                          #setConfiguration
     | RESET configKey                                                  #resetQuotedConfiguration
     | RESET .*?                                                        #resetConfiguration
+    | CREATE INDEX (IF NOT EXISTS)? identifier ON TABLE?

Review comment:
       just curious why `TABLE` is optional here?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946443504


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48869/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934625442


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48371/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932985310


   **[Test build #143810 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143810/testReport)** for PR 34148 at commit [`bc71de3`](https://github.com/apache/spark/commit/bc71de3c94645763bf16b765b9274bbd46041f36).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-931595721


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48282/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r721877029



##########
File path: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
##########
@@ -244,6 +244,9 @@ statement
     | SET .*?                                                          #setConfiguration
     | RESET configKey                                                  #resetQuotedConfiguration
     | RESET .*?                                                        #resetConfiguration
+    | CREATE indexType=STRING? INDEX (IF NOT EXISTS)? identifier ON TABLE?

Review comment:
       ```suggestion
       | CREATE indexType=identifier? INDEX (IF NOT EXISTS)? identifier ON TABLE?
   ```
   
   STRING only matches `'abc'`, but we want to support `CREATE bloom_filter INDEX ...`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-937419912


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48418/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939390588


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48536/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934673339


   **[Test build #143860 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143860/testReport)** for PR 34148 at commit [`cf1e821`](https://github.com/apache/spark/commit/cf1e821fff29ba7c2594e61d21a2e92304142b05).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939379000


   **[Test build #144057 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144057/testReport)** for PR 34148 at commit [`e32a053`](https://github.com/apache/spark/commit/e32a053fe9b6f782b0117625b3ecd9f4b76aedc0).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-935087194


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143860/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932999847


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48322/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-942814747


   **[Test build #144220 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144220/testReport)** for PR 34148 at commit [`20fac1c`](https://github.com/apache/spark/commit/20fac1c412948a67b4b94c733d8db7a8775afeac).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-938063377


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48475/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-937400494






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-942956384


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144220/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945991584


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48849/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946195040


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r731500359



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
##########
@@ -178,6 +177,7 @@ private case object MySQLDialect extends JdbcDialect with SQLConfHelper {
       tableName: String,
       options: JDBCOptions): Array[TableIndex] = {
     val sql = s"SHOW INDEXES FROM $tableName"
+    Array.empty[util.Map[NamedReference, util.HashMap[String, String]]]

Review comment:
       ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945224698


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48814/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945208224


   **[Test build #144334 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144334/testReport)** for PR 34148 at commit [`ed862ed`](https://github.com/apache/spark/commit/ed862ed1a9c1837d647720b4036a23dec801ac8d).
    * This patch **fails Java style tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945206503


   **[Test build #144334 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144334/testReport)** for PR 34148 at commit [`ed862ed`](https://github.com/apache/spark/commit/ed862ed1a9c1837d647720b4036a23dec801ac8d).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945229672


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48813/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945332904


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48817/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946269917


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48855/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946385318


   **[Test build #144395 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144395/testReport)** for PR 34148 at commit [`57fbf67`](https://github.com/apache/spark/commit/57fbf67cb1808c0489872854e66a9abcd68881e6).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946635089


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144395/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] dbtsai closed pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
dbtsai closed pull request #34148:
URL: https://github.com/apache/spark/pull/34148


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r728857163



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import java.sql.SQLFeatureNotSupportedException
+import java.util
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog}
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    catalog: TableCatalog,
+    ident: Identifier,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[NamedReference],
+    columnProperties: Seq[Map[String, String]],
+    private var properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+    try {
+      val table = catalog.loadTable(ident)
+      val colProperties: Array[util.Map[NamedReference, util.Properties]] =
+        columns.zip(columnProperties).map {
+        case (column, map) =>
+          val props = new util.Properties
+          map.foreach { case (key, value) => props.setProperty(key, value) }
+          val propMap = new util.HashMap[NamedReference, util.Properties]()
+          propMap.put(column, props)
+          propMap
+      }.toArray
+      val indexProperties = new util.Properties
+      properties.foreach { case (key, value) => indexProperties.setProperty(key, value) }
+      table match {
+        case _: SupportsIndex =>
+          table.asInstanceOf[SupportsIndex].createIndex(indexName, indexType, columns.toArray,
+            colProperties, indexProperties)

Review comment:
       unrelated to this PR, why does the API prefer `Properties` over `Map<String, String>`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939379000


   **[Test build #144057 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144057/testReport)** for PR 34148 at commit [`e32a053`](https://github.com/apache/spark/commit/e32a053fe9b6f782b0117625b3ecd9f4b76aedc0).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-938019280


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48475/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939391467






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932864999


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48320/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-931766616


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r721911549



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.datasources.v2
+
+import java.sql.SQLFeatureNotSupportedException
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog}
+import org.apache.spark.sql.connector.expressions.FieldReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    catalog: TableCatalog,
+    ident: Identifier,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[FieldReference],
+    columnProperties: Seq[Map[String, String]],
+    private var properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+
+    try {
+      // Todo: replace this with the createIndex

Review comment:
       I have JDBC v2 source `createIndex` implemented in https://github.com/apache/spark/pull/34164. We can probably merge that PR first, and then I can remove this `Todo`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r721875897



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.datasources.v2
+
+import java.sql.SQLFeatureNotSupportedException
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog}
+import org.apache.spark.sql.connector.expressions.FieldReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    catalog: TableCatalog,
+    ident: Identifier,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[FieldReference],
+    columnProperties: Seq[Map[String, String]],
+    private var properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+
+    try {
+      // Todo: replace this with the createIndex

Review comment:
       The actual implementation is as simple as calling v2 `createIndex` API, right? I think we should throw exception in the JDBC v2 source instead.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934578126


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48371/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932888908


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143808/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939403395


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939403522


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144057/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945272154


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48814/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945355938


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144335/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945941596


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48849/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] sunchao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r731194906



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -429,6 +430,17 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
       val table = a.table.asInstanceOf[ResolvedTable]
       AlterTableExec(table.catalog, table.identifier, a.changes) :: Nil
 
+    case CreateIndex(ResolvedTable(_, _, table, _),
+        indexName, indexType, ifNotExists, columns, properties) =>
+      table match {
+        case s: SupportsIndex =>
+          CreateIndexExec(s, indexName, indexType, ifNotExists, columns, properties):: Nil
+        case _ => throw QueryCompilationErrors.tableIndexNotSupportedError(
+          s"CreateIndex is not supported in this table." +
+          s" IndexName $indexName indexType $indexType columns $columns" +

Review comment:
       nit: not sure if we should dump all these - it may not be properly formatted too




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r731216387



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/index/SupportsIndex.java
##########
@@ -38,7 +37,8 @@
    * Creates an index.
    *
    * @param indexName the name of the index to be created
-   * @param indexType the IndexType of the index to be created
+   * @param indexType the IndexType of the index to be created. If this is not specified, Spark

Review comment:
       Fixed

##########
File path: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
##########
@@ -247,6 +247,10 @@ statement
     | SET .*?                                                          #setConfiguration
     | RESET configKey                                                  #resetQuotedConfiguration
     | RESET .*?                                                        #resetConfiguration
+    | CREATE INDEX (IF NOT EXISTS)? identifier ON TABLE?

Review comment:
       that's the keyword `TABLE`, not table name. Most of the DBMS don't include this `TABLE` keyword in `CREATE INDEX`, but Delta Lake has this optional `TABLE` keyword.
   `CREATE BLOOMFILTER INDEX
   ON [TABLE] table_identifier
   [FOR COLUMNS(columnName1 [OPTIONS(..)], columnName2, ...)]
   [OPTIONS ( key1 [ = ] val1, key2 [ = ] val2, ... ) ]`

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.datasources.v2
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    table: SupportsIndex,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[(NamedReference, Map[String, String])],
+    properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+    try {
+      val colProperties: Array[util.Map[NamedReference, util.Map[String, String]]] = columns.map {
+        case (column, map) =>
+          val propMap = new util.HashMap[NamedReference, util.Map[String, String]]
+          propMap.put(column, map.asJava)
+          propMap

Review comment:
       because every column has its own property

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.datasources.v2
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    table: SupportsIndex,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[(NamedReference, Map[String, String])],
+    properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+    try {
+      val colProperties: Array[util.Map[NamedReference, util.Map[String, String]]] = columns.map {
+        case (column, map) =>
+          val propMap = new util.HashMap[NamedReference, util.Map[String, String]]
+          propMap.put(column, map.asJava)
+          propMap
+      }.toArray
+      val (cols, _) = columns.unzip
+      table.createIndex(indexName, indexType, cols.toArray, colProperties, properties.asJava)
+    } catch {
+      case _: IndexAlreadyExistsException if ignoreIfExists =>
+        logWarning(s"Index ${indexName} already exists. Ignoring.")

Review comment:
       added

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -429,6 +430,17 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
       val table = a.table.asInstanceOf[ResolvedTable]
       AlterTableExec(table.catalog, table.identifier, a.changes) :: Nil
 
+    case CreateIndex(ResolvedTable(_, _, table, _),
+        indexName, indexType, ifNotExists, columns, properties) =>
+      table match {
+        case s: SupportsIndex =>
+          CreateIndexExec(s, indexName, indexType, ifNotExists, columns, properties):: Nil
+        case _ => throw QueryCompilationErrors.tableIndexNotSupportedError(
+          s"CreateIndex is not supported in this table." +
+          s" IndexName $indexName indexType $indexType columns $columns" +

Review comment:
       I will keep table name only, and remove the rest




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] sunchao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r731257858



##########
File path: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
##########
@@ -247,6 +247,10 @@ statement
     | SET .*?                                                          #setConfiguration
     | RESET configKey                                                  #resetQuotedConfiguration
     | RESET .*?                                                        #resetConfiguration
+    | CREATE INDEX (IF NOT EXISTS)? identifier ON TABLE?

Review comment:
       yeah I understand this is a keyword, just not sure the reason why we make it optional instead of always requiring or not requiring it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946143893


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48853/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946224269


   **[Test build #144381 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144381/testReport)** for PR 34148 at commit [`5b5880c`](https://github.com/apache/spark/commit/5b5880cd054023def4af4912a7fd482930bc1c81).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934537932


   **[Test build #143858 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143858/testReport)** for PR 34148 at commit [`c314c6d`](https://github.com/apache/spark/commit/c314c6d645e52614bf350d8bb17c17bdde36d3ae).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r721876321



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala
##########
@@ -425,4 +425,45 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession {
       assert(m.contains("\"TABLEENGINENAME\" not found"))
     }
   }
+
+  test("CREATE INDEX") {

Review comment:
       can we also add parser UT?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932985321


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143810/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-931774842


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143771/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932859639


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48320/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-930703670


   @cloud-fan @viirya 
   Could you please take a look?
   `SQLKeywordSuite` failed because the new keywords `BLOOM_FILTER_INDEX`, `BTREE_INDEX` and `Z_ORDERING_INDEX`  for index type are not in the documentation yet. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-933002759


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48322/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932886219


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-931637145


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48282/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934537932


   **[Test build #143858 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143858/testReport)** for PR 34148 at commit [`c314c6d`](https://github.com/apache/spark/commit/c314c6d645e52614bf350d8bb17c17bdde36d3ae).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939403522


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144057/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939380754


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48535/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-937416422


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48418/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-937400494


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48418/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-935072692


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-937419912






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-942840469


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48699/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945047815


   **[Test build #144328 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144328/testReport)** for PR 34148 at commit [`ed8985c`](https://github.com/apache/spark/commit/ed8985c8e8879065b3e5655b37238b70f0b71f3b).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945310691


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48817/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-942954665


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945212801


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48813/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945206503


   **[Test build #144334 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144334/testReport)** for PR 34148 at commit [`ed862ed`](https://github.com/apache/spark/commit/ed862ed1a9c1837d647720b4036a23dec801ac8d).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945217151


   **[Test build #144335 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144335/testReport)** for PR 34148 at commit [`748157e`](https://github.com/apache/spark/commit/748157e8356c8a3920e7c8a9f54596d54fad3fab).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945237142


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48814/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946143340


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144375/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946143868


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48853/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] viirya commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r720783018



##########
File path: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
##########
@@ -110,9 +110,9 @@ statement
     | CREATE namespace (IF NOT EXISTS)? multipartIdentifier
         (commentSpec |
          locationSpec |
-         (WITH (DBPROPERTIES | PROPERTIES) tablePropertyList))*        #createNamespace
+         (WITH (DBPROPERTIES | PROPERTIES) propertyList))*             #createNamespace

Review comment:
       Seems you add some additional spaces? Can you remove them?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945047815


   **[Test build #144328 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144328/testReport)** for PR 34148 at commit [`ed8985c`](https://github.com/apache/spark/commit/ed8985c8e8879065b3e5655b37238b70f0b71f3b).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r719847220



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##########
@@ -4492,6 +4502,55 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
     CommentOnTable(createUnresolvedTable(ctx.multipartIdentifier, "COMMENT ON TABLE"), comment)
   }
 
+  /**
+   * Create an index, returning a [[CreateIndex]] logical plan.
+   * For example:
+   * {{{
+   * CREATE [index_type] INDEX [index_name] ON [TABLE] table_name (column_index_property_list)
+   *   [OPTIONS indexPropertyList]
+   *   column_index_property_list: column_name [OPTIONS(indexPropertyList)]  [ ,  . . . ]
+   *   indexPropertyList: index_property_name = index_property_value [ ,  . . . ]
+   * }}}
+   */
+  override def visitCreateIndex(ctx: CreateIndexContext): LogicalPlan = withOrigin(ctx) {
+    val indexName = ctx.identifier.getText
+    val indexType = if (ctx.indexType == null) "" else ctx.indexType.getText
+    val columns = ctx.columns.multipartIdentifierProperty.asScala
+      .map(_.multipartIdentifier.getText).toSeq
+    val columnsProperties = ctx.columns.multipartIdentifierProperty.asScala
+      .map(x => (Option(x.options).map(visitIndexPropertyKeyValues).getOrElse(Map.empty))).toSeq
+    val options = Option(ctx.options).map(visitIndexPropertyKeyValues).getOrElse(Map.empty)
+
+    CreateIndex(
+      UnresolvedDBObjectName(
+        visitMultipartIdentifier(ctx.multipartIdentifier),
+        isNamespace = true),
+      indexName,
+      indexType,
+      ctx.EXISTS != null,
+      columns.map(FieldReference(_).asInstanceOf[FieldReference]),
+      columnsProperties,
+      options)
+  }
+
+  /**
+   * Convert a index property list into a key-value map.
+   */
+  override def visitIndexPropertyList(
+      ctx: IndexPropertyListContext): Map[String, String] = withOrigin(ctx) {
+    val temp = ctx.property

Review comment:
       will remove




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932984542


   **[Test build #143810 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143810/testReport)** for PR 34148 at commit [`bc71de3`](https://github.com/apache/spark/commit/bc71de3c94645763bf16b765b9274bbd46041f36).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934756997


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48373/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934708083


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48373/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-931544491


   I removed these specific index types `BLOOM_FILTER_INDEX`, `BTREE_INDEX` and `Z_ORDERING_INDEX` for now to pass the `SQLKeywordSuite`. We can discuss what index types to support and document in next PR after Wenchen comes back from the holiday break.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-943503770


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48746/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r728859873



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
##########
@@ -128,7 +128,7 @@ private case object MySQLDialect extends JdbcDialect with SQLConfHelper {
         indexProperties = indexProperties + " " + s"$k $v"
       }
     }
-    val iType = if (indexType.isEmpty) {
+    val iType = if (indexType == null || indexType.isEmpty) {

Review comment:
       or we can follow table provider and define a reserved index property key like `index_type`. If `USING index_type` is not specified, we don't put this key in the index properties.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-943460270


   **[Test build #144267 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144267/testReport)** for PR 34148 at commit [`c116268`](https://github.com/apache/spark/commit/c116268b792bb5b31130230e17aa26e8e7d09d94).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932985321


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143810/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-938061013


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48475/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939379621


   **[Test build #144058 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144058/testReport)** for PR 34148 at commit [`9557c49`](https://github.com/apache/spark/commit/9557c494da6b4e20fa194707dc808d679da203b6).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r728858987



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
##########
@@ -128,7 +128,7 @@ private case object MySQLDialect extends JdbcDialect with SQLConfHelper {
         indexProperties = indexProperties + " " + s"$k $v"
       }
     }
-    val iType = if (indexType.isEmpty) {
+    val iType = if (indexType == null || indexType.isEmpty) {

Review comment:
       can we make it clear in the API doc? It seems better to avoid null, and we can write in the doc that Spark will pass empty string if index type is not specified.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r721874980



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##########
@@ -4492,6 +4492,35 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
     CommentOnTable(createUnresolvedTable(ctx.multipartIdentifier, "COMMENT ON TABLE"), comment)
   }
 
+  /**
+   * Create an index, returning a [[CreateIndex]] logical plan.
+   * For example:
+   * {{{
+   * CREATE [index_type] INDEX [index_name] ON [TABLE] table_name (column_index_property_list)

Review comment:
       ```suggestion
      * CREATE [index_type] INDEX index_name ON [TABLE] table_name (column_index_property_list)
   ```
   
   index name is not optional




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] sunchao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r720709548



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala
##########
@@ -425,4 +425,27 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession {
       assert(m.contains("\"TABLEENGINENAME\" not found"))
     }
   }
+
+  test("CREATE INDEX") {
+    withTable("h2.test.new_table") {
+      sql("CREATE TABLE h2.test.new_table(col1 INT, col2 STRING)")
+      val e1 = intercept[SQLFeatureNotSupportedException] {
+        sql("CREATE index i1 ON h2.test.new_table (col1)")
+      }.getMessage
+      assert(e1.contains("IndexName i1"))
+      assert(e1.contains("indexType "))
+      assert(e1.contains("columns ArrayBuffer(col1)"))
+
+      val e2 = intercept[SQLFeatureNotSupportedException] {

Review comment:
       can we test more combinations? e.g., when there is no option given to a column, or no option is given to the index, or an index type is specified etc.

##########
File path: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
##########
@@ -1780,6 +1795,7 @@ WINDOW: 'WINDOW';
 WITH: 'WITH';
 YEAR: 'YEAR';
 ZONE: 'ZONE';
+

Review comment:
       nit: unrelated change

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala
##########
@@ -88,6 +88,17 @@ class ResolveCatalogs(val catalogManager: CatalogManager)
         val namespace = if (ns.nonEmpty) Some(ns) else None
         SetCatalogAndNamespace(catalogManager, Some(catalog.name()), namespace)
       }
+
+    case c @ CreateIndex(UnresolvedDBObjectName(

Review comment:
       why do we need to use `UnresolvedDBObjectName` here?

##########
File path: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
##########
@@ -1034,6 +1045,10 @@ alterColumnAction
     | setOrDrop=(SET | DROP) NOT NULL
     ;
 
+indexPropertyList

Review comment:
       can we combine this with `tablePropertyList`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r731500964



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/index/SupportsIndex.java
##########
@@ -38,7 +37,8 @@
    * Creates an index.
    *
    * @param indexName the name of the index to be created
-   * @param indexType the IndexType of the index to be created
+   * @param indexType the type of the index to be created. If this is not specified, Spark
+   *                  will use empty String.

Review comment:
       I think it's better to follow the table provider and treat index type as a special index property. Then it's easier to indicate when index type is not specified.
   
   We can do it in a followup.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r731499768



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.datasources.v2
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    table: SupportsIndex,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[(NamedReference, Map[String, String])],
+    properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+    try {

Review comment:
       nit: we should only wrap `createIndex` with try-catch




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945208234


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144334/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945407474


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144338/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r730988347



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.datasources.v2
+
+import java.util
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    table: SupportsIndex,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[(NamedReference, Map[String, String])],
+    properties: Map[String, String])
+  extends LeafV2CommandExec {
+  override protected def run(): Seq[InternalRow] = {
+    try {
+      val colProperties: Array[util.Map[NamedReference, util.Map[String, String]]] = columns.map {
+        case (column, map) =>
+          val javaMap = new util.HashMap[String, String]
+          map.foreach { case (key, value) => javaMap.put(key, value) }
+          val propMap = new util.HashMap[NamedReference, util.Map[String, String]]

Review comment:
       Can we use the scala convert to turn scala map to java map?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946002268


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48849/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] dbtsai commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
dbtsai commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-952191958


   LGTM. Merged into master. Thank you!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932984542


   **[Test build #143810 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143810/testReport)** for PR 34148 at commit [`bc71de3`](https://github.com/apache/spark/commit/bc71de3c94645763bf16b765b9274bbd46041f36).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-930744501


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143737/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-930733913


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946620285


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946138704


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945902471


   **[Test build #144375 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144375/testReport)** for PR 34148 at commit [`93600f7`](https://github.com/apache/spark/commit/93600f731ff282bd7b5eef2455653b444dd79e27).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946224269


   **[Test build #144381 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144381/testReport)** for PR 34148 at commit [`5b5880c`](https://github.com/apache/spark/commit/5b5880cd054023def4af4912a7fd482930bc1c81).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-946423025


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48869/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939383364


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48534/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r729830325



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -1056,3 +1056,18 @@ case class UncacheTable(
 
   override def markAsAnalyzed(): LogicalPlan = copy(isAnalyzed = true)
 }
+
+/**
+ * The logical plan of the CREATE INDEX command.
+ */
+case class CreateIndex(
+    child: LogicalPlan,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[NamedReference],

Review comment:
       shall we combine it? `columns: Seq[NamedReference -> Map[String, String]]`

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -1056,3 +1056,18 @@ case class UncacheTable(
 
   override def markAsAnalyzed(): LogicalPlan = copy(isAnalyzed = true)
 }
+
+/**
+ * The logical plan of the CREATE INDEX command.
+ */
+case class CreateIndex(
+    child: LogicalPlan,
+    indexName: String,
+    indexType: String,
+    ignoreIfExists: Boolean,
+    columns: Seq[NamedReference],

Review comment:
       shall we combine it? `columns: Seq[(NamedReference, Map[String, String])]`

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import java.util
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.catalog.index.SupportsIndex
+import org.apache.spark.sql.connector.expressions.NamedReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    table: Table,

Review comment:
       ```suggestion
       table: SupportsIndex,
   ```

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -429,6 +432,17 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
       val table = a.table.asInstanceOf[ResolvedTable]
       AlterTableExec(table.catalog, table.identifier, a.changes) :: Nil
 
+    case CreateIndex(ResolvedTable(_, _, table, _),
+        indexName, indexType, ifNotExists, columns, columnProperties, properties) =>
+      table match {
+        case _: SupportsIndex =>
+          CreateIndexExec(table, indexName, indexType, ifNotExists, columns,
+            columnProperties, properties):: Nil
+        case _ => throw new SQLFeatureNotSupportedException(s"CreateIndex not supported yet." +

Review comment:
       Let's move it to `QueryCompilationErrors`.
   
   The error message should highlight that it's this table that does not support `CREATE INDEX`, not Spark.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -429,6 +432,17 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
       val table = a.table.asInstanceOf[ResolvedTable]
       AlterTableExec(table.catalog, table.identifier, a.changes) :: Nil
 
+    case CreateIndex(ResolvedTable(_, _, table, _),
+        indexName, indexType, ifNotExists, columns, columnProperties, properties) =>
+      table match {
+        case _: SupportsIndex =>
+          CreateIndexExec(table, indexName, indexType, ifNotExists, columns,
+            columnProperties, properties):: Nil
+        case _ => throw new SQLFeatureNotSupportedException(s"CreateIndex not supported yet." +

Review comment:
       Let's move it to `QueryCompilationErrors`.
   
   The error message should highlight that it's this table that does not support `INDEX`, not Spark.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939383980


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48536/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939407727


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939389680


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48534/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-939380749


   Kubernetes integration test unable to build dist.
   
   exiting with code: 1
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48535/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-931637145


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48282/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-931625602


   @dongjoon-hyun @sunchao @viirya 
   This PR is ready for review. The failed test `MariaDBKrbIntegrationSuite` is not relevant.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-937419912






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-942814747


   **[Test build #144220 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144220/testReport)** for PR 34148 at commit [`20fac1c`](https://github.com/apache/spark/commit/20fac1c412948a67b4b94c733d8db7a8775afeac).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r728854973



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##########
@@ -4446,6 +4446,39 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
     CommentOnTable(createUnresolvedTable(ctx.multipartIdentifier, "COMMENT ON TABLE"), comment)
   }
 
+  /**
+   * Create an index, returning a [[CreateIndex]] logical plan.
+   * For example:
+   * {{{
+   * CREATE INDEX index_name ON [TABLE] table_name [USING index_type] (column_index_property_list)
+   *   [OPTIONS indexPropertyList]
+   *   column_index_property_list: column_name [OPTIONS(indexPropertyList)]  [ ,  . . . ]
+   *   indexPropertyList: index_property_name [= index_property_value] [ ,  . . . ]
+   * }}}
+   */
+  override def visitCreateIndex(ctx: CreateIndexContext): LogicalPlan = withOrigin(ctx) {
+    val (indexName, indexType) = if (ctx.identifier.size() == 1) {
+      (ctx.identifier(0).getText, null)
+    } else {
+      (ctx.identifier(0).getText, ctx.identifier(1).getText)
+    }
+
+    val columns = ctx.columns.multipartIdentifierProperty.asScala
+      .map(_.multipartIdentifier.getText).toSeq
+    val columnsProperties = ctx.columns.multipartIdentifierProperty.asScala
+      .map(x => (Option(x.options).map(visitPropertyKeyValues).getOrElse(Map.empty))).toSeq
+    val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty)
+
+    CreateIndex(
+      createUnresolvedTable(ctx.multipartIdentifier(), "CREATE INDEX"),
+      indexName,
+      indexType,
+      ctx.EXISTS != null,
+      columns.map(FieldReference(_).asInstanceOf[FieldReference]),

Review comment:
       ```suggestion
         columns.map(FieldReference(_)),
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-934801362


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48373/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-935087194


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143860/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] viirya commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r720783676



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##########
@@ -2785,11 +2785,11 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
    * Convert a table property list into a key-value map.

Review comment:
       not `table property` anymore.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-931936846


   @huaxingao just a quick question. do you have an implementation of the index support in DSv2? I am asking this because pandas API on Spark already implemented this actually. I would like to investigate the feasibility of migration in the future more just for curiosity .. If you don't have it now, it should be good to have one in test or documentation.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945224593


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48813/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945332112


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48817/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945355938


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144335/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-945902471


   **[Test build #144375 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144375/testReport)** for PR 34148 at commit [`93600f7`](https://github.com/apache/spark/commit/93600f731ff282bd7b5eef2455653b444dd79e27).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] huaxingao commented on a change in pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
huaxingao commented on a change in pull request #34148:
URL: https://github.com/apache/spark/pull/34148#discussion_r719814826



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -1022,3 +1022,15 @@ case class UncacheTable(
 
   override def markAsAnalyzed(): LogicalPlan = copy(isAnalyzed = true)
 }
+
+/**
+ * The logical plan of the CREATE INDEX command.
+ */
+case class CreateIndex(
+    child: LogicalPlan,
+    indexName: String,
+    indexType: String,
+    ifNotExists: Boolean,

Review comment:
       This should be `ignoreIfExists`. I will fix this.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.datasources.v2
+
+import java.sql.SQLFeatureNotSupportedException
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog}
+import org.apache.spark.sql.connector.expressions.FieldReference
+
+/**
+ * Physical plan node for creating an index.
+ */
+case class CreateIndexExec(
+    catalog: TableCatalog,
+    ident: Identifier,
+    indexName: String,
+    indexType: String,
+    ifNotExists: Boolean,

Review comment:
       This should be `ignoreIfExists`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #34148: [SPARK-36895][SQL] Add Create Index syntax support

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34148:
URL: https://github.com/apache/spark/pull/34148#issuecomment-932991226


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48322/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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