-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-7563] Add support to drop index using sql #11951
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -32,7 +32,7 @@ | |
| import org.apache.hudi.exception.HoodieException; | ||
| import org.apache.hudi.exception.HoodieFunctionalIndexException; | ||
| import org.apache.hudi.metadata.MetadataPartitionType; | ||
| import org.apache.hudi.table.action.index.functional.BaseHoodieFunctionalIndexClient; | ||
| import org.apache.hudi.table.action.index.functional.BaseHoodieIndexClient; | ||
|
|
||
| import org.apache.spark.sql.SparkSession; | ||
| import org.slf4j.Logger; | ||
|
|
@@ -56,24 +56,24 @@ | |
| import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_SECONDARY_INDEX; | ||
| import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_SECONDARY_INDEX_PREFIX; | ||
|
|
||
| public class HoodieSparkFunctionalIndexClient extends BaseHoodieFunctionalIndexClient { | ||
| public class HoodieSparkIndexClient extends BaseHoodieIndexClient { | ||
|
|
||
| private static final Logger LOG = LoggerFactory.getLogger(HoodieSparkFunctionalIndexClient.class); | ||
| private static final Logger LOG = LoggerFactory.getLogger(HoodieSparkIndexClient.class); | ||
|
|
||
| private static volatile HoodieSparkFunctionalIndexClient _instance; | ||
| private static volatile HoodieSparkIndexClient _instance; | ||
|
|
||
| private final SparkSession sparkSession; | ||
|
|
||
| private HoodieSparkFunctionalIndexClient(SparkSession sparkSession) { | ||
| private HoodieSparkIndexClient(SparkSession sparkSession) { | ||
| super(); | ||
| this.sparkSession = sparkSession; | ||
| } | ||
|
|
||
| public static HoodieSparkFunctionalIndexClient getInstance(SparkSession sparkSession) { | ||
| public static HoodieSparkIndexClient getInstance(SparkSession sparkSession) { | ||
| if (_instance == null) { | ||
| synchronized (HoodieSparkFunctionalIndexClient.class) { | ||
| synchronized (HoodieSparkIndexClient.class) { | ||
| if (_instance == null) { | ||
| _instance = new HoodieSparkFunctionalIndexClient(sparkSession); | ||
| _instance = new HoodieSparkIndexClient(sparkSession); | ||
| } | ||
| } | ||
| } | ||
|
|
@@ -112,6 +112,24 @@ public void create(HoodieTableMetaClient metaClient, String indexName, String in | |
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void drop(HoodieTableMetaClient metaClient, String indexName, boolean ignoreIfNotExists) { | ||
| if (!indexExists(metaClient, indexName)) { | ||
| if (ignoreIfNotExists) { | ||
| return; | ||
| } else { | ||
| throw new HoodieFunctionalIndexException("Index does not exist: " + indexName); | ||
| } | ||
| } | ||
|
|
||
| LOG.info("Dropping index {}", indexName); | ||
| HoodieIndexDefinition indexDefinition = metaClient.getIndexMetadata().get().getIndexDefinitions().get(indexName); | ||
| try (SparkRDDWriteClient writeClient = HoodieCLIUtils.createHoodieWriteClient( | ||
| sparkSession, metaClient.getBasePath().toString(), mapAsScalaImmutableMap(buildWriteConfig(metaClient, indexDefinition)), toScalaOption(Option.empty()))) { | ||
| writeClient.dropIndex(Collections.singletonList(indexName)); | ||
|
Comment on lines
+117
to
+129
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For future reference, this logic does not have much specifics to engine itself, so it can be abstracted to the index client by plugging in the engine-specific write client.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Don't we need the engine-specific write client to call the base API
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There is |
||
| } | ||
| } | ||
|
|
||
| private static Option<String> doSchedule(SparkRDDWriteClient<HoodieRecordPayload> client, HoodieTableMetaClient metaClient, String indexName) { | ||
| List<MetadataPartitionType> partitionTypes = Collections.singletonList(MetadataPartitionType.FUNCTIONAL_INDEX); | ||
| checkArgument(partitionTypes.size() == 1, "Currently, only one index type can be scheduled at a time."); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,12 +20,12 @@ | |
| package org.apache.spark.sql.hudi.command | ||
|
|
||
| import org.apache.hudi.HoodieConversionUtils.toScalaOption | ||
| import org.apache.hudi.HoodieSparkFunctionalIndexClient | ||
| import org.apache.hudi.HoodieSparkIndexClient | ||
| import org.apache.hudi.common.table.HoodieTableMetaClient | ||
| import org.apache.hudi.common.util.JsonUtils | ||
| import org.apache.hudi.hadoop.fs.HadoopFSUtils | ||
| import org.apache.hudi.index.secondary.SecondaryIndexManager | ||
|
|
||
| import org.apache.hudi.metadata.MetadataPartitionType | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.sql.catalyst.catalog.CatalogTable | ||
| import org.apache.spark.sql.catalyst.expressions.Attribute | ||
|
|
@@ -52,7 +52,7 @@ case class CreateIndexCommand(table: CatalogTable, | |
| columns.map(c => columnsMap.put(c._1.mkString("."), c._2.asJava)) | ||
|
|
||
| if (options.contains("func") || indexType.equals("secondary_index")) { | ||
| HoodieSparkFunctionalIndexClient.getInstance(sparkSession).create( | ||
| HoodieSparkIndexClient.getInstance(sparkSession).create( | ||
| metaClient, indexName, indexType, columnsMap, options.asJava) | ||
| } else { | ||
| SecondaryIndexManager.getInstance().create( | ||
|
|
@@ -76,7 +76,14 @@ case class DropIndexCommand(table: CatalogTable, | |
| override def run(sparkSession: SparkSession): Seq[Row] = { | ||
| val tableId = table.identifier | ||
| val metaClient = createHoodieTableMetaClient(tableId, sparkSession) | ||
| SecondaryIndexManager.getInstance().drop(metaClient, indexName, ignoreIfNotExists) | ||
| try { | ||
| // need to ensure that the index name is for a valid partition type | ||
| MetadataPartitionType.fromPartitionPath(indexName) | ||
| HoodieSparkIndexClient.getInstance(sparkSession).drop(metaClient, indexName, ignoreIfNotExists) | ||
| } catch { | ||
| case _: IllegalArgumentException => | ||
| SecondaryIndexManager.getInstance().drop(metaClient, indexName, ignoreIfNotExists) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why drop here again?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is legacy code due to incomplete RFC-52.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Got it. Let's have JIRA to track that. |
||
| } | ||
|
|
||
| // Invalidate cached table for queries do not access related table | ||
| // through {@code DefaultSource} | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.