You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/10/24 09:14:31 UTC

[GitHub] [hudi] ChenShuai1981 opened a new issue, #7047: [SUPPORT] HoodieFlinkCompactor with NoSuchMethodError: org.apache.hudi.org.apache.avro.specific.SpecificRecordBuilderBase

ChenShuai1981 opened a new issue, #7047:
URL: https://github.com/apache/hudi/issues/7047

   **_Tips before filing an issue_**
   
   - Have you gone through our [FAQs](https://hudi.apache.org/learn/faq/)?
   
   - Join the mailing list to engage in conversations and get faster support at dev-subscribe@hudi.apache.org.
   
   - If you have triaged this as a bug, then file an [issue](https://issues.apache.org/jira/projects/HUDI/issues) directly.
   
   **Describe the problem you faced**
   
   A clear and concise description of the problem.
   
   **To Reproduce**
   
   Steps to reproduce the behavior:
   
   1. git clone apache hudi project and checkout master branch
   2. package with ' mvn -Pscala-2.12 -Pspark3.3 -Pflink1.15 -Pflink-bundle-shade-hive3 clean install -DskipTests'
   3. put packaging/hudi-flink-bundle/target/hudi-flink1.15-bundle-0.13.0-SNAPSHOT.jar to $FLINK_HOME/lib folder
   4. run 'bin/sql-client embedded' command, and execute the following scripts:
   ```
   CREATE CATALOG hudi WITH (
     'type' = 'hudi',
     'catalog.path' = 'hdfs://node1:8020/usr/hive/warehouse',
     'hive.conf.dir' = 'file:///var/lib/container/software/hive/conf',
     'mode'='hms'
   );
   use catalog hudi;
   create database hudidb;
   use hudidb;
   CREATE TABLE hudi.hudidb.orders_hudi(
     uuid INT,
     ts INT,
     num INT,
     PRIMARY KEY(uuid) NOT ENFORCED
   ) WITH (
     'connector' = 'hudi',
     'table.type' = 'MERGE_ON_READ'
   );
   insert into hudi.hudidb.orders_hudi values
     (1, 1, 2),
     (2, 2, 3),
     (3, 3, 4);
   ```
   5. after insert data into MOR hudi table done then execute compaction action to generate parquet files
   ```
   ./bin/flink run -c \
   org.apache.hudi.sink.compact.HoodieFlinkCompactor \
   lib/hudi-flink1.15-bundle-0.13.0-SNAPSHOT.jar \
   --path hdfs://node1:8020/usr/hive/warehouse/hudidb/orders_hudi \
   --schedule
   ```
   
   **Expected behavior**
   
   Should be success with parquet file generated.
   
   **Environment Description**
   
   * Hudi version : 0.13.0-SNAPSHOT
   
   * Spark version : 3.3.2
   
   * Hive version : 3.1.2
   
   * Hadoop version : 3.2.4
   
   * Storage (HDFS/S3/GCS..) : HDFS
   
   * Running on Docker? (yes/no) : no
   
   
   **Additional context**
   
   I found org.apache.avro already shaded in hudi-flink-bundle pom.xml,but why class conflicts happened?
   <relocation>
     <pattern>org.apache.avro.</pattern>
     <shadedPattern>${flink.bundle.shade.prefix}org.apache.avro.</shadedPattern>
   </relocation>
   
   **Stacktrace**
   
   ```[docker@node1 flink]$ ./bin/flink run -c \
   > org.apache.hudi.sink.compact.HoodieFlinkCompactor \
   > lib/hudi-flink-bundle.jar \
   > --path hdfs://node1:8020/usr/hive/warehouse/hudidb/orders_hudi \
   > --schedule
   SLF4J: Class path contains multiple SLF4J bindings.
   SLF4J: Found binding in [jar:file:/var/lib/container/software/flink-1.15.2/lib/log4j-slf4j-impl-2.17.1.jar!/org/slf4j/impl/StaticLoggerBinder.class]
   SLF4J: Found binding in [jar:file:/var/lib/container/software/hadoop-3.2.4/share/hadoop/common/lib/slf4j-reload4j-1.7.35.jar!/org/slf4j/impl/StaticLoggerBinder.class]
   SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation.
   SLF4J: Actual binding is of type [org.apache.logging.slf4j.Log4jLoggerFactory]
   java.lang.NoSuchMethodError: org.apache.hudi.org.apache.avro.specific.SpecificRecordBuilderBase.<init>(Lorg/apache/hudi/org/apache/avro/Schema;Lorg/apache/hudi/org/apache/avro/specific/SpecificData;)V
   	at org.apache.hudi.avro.model.HoodieCompactionOperation$Builder.<init>(HoodieCompactionOperation.java:318)
   	at org.apache.hudi.avro.model.HoodieCompactionOperation$Builder.<init>(HoodieCompactionOperation.java:305)
   	at org.apache.hudi.avro.model.HoodieCompactionOperation.newBuilder(HoodieCompactionOperation.java:272)
   	at org.apache.hudi.common.util.CompactionUtils.buildHoodieCompactionOperation(CompactionUtils.java:106)
   	at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
   	at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1384)
   	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
   	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
   	at java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708)
   	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
   	at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:566)
   	at org.apache.hudi.table.action.compact.plan.generators.BaseHoodieCompactionPlanGenerator.generateCompactionPlan(BaseHoodieCompactionPlanGenerator.java:123)
   	at org.apache.hudi.table.action.compact.ScheduleCompactionActionExecutor.scheduleCompaction(ScheduleCompactionActionExecutor.java:142)
   	at org.apache.hudi.table.action.compact.ScheduleCompactionActionExecutor.execute(ScheduleCompactionActionExecutor.java:111)
   	at org.apache.hudi.table.HoodieFlinkMergeOnReadTable.scheduleCompaction(HoodieFlinkMergeOnReadTable.java:112)
   	at org.apache.hudi.client.BaseHoodieWriteClient.scheduleTableServiceInternal(BaseHoodieWriteClient.java:1436)
   	at org.apache.hudi.client.BaseHoodieWriteClient.scheduleTableService(BaseHoodieWriteClient.java:1413)
   	at org.apache.hudi.client.BaseHoodieWriteClient.scheduleCompactionAtInstant(BaseHoodieWriteClient.java:1023)
   	at org.apache.hudi.sink.compact.HoodieFlinkCompactor$AsyncCompactionService.compact(HoodieFlinkCompactor.java:231)
   	at org.apache.hudi.sink.compact.HoodieFlinkCompactor$AsyncCompactionService.access$000(HoodieFlinkCompactor.java:131)
   	at org.apache.hudi.sink.compact.HoodieFlinkCompactor.start(HoodieFlinkCompactor.java:98)
   	at org.apache.hudi.sink.compact.HoodieFlinkCompactor.main(HoodieFlinkCompactor.java:79)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:355)
   	at org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:222)
   	at org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:114)
   	at org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:836)
   	at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:247)
   	at org.apache.flink.client.cli.CliFrontend.parseAndRun(CliFrontend.java:1078)
   	at org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:1156)
   	at java.security.AccessController.doPrivileged(Native Method)
   	at javax.security.auth.Subject.doAs(Subject.java:422)
   	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1762)
   	at org.apache.flink.runtime.security.contexts.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
   	at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1156)
   Exception in thread "Thread-4" java.lang.IllegalStateException: Trying to access closed classloader. Please check if you store classloaders directly or indirectly in static fields. If the stacktrace suggests that the leak occurs in a third party library and cannot be fixed immediately, you can disable this check with the configuration 'classloader.check-leaked-classloader'.
   	at org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.ensureInner(FlinkUserCodeClassLoaders.java:164)
   	at org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.getResource(FlinkUserCodeClassLoaders.java:183)
   	at org.apache.hadoop.conf.Configuration.getResource(Configuration.java:2778)
   	at org.apache.hadoop.conf.Configuration.getStreamReader(Configuration.java:3052)
   	at org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:3011)
   	at org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:2984)
   	at org.apache.hadoop.conf.Configuration.loadProps(Configuration.java:2862)
   	at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2844)
   	at org.apache.hadoop.conf.Configuration.get(Configuration.java:1200)
   	at org.apache.hadoop.conf.Configuration.getTimeDuration(Configuration.java:1812)
   	at org.apache.hadoop.conf.Configuration.getTimeDuration(Configuration.java:1789)
   	at org.apache.hadoop.util.ShutdownHookManager.getShutdownTimeout(ShutdownHookManager.java:183)
   	at org.apache.hadoop.util.ShutdownHookManager.shutdownExecutor(ShutdownHookManager.java:145)
   	at org.apache.hadoop.util.ShutdownHookManager.access$300(ShutdownHookManager.java:65)
   	at org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:102)```
   
   


-- 
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: commits-unsubscribe@hudi.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on issue #7047: [SUPPORT] HoodieFlinkCompactor with NoSuchMethodError: org.apache.hudi.org.apache.avro.specific.SpecificRecordBuilderBase

Posted by GitBox <gi...@apache.org>.
danny0405 commented on issue #7047:
URL: https://github.com/apache/hudi/issues/7047#issuecomment-1306529412

   I guess it is caused by the avro version in-consistent in Hudi, for h`udi-flink-bundle`, we explicitly declare the avro version as 1.10.0, but in root pom, it is `1.8.2`.


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] xushiyan commented on issue #7047: [SUPPORT] HoodieFlinkCompactor with NoSuchMethodError: org.apache.hudi.org.apache.avro.specific.SpecificRecordBuilderBase

Posted by GitBox <gi...@apache.org>.
xushiyan commented on issue #7047:
URL: https://github.com/apache/hudi/issues/7047#issuecomment-1322345073

   same caused as described in https://github.com/apache/hudi/issues/7259#issuecomment-1322344107


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] caokz commented on issue #7047: [SUPPORT] HoodieFlinkCompactor with NoSuchMethodError: org.apache.hudi.org.apache.avro.specific.SpecificRecordBuilderBase

Posted by "caokz (via GitHub)" <gi...@apache.org>.
caokz commented on issue #7047:
URL: https://github.com/apache/hudi/issues/7047#issuecomment-1463591667

   I also encountered this problem and found that the reason is that the avro version used to package the hudi-common module and the avro version used to package the hudi-flink-bundle module are not the same, they use avro version 1.11.1 (if using spark 3.3 for packaging) and 1.10.0 respectively, only in avro version 1.11.1 There is a corresponding SpecificRecordBuilderBase constructor. This issue is still not fixed in hudi-0.13, I'm using flink 1.16 and the workaround is to change "flink.avro.version" in the root pom to 1.11.1.


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] xushiyan closed issue #7047: [SUPPORT] HoodieFlinkCompactor with NoSuchMethodError: org.apache.hudi.org.apache.avro.specific.SpecificRecordBuilderBase

Posted by GitBox <gi...@apache.org>.
xushiyan closed issue #7047: [SUPPORT] HoodieFlinkCompactor with NoSuchMethodError: org.apache.hudi.org.apache.avro.specific.SpecificRecordBuilderBase
URL: https://github.com/apache/hudi/issues/7047


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org