diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/index/SupportsIndex.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/index/SupportsIndex.java new file mode 100644 index 0000000000000..a8d55fb0b9c85 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/index/SupportsIndex.java @@ -0,0 +1,75 @@ +/* + * 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.connector.catalog.index; + +import java.util.Map; +import java.util.Properties; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException; +import org.apache.spark.sql.catalyst.analysis.NoSuchIndexException; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.expressions.NamedReference; + +/** + * Table methods for working with index + * + * @since 3.3.0 + */ +@Evolving +public interface SupportsIndex extends Table { + + /** + * Creates an index. + * + * @param indexName the name of the index to be created + * @param indexType the IndexType of the index to be created + * @param columns the columns on which index to be created + * @param columnProperties the properties of the columns on which index to be created + * @param properties the properties of the index to be created + * @throws IndexAlreadyExistsException If the index already exists (optional) + */ + void createIndex(String indexName, + String indexType, + NamedReference[] columns, + Map[] columnProperties, + Properties properties) + throws IndexAlreadyExistsException; + + /** + * Drops the index with the given name. + * + * @param indexName the name of the index to be dropped. + * @return true if the index is dropped + * @throws NoSuchIndexException If the index does not exist (optional) + */ + boolean dropIndex(String indexName) throws NoSuchIndexException; + + /** + * Checks whether an index exists in this table. + * + * @param indexName the name of the index + * @return true if the index exists, false otherwise + */ + boolean indexExists(String indexName); + + /** + * Lists all the indexes in this table. + */ + TableIndex[] listIndexes(); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/index/TableIndex.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/index/TableIndex.java new file mode 100644 index 0000000000000..99fce806a11b9 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/index/TableIndex.java @@ -0,0 +1,79 @@ +/* + * 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.connector.catalog.index; + +import java.util.Collections; +import java.util.Map; +import java.util.Properties; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.connector.expressions.NamedReference; + +/** + * Index in a table + * + * @since 3.3.0 + */ +@Evolving +public final class TableIndex { + private String indexName; + private String indexType; + private NamedReference[] columns; + private Map columnProperties = Collections.emptyMap(); + private Properties properties; + + public TableIndex( + String indexName, + String indexType, + NamedReference[] columns, + Map columnProperties, + Properties properties) { + this.indexName = indexName; + this.indexType = indexType; + this.columns = columns; + this.columnProperties = columnProperties; + this.properties = properties; + } + + /** + * @return the Index name. + */ + String indexName() { return indexName; } + + /** + * @return the indexType of this Index. + */ + String indexType() { return indexType; } + + /** + * @return the column(s) this Index is on. Could be multi columns (a multi-column index). + */ + NamedReference[] columns() { return columns; } + + /** + * @return the map of column and column property map. + */ + Map columnProperties() { return columnProperties; } + + /** + * Returns the index properties. + */ + Properties properties() { + return properties; + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala index 70f821d5f8af0..ce48cfa89a389 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala @@ -78,3 +78,6 @@ class PartitionsAlreadyExistException(message: String) extends AnalysisException class FunctionAlreadyExistsException(db: String, func: String) extends AnalysisException(s"Function '$func' already exists in database '$db'") + +class IndexAlreadyExistsException(indexName: String, table: Identifier) + extends AnalysisException(s"Index '$indexName' already exists in table ${table.quoted}") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala index ba5a9c618c650..7a9f7b5c6bced 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala @@ -95,3 +95,6 @@ class NoSuchPartitionsException(message: String) extends AnalysisException(messa class NoSuchTempFunctionException(func: String) extends AnalysisException(s"Temporary function '$func' not found") + +class NoSuchIndexException(indexName: String) + extends AnalysisException(s"Index '$indexName' not found")