codope commented on code in PR #11951:
URL: https://github.com/apache/hudi/pull/11951#discussion_r1769485575
##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/IndexCommands.scala:
##########
@@ -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)
Review Comment:
This is legacy code due to incomplete RFC-52. `SecondaryIndexManager` was
introduced in [RFC-52](https://github.com/apache/hudi/pull/5370) but it's just
a wrapper code and does not really manage any index underneath. From RFC, it's
supposed to support index built on third party libraries such as Lucene. But,
we have not yet added any support so far. In my opinion, we should remove all
that code. Just keeping it here to make some tests pass. If you agree, I can
take the cleanup as a followup later.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]