[
https://issues.apache.org/jira/browse/HUDI-7379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17817302#comment-17817302
]
Prabhu Joseph commented on HUDI-7379:
-------------------------------------
Thanks [~danny0405]
> hudi-aws-bundle included jackson-module-afterburner without relocating
> jackson-databind
> ---------------------------------------------------------------------------------------
>
> Key: HUDI-7379
> URL: https://issues.apache.org/jira/browse/HUDI-7379
> Project: Apache Hudi
> Issue Type: Bug
> Components: flink
> Affects Versions: 0.14.1
> Reporter: Prabhu Joseph
> Priority: Major
> Labels: pull-request-available
> Fix For: 1.0.0, 0.14.2
>
>
> Hudi Flink (1.18.1) Write fails when using hudi-aws-bundle on Hudi 0.14.1. It
> works fine on Hudi 0.14.0.
>
> *Error*
> {code:java}
> Caused by: java.lang.VerifyError: Bad type on operand stack
> Exception Details:
> Location:
> org/apache/hudi/timeline/service/RequestHandler.<clinit>()V @14:
> invokevirtual
> Reason:
> Type
> 'org/apache/hudi/com/fasterxml/jackson/module/afterburner/AfterburnerModule'
> (current frame, stack[1]) is not assignable to
> 'org/apache/hudi/com/fasterxml/jackson/databind/Module'
> Current Frame:
> bci: @14
> flags: { }
> locals: { }
> stack: { 'org/apache/hudi/com/fasterxml/jackson/databind/ObjectMapper',
> 'org/apache/hudi/com/fasterxml/jackson/module/afterburner/AfterburnerModule' }
> Bytecode:
> 0x0000000: bb00 8c59 b704 9dbb 049f 59b7 04a0 b604
> 0x0000010: a4b3 0157 1202 b804 aab3 010d b1
> at
> org.apache.hudi.timeline.service.TimelineService.startService(TimelineService.java:358)
> ~[hudi-flink1.18-bundle-0.14.1-amzn-0-SNAPSHOT.jar:0.14.1-amzn-0-SNAPSHOT]
> at
> org.apache.hudi.client.embedded.EmbeddedTimelineService.startServer(EmbeddedTimelineService.java:180)
> ~[hudi-flink1.18-bundle-0.14.1-amzn-0-SNAPSHOT.jar:0.14.1-amzn-0-SNAPSHOT]
> at
> org.apache.hudi.client.embedded.EmbeddedTimelineService.createAndStartService(EmbeddedTimelineService.java:121)
> ~[hudi-flink1.18-bundle-0.14.1-amzn-0-SNAPSHOT.jar:0.14.1-amzn-0-SNAPSHOT]
> at
> org.apache.hudi.client.embedded.EmbeddedTimelineService.getOrStartEmbeddedTimelineService(EmbeddedTimelineService.java:107)
> ~[hudi-flink1.18-bundle-0.14.1-amzn-0-SNAPSHOT.jar:0.14.1-amzn-0-SNAPSHOT]
> at
> org.apache.hudi.client.embedded.EmbeddedTimelineService.getOrStartEmbeddedTimelineService(EmbeddedTimelineService.java:92)
> ~[hudi-flink1.18-bundle-0.14.1-amzn-0-SNAPSHOT.jar:0.14.1-amzn-0-SNAPSHOT]
> at
> org.apache.hudi.client.embedded.EmbeddedTimelineServerHelper.createEmbeddedTimelineService(EmbeddedTimelineServerHelper.java:44)
> ~[hudi-flink1.18-bundle-0.14.1-amzn-0-SNAPSHOT.jar:0.14.1-amzn-0-SNAPSHOT]
> at
> org.apache.hudi.client.BaseHoodieClient.startEmbeddedServerView(BaseHoodieClient.java:133)
> ~[hudi-flink1.18-bundle-0.14.1-amzn-0-SNAPSHOT.jar:0.14.1-amzn-0-SNAPSHOT]
> at
> org.apache.hudi.client.BaseHoodieClient.<init>(BaseHoodieClient.java:98)
> ~[hudi-flink1.18-bundle-0.14.1-amzn-0-SNAPSHOT.jar:0.14.1-amzn-0-SNAPSHOT]
> at
> org.apache.hudi.client.BaseHoodieWriteClient.<init>(BaseHoodieWriteClient.java:164)
> ~[hudi-flink1.18-bundle-0.14.1-amzn-0-SNAPSHOT.jar:0.14.1-amzn-0-SNAPSHOT]
> at
> org.apache.hudi.client.BaseHoodieWriteClient.<init>(BaseHoodieWriteClient.java:149)
> ~[hudi-flink1.18-bundle-0.14.1-amzn-0-SNAPSHOT.jar:0.14.1-amzn-0-SNAPSHOT]
> at
> org.apache.hudi.client.HoodieFlinkWriteClient.<init>(HoodieFlinkWriteClient.java:88)
> ~[hudi-flink1.18-bundle-0.14.1-amzn-0-SNAPSHOT.jar:0.14.1-amzn-0-SNAPSHOT]
> at
> org.apache.hudi.util.FlinkWriteClients.createWriteClient(FlinkWriteClients.java:71)
> ~[hudi-flink1.18-bundle-0.14.1-amzn-0-SNAPSHOT.jar:0.14.1-amzn-0-SNAPSHOT]
> at
> org.apache.hudi.sink.StreamWriteOperatorCoordinator.start(StreamWriteOperatorCoordinator.java:191)
> ~[hudi-flink1.18-bundle-0.14.1-amzn-0-SNAPSHOT.jar:0.14.1-amzn-0-SNAPSHOT]
> at
> org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder.start(OperatorCoordinatorHolder.java:185)
> ~[flink-dist-1.18.1-amzn-0-SNAPSHOT.jar:1.18.1-amzn-0-SNAPSHOT]
> at
> org.apache.flink.runtime.scheduler.DefaultOperatorCoordinatorHandler.startOperatorCoordinators(DefaultOperatorCoordinatorHandler.java:165)
> ~[flink-dist-1.18.1-amzn-0-SNAPSHOT.jar:1.18.1-amzn-0-SNAPSHOT]
> at
> org.apache.flink.runtime.scheduler.DefaultOperatorCoordinatorHandler.startAllOperatorCoordinators(DefaultOperatorCoordinatorHandler.java:82)
> ~[flink-dist-1.18.1-amzn-0-SNAPSHOT.jar:1.18.1-amzn-0-SNAPSHOT]
> at
> org.apache.flink.runtime.scheduler.SchedulerBase.startScheduling(SchedulerBase.java:627)
> ~[flink-dist-1.18.1-amzn-0-SNAPSHOT.jar:1.18.1-amzn-0-SNAPSHOT]
> at
> org.apache.flink.runtime.jobmaster.JobMaster.startScheduling(JobMaster.java:1160)
> ~[flink-dist-1.18.1-amzn-0-SNAPSHOT.jar:1.18.1-amzn-0-SNAPSHOT]
> at
> org.apache.flink.runtime.jobmaster.JobMaster.startJobExecution(JobMaster.java:1077)
> ~[flink-dist-1.18.1-amzn-0-SNAPSHOT.jar:1.18.1-amzn-0-SNAPSHOT]
> at
> org.apache.flink.runtime.jobmaster.JobMaster.onStart(JobMaster.java:447)
> ~[flink-dist-1.18.1-amzn-0-SNAPSHOT.jar:1.18.1-amzn-0-SNAPSHOT]
> at
> org.apache.flink.runtime.rpc.RpcEndpoint.internalCallOnStart(RpcEndpoint.java:198)
> ~[flink-dist-1.18.1-amzn-0-SNAPSHOT.jar:1.18.1-amzn-0-SNAPSHOT]
> at
> org.apache.flink.runtime.rpc.pekko.PekkoRpcActor$StoppedState.lambda$start$0(PekkoRpcActor.java:618)
>
> ~[flink-rpc-akka38d36ae1-fa70-4470-87ff-1fdfc77df55f.jar:1.18.1-amzn-0-SNAPSHOT]
> at
> org.apache.flink.runtime.concurrent.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:68)
> ~[flink-dist-1.18.1-amzn-0-SNAPSHOT.jar:1.18.1-amzn-0-SNAPSHOT]
> at
> org.apache.flink.runtime.rpc.pekko.PekkoRpcActor$StoppedState.start(PekkoRpcActor.java:617)
>
> ~[flink-rpc-akka38d36ae1-fa70-4470-87ff-1fdfc77df55f.jar:1.18.1-amzn-0-SNAPSHOT]
> ... 20 more
> {code}
>
> *Repro*
> {code:java}
> 1. Place hudi-aws-bundle, hudi-flink-bundle and
> aws-glue-datacatalog-hive3-client-3.jar into $FLINK_HOME/lib.
> 2. Enable Glue Catalog Factory in /etc/hive/conf/hive-site.xml
> <property>
> <name>hive.metastore.client.factory.class</name>
>
> <value>com.amazonaws.glue.catalog.metastore.AWSGlueDataCatalogHiveClientFactory</value>
> </property>
> 3. Launch FLink YARN Session
> flink-yarn-session -d
> 4. Run Flink Queries from /usr/lib/flink/bin/sql-client.sh
> CREATE CATALOG IT_hudi_glue_catalog WITH (
> 'type' = 'hudi',
> 'mode' = 'hms',
> 'table.external' = 'true',
> 'default-database' = 'default',
> 'hive.conf.dir' = '/etc/hive/conf',
> 'catalog.path' = 's3a://prabhuflinks3/FLINK_HUDI_IT1/warehouse1/'
> );
> USE CATALOG IT_hudi_glue_catalog;
> CREATE DATABASE IF NOT EXISTS IT_hudi_db;
> use IT_hudi_db;
> CREATE TABLE IF NOT EXISTS hudi_flink_sql_cow_eks(
> uuid VARCHAR(20),
> name VARCHAR(10),
> age INT,
> ts TIMESTAMP(3),
> `partition` VARCHAR(20)
> )
> PARTITIONED BY (`partition`)
> WITH (
> 'connector' = 'hudi',
> 'path' = 's3://prabhuflinks3/hudi_flink_sql_cow_eks',
> 'hive_sync.enable' = 'true',
> 'hive_sync.mode' = 'glue',
> 'hive_sync.table' = 'hudi_flink_sql_cow_eks',
> 'hive_sync.db' = 'default',
> 'compaction.delta_commits' = '1',
> 'hive_sync.partition_fields' = 'partition',
> 'hive_sync.partition_extractor_class' =
> 'org.apache.hudi.hive.MultiPartKeysValueExtractor',
> 'table.type' = 'COPY_ON_WRITE'
> );
> INSERT INTO hudi_flink_sql_cow_eks VALUES
> ('id1','Alex',23,TIMESTAMP '1970-01-01 00:00:01','par1'),
> ('id2','Stephen',33,TIMESTAMP '1970-01-01 00:00:02','par1'),
> ('id3','Julian',53,TIMESTAMP '1970-01-01 00:00:03','par2'),
> ('id4','Fabian',31,TIMESTAMP '1970-01-01 00:00:04','par2'),
> ('id5','Sophia',18,TIMESTAMP '1970-01-01 00:00:05','par3'),
> ('id6','Emma',20,TIMESTAMP '1970-01-01 00:00:06','par3'),
> ('id7','Bob',44,TIMESTAMP '1970-01-01 00:00:07','par4'),
> ('id8','Han',56,TIMESTAMP '1970-01-01 00:00:08','par4');
> {code}
>
> *Analysis*
> hudi-aws-bundle.jar has included relocated
> com.fasterxml.jackson.module:jackson-module-afterburner without relocating
> com.fasterxml.jackson.core:jackson-databind.
> AfterburnerModule class part of hudi-aws-bundle.jar has not relocated the
> Module class from jackson-databind.
> {code:java}
> package org.apache.hudi.com.fasterxml.jackson.module.afterburner;
> import com.fasterxml.jackson.core.Version;
> import com.fasterxml.jackson.databind.Module;
> import com.fasterxml.jackson.databind.Module.SetupContext;
> import java.io.Serializable;
> import
> org.apache.hudi.com.fasterxml.jackson.module.afterburner.deser.DeserializerModifier;
> import
> org.apache.hudi.com.fasterxml.jackson.module.afterburner.ser.SerializerModifier;
> public class AfterburnerModule extends Module implements Serializable {
> {code}
> whereas AfterburnerModule class part of hudi-flink-bundle has relocated the
> Module class from jackson-databind
> {code:java}
> package org.apache.hudi.com.fasterxml.jackson.module.afterburner;
> import java.io.Serializable;
> import org.apache.hudi.com.fasterxml.jackson.core.Version;
> import org.apache.hudi.com.fasterxml.jackson.databind.Module;
> import org.apache.hudi.com.fasterxml.jackson.databind.Module.SetupContext;
> import
> org.apache.hudi.com.fasterxml.jackson.module.afterburner.deser.DeserializerModifier;
> import
> org.apache.hudi.com.fasterxml.jackson.module.afterburner.ser.SerializerModifier;
> public class AfterburnerModule extends Module implements Serializable {
> {code}
> The failure happens as RequestHandler class passes AfterburnerModule (extends
> com.fasterxml.jackson.databind.Module) from hudi-aws-bundle to the
> ObjectMapper class from hudi-flink-bundle which is expecting the class
> org.apache.hudi.com.fasterxml.jackson.databind.Module.
> {code:java}
> private static final ObjectMapper OBJECT_MAPPER = new
> ObjectMapper().registerModule(new AfterburnerModule());
> {code}
>
> Either hudi-aws-bundle should exclude classes from jackson-module-afterburner
> or relocate the jackson-databind
--
This message was sent by Atlassian Jira
(v8.20.10#820010)