Prabhu Joseph created HUDI-7005: ----------------------------------- Summary: Flink SQL Queries on Hudi Table fail when using the hudi-aws-bundle jar Key: HUDI-7005 URL: https://issues.apache.org/jira/browse/HUDI-7005 Project: Apache Hudi Issue Type: Bug Components: flink-sql Affects Versions: 0.14.0 Reporter: Prabhu Joseph
Flink SQL Queries on Hudi Table fail when using the hudi-aws-bundle jar. hudi-aws-bundle jar is needed for metastore sync into AWS Glue. *Error stack trace from Flink SQL Client Log* {code} Caused by: java.lang.NoSuchMethodError: org.apache.hudi.common.table.TableSchemaResolver.getTableAvroSchema(Z)Lorg/apache/hudi/org/apache/avro/Schema; at org.apache.hudi.util.StreamerUtil.getTableAvroSchema(StreamerUtil.java:431) at org.apache.hudi.util.StreamerUtil.getLatestTableSchema(StreamerUtil.java:441) at org.apache.hudi.table.catalog.HoodieHiveCatalog.getTable(HoodieHiveCatalog.java:420) at org.apache.flink.table.catalog.CatalogManager.getUnresolvedTable(CatalogManager.java:685) at org.apache.flink.table.catalog.CatalogManager.dropTableInternal(CatalogManager.java:1236) at org.apache.flink.table.catalog.CatalogManager.dropTable(CatalogManager.java:1207) at org.apache.flink.table.operations.ddl.DropTableOperation.execute(DropTableOperation.java:72) at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:1092) at org.apache.flink.table.gateway.service.operation.OperationExecutor.callOperation(OperationExecutor.java:556) at org.apache.flink.table.gateway.service.operation.OperationExecutor.executeOperation(OperationExecutor.java:444) at org.apache.flink.table.gateway.service.operation.OperationExecutor.executeStatement(OperationExecutor.java:207) at org.apache.flink.table.gateway.service.SqlGatewayServiceImpl.lambda$executeStatement$1(SqlGatewayServiceImpl.java:212) at org.apache.flink.table.gateway.service.operation.OperationManager.lambda$submitOperation$1(OperationManager.java:119) at org.apache.flink.table.gateway.service.operation.OperationManager$Operation.lambda$run$0(OperationManager.java:258) ... 7 more {code} *Repro* {code} cd /usr/lib/flink/lib wget https://repo1.maven.org/maven2/org/apache/hudi/hudi-flink1.17-bundle/0.14.0/hudi-flink1.17-bundle-0.14.0.jar wget https://repo1.maven.org/maven2/org/apache/hudi/hudi-aws-bundle/0.14.0/hudi-aws-bundle-0.14.0.jar flink-yarn-session -d /usr/lib/flink/bin/sql-client.sh embedded CREATE CATALOG glue_catalog_for_hudi WITH ( 'type' = 'hudi', 'mode' = 'hms', 'table.external' = 'true', 'default-database' = 'default', 'hive.conf.dir' = '/etc/hive/conf.dist', 'catalog.path' = 's3://prabhuflinks3/HUDICDC/warehouse/' ); USE CATALOG glue_catalog_for_hudi; CREATE DATABASE IF NOT EXISTS flink_glue_hudi_db; use flink_glue_hudi_db; CREATE TABLE `glue_catalog_for_hudi`.`flink_glue_hudi_db`.`Persons_src` ( ID INT NOT NULL, FirstName STRING, Age STRING, PRIMARY KEY (`ID`) NOT ENFORCED ) WITH ( 'connector' = 'hudi', 'write.tasks' = '2', 'path' = 's3://prabhuflinks3/HUDICDC/warehouse/Persons_src', 'table.type' = 'COPY_ON_WRITE', 'read.streaming.enabled' = 'true', 'read.streaming.check-interval' = '1', 'hoodie.embed.timeline.server' = 'false', 'hive_sync.mode' = 'glue' ); {code} *Root Cause* hudi-flink-bundle has relocated the org.apache.avro. into org.apache.hudi.org.apache.avro. while shading the hudi-common classes (TableSchemaResolver), whereas hudi-aws-bundle has not relocated it while shading hudi-common classes. There are two TableSchemaResolver classes found in the classpath: one from hudi-flink-bundle, which has relocated org.apache.avro, and another from hudi-aws-bundle, which has not relocated org.apache.avro. This causes NoSuchMethodError issues. *Fix* The fix is to relocate org.apache.avro. into org.apache.hudi.org.apache.avro. in hudi-aws-bundle. This change has fixed the above issue. -- This message was sent by Atlassian Jira (v8.20.10#820010)