You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "kkrugler (via GitHub)" <gi...@apache.org> on 2023/03/09 23:09:46 UTC

[GitHub] [hudi] kkrugler opened a new issue, #8147: [SUPPORT] Missing dependency on hive-exec (core)

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

   **Describe the problem you faced**
   
   When using Flink to do an incremental query read from a table, using the 0.12.2 and Flink 1.15, I occasionally get a ClassNotFoundException for `org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat`. This usually happens when running the test from inside Eclipse, occasionally from the command line.
   
   **To Reproduce**
   
   Steps to reproduce the behavior:
   
   1. `git clone https://github.com/kkrugler/flink-hudi-query-test`
   2. `cd flink-hudi-query-test`
   3. `git checkout flink-1.15-hudi-0.12`
   4. `mvn clean package`
   
   **Expected behavior**
   
   The tests should all pass.
   
   **Environment Description**
   
   * Hudi version : 0.12.2
   
   * Flink version : 1.15.1
   
   **Additional context**
   
   I believe the problem is that the `hudi-hadoop-mr` dependency on `hive-exec` (with classifier `core`) is marked as provided, but when running a Flink workflow in a typical Flink cluster you don't have Hive jars installed. I think maybe it's OK for `hudi-hadoop-mr` to say this is provided, but `hudi-flink` should then have an explicit dependency on this artifact, something like:
   
   ```
           <dependency>
               <groupId>org.apache.hive</groupId>
               <artifactId>hive-exec</artifactId>
               <classifier>core</classifier>
               <version>${hive.version}</version>
               <exclusions>
                   <exclusion>
                       <groupId>*</groupId>
                       <artifactId>*</artifactId>
                   </exclusion>
               </exclusions>
           </dependency>
   ```
   
   Note the exclusion of all transitive dependencies. All that Hudi needs from `hive-exec` is the one missing class, as Hudi uses HoodieParquetInputFormatBase as the base class, as per:
   
   ``` java
   /**
    * !!! PLEASE READ CAREFULLY !!!
    *
    * NOTE: Hive bears optimizations which are based upon validating whether {@link FileInputFormat}
    * implementation inherits from {@link MapredParquetInputFormat}.
    *
    * To make sure that Hudi implementations are leveraging these optimizations to the fullest, this class
    * serves as a base-class for every {@link FileInputFormat} implementations working with Parquet file-format.
    *
    * However, this class serves as a simple delegate to the actual implementation hierarchy: it expects
    * either {@link HoodieCopyOnWriteTableInputFormat} or {@link HoodieMergeOnReadTableInputFormat} to be supplied
    * to which it delegates all of its necessary methods.
    */
   public abstract class HoodieParquetInputFormatBase extends MapredParquetInputFormat implements Configurable {
   ```
   
   And if you don't do this exclusion, you wind up pulling in lots of additional code that's not needed (AFAICT).
   
   **Stacktrace**
   
   ```
   23/03/09 10:08:22 INFO executiongraph.ExecutionGraph:1423 - Source: split_monitor(table=[example-table], fields=[event_time, data, enrichment, key, partition]) (1/1) (16f707e9f9462ca1ac57f69e5bc9ae4e) switched from RUNNING to FAILED on 9cbbe102-0f19-48d6-849c-4755cab4fa2d @ localhost (dataPort=-1).
   java.lang.NoClassDefFoundError: org/apache/hadoop/hive/ql/io/parquet/MapredParquetInputFormat
   	at java.lang.ClassLoader.defineClass1(Native Method) ~[?:?]
   	at java.lang.ClassLoader.defineClass(ClassLoader.java:1016) ~[?:?]
   	at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:174) ~[?:?]
   	at jdk.internal.loader.BuiltinClassLoader.defineClass(BuiltinClassLoader.java:800) ~[?:?]
   	at jdk.internal.loader.BuiltinClassLoader.findClassOnClassPathOrNull(BuiltinClassLoader.java:698) ~[?:?]
   	at jdk.internal.loader.BuiltinClassLoader.loadClassOrNull(BuiltinClassLoader.java:621) ~[?:?]
   	at jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:579) ~[?:?]
   	at jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178) ~[?:?]
   	at java.lang.ClassLoader.loadClass(ClassLoader.java:521) ~[?:?]
   	at java.lang.ClassLoader.defineClass1(Native Method) ~[?:?]
   	at java.lang.ClassLoader.defineClass(ClassLoader.java:1016) ~[?:?]
   	at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:174) ~[?:?]
   	at jdk.internal.loader.BuiltinClassLoader.defineClass(BuiltinClassLoader.java:800) ~[?:?]
   	at jdk.internal.loader.BuiltinClassLoader.findClassOnClassPathOrNull(BuiltinClassLoader.java:698) ~[?:?]
   	at jdk.internal.loader.BuiltinClassLoader.loadClassOrNull(BuiltinClassLoader.java:621) ~[?:?]
   	at jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:579) ~[?:?]
   	at jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178) ~[?:?]
   	at java.lang.ClassLoader.loadClass(ClassLoader.java:521) ~[?:?]
   	at java.lang.ClassLoader.defineClass1(Native Method) ~[?:?]
   	at java.lang.ClassLoader.defineClass(ClassLoader.java:1016) ~[?:?]
   	at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:174) ~[?:?]
   	at jdk.internal.loader.BuiltinClassLoader.defineClass(BuiltinClassLoader.java:800) ~[?:?]
   	at jdk.internal.loader.BuiltinClassLoader.findClassOnClassPathOrNull(BuiltinClassLoader.java:698) ~[?:?]
   	at jdk.internal.loader.BuiltinClassLoader.loadClassOrNull(BuiltinClassLoader.java:621) ~[?:?]
   	at jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:579) ~[?:?]
   	at jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178) ~[?:?]
   	at java.lang.ClassLoader.loadClass(ClassLoader.java:521) ~[?:?]
   	at org.apache.hudi.sink.partitioner.profile.WriteProfiles.getCommitMetadata(WriteProfiles.java:236) ~[hudi-flink-0.12.2.jar:0.12.2]
   	at org.apache.hudi.source.IncrementalInputSplits.lambda$inputSplits$2(IncrementalInputSplits.java:285) ~[hudi-flink-0.12.2.jar:0.12.2]
   	at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) ~[?:?]
   	at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1655) ~[?:?]
   	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484) ~[?:?]
   	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474) ~[?:?]
   	at java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913) ~[?:?]
   	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) ~[?:?]
   	at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578) ~[?:?]
   	at org.apache.hudi.source.IncrementalInputSplits.inputSplits(IncrementalInputSplits.java:285) ~[hudi-flink-0.12.2.jar:0.12.2]
   	at org.apache.hudi.source.StreamReadMonitoringFunction.monitorDirAndForwardSplits(StreamReadMonitoringFunction.java:199) ~[hudi-flink-0.12.2.jar:0.12.2]
   	at org.apache.hudi.source.StreamReadMonitoringFunction.run(StreamReadMonitoringFunction.java:172) ~[hudi-flink-0.12.2.jar:0.12.2]
   	at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:110) ~[flink-streaming-java-1.15.1.jar:1.15.1]
   	at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:67) ~[flink-streaming-java-1.15.1.jar:1.15.1]
   	at org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:332) ~[flink-streaming-java-1.15.1.jar:1.15.1]
   Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
   	at jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:581) ~[?:?]
   	at jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178) ~[?:?]
   	at java.lang.ClassLoader.loadClass(ClassLoader.java:521) ~[?:?]
   	... 42 more
   ```
   
   


-- 
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 #8147: [SUPPORT] Missing dependency on hive-exec (core)

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

   The error stack trace confused me a log, because `WriteProfiles.getCommitMetadata` does not depend on the `MapredParquetInputFormat` in the code path, why it tries to load it then?


-- 
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] wmcfight commented on issue #8147: [SUPPORT] Missing dependency on hive-exec (core)

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

   > > @gamblewin - if you can, I think the easiest solution is to ad the `hive-exec` jar to your Flink /lib directory.
   > 
   > @kkrugler thx for replying
   > 
   > * things I tired
   >   I tried to add `hive-exec-2.3.1`jar to Flink/lib directory, the reason i use 2.3.1 is because the default hive version in `pom.xml` is 2.3.1, then when I try to insert data into a hudi table, some exception happened.
   >   Then I change `hive-exec-2.3.1` to `hive-exec-3.1.3`, still encountered same mistake when insert data into hudi table.
   > * stacktrace
   >   2023-03-21 13:26:35
   >   java.lang.NoSuchMethodError: org.apache.parquet.schema.Types$PrimitiveBuilder.as(Lorg/apache/parquet/schema/LogicalTypeAnnotation;)Lorg/apache/parquet/schema/Types$Builder;
   >   at org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:177)
   >   at org.apache.parquet.avro.AvroSchemaConverter.convertUnion(AvroSchemaConverter.java:242)
   >   at org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:199)
   >   at org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:152)
   >   at org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:260)
   >   at org.apache.parquet.avro.AvroSchemaConverter.convertFields(AvroSchemaConverter.java:146)
   >   at org.apache.parquet.avro.AvroSchemaConverter.convert(AvroSchemaConverter.java:137)
   >   at org.apache.hudi.io.storage.HoodieAvroFileWriterFactory.newParquetFileWriter(HoodieAvroFileWriterFactory.java:53)
   >   at org.apache.hudi.io.storage.HoodieFileWriterFactory.getFileWriterByFormat(HoodieFileWriterFactory.java:80)
   >   at org.apache.hudi.io.storage.HoodieFileWriterFactory.getFileWriter(HoodieFileWriterFactory.java:67)
   >   at org.apache.hudi.io.HoodieCreateHandle.(HoodieCreateHandle.java:103)
   >   at org.apache.hudi.io.HoodieCreateHandle.(HoodieCreateHandle.java:82)
   >   at org.apache.hudi.io.FlinkCreateHandle.(FlinkCreateHandle.java:66)
   >   at org.apache.hudi.io.FlinkCreateHandle.(FlinkCreateHandle.java:59)
   >   at org.apache.hudi.io.FlinkWriteHandleFactory$BaseCommitWriteHandleFactory.create(FlinkWriteHandleFactory.java:114)
   >   at org.apache.hudi.client.HoodieFlinkWriteClient.getOrCreateWriteHandle(HoodieFlinkWriteClient.java:463)
   >   at org.apache.hudi.client.HoodieFlinkWriteClient.access$000(HoodieFlinkWriteClient.java:75)
   >   at org.apache.hudi.client.HoodieFlinkWriteClient$AutoCloseableWriteHandle.(HoodieFlinkWriteClient.java:512)
   >   at org.apache.hudi.client.HoodieFlinkWriteClient.upsert(HoodieFlinkWriteClient.java:146)
   >   at org.apache.hudi.sink.StreamWriteFunction.lambda$initWriteFunction$1(StreamWriteFunction.java:191)
   >   at org.apache.hudi.sink.StreamWriteFunction.lambda$flushRemaining$7(StreamWriteFunction.java:478)
   >   at java.util.LinkedHashMap$LinkedValues.forEach(LinkedHashMap.java:608)
   >   at org.apache.hudi.sink.StreamWriteFunction.flushRemaining(StreamWriteFunction.java:468)
   >   at org.apache.hudi.sink.StreamWriteFunction.endInput(StreamWriteFunction.java:158)
   >   at org.apache.hudi.sink.common.AbstractWriteOperator.endInput(AbstractWriteOperator.java:48)
   >   at org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.endOperatorInput(StreamOperatorWrapper.java:91)
   >   at org.apache.flink.streaming.runtime.tasks.OperatorChain.endInput(OperatorChain.java:428)
   >   at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:69)
   >   at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:423)
   >   at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:204)
   >   at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:684)
   >   at org.apache.flink.streaming.runtime.tasks.StreamTask.executeInvoke(StreamTask.java:639)
   >   at org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:650)
   >   at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:623)
   >   at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:779)
   >   at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566)
   >   at java.lang.Thread.run(Thread.java:748)
   
   I met the same issues,  did you fix the problem finally? @gamblewin 


-- 
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


Re: [I] [SUPPORT] Missing dependency on hive-exec (core) [hudi]

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

   I think the issue you see is due to having multiple versions of Parquet support classes in different jars. Unfortunately the only way to fix it is to carefully review versions of all transitive dependencies of all jars, to figure out which jar(s) are in conflict. I've re-purposed the "JarHell" class from OpenSearch to track down issues like this (e.g. call it from the open method of some UDF in your case, to find run-time class conflicts).


-- 
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] gamblewin commented on issue #8147: [SUPPORT] Missing dependency on hive-exec (core)

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

   > @gamblewin - if you can, I think the easiest solution is to ad the `hive-exec` jar to your Flink /lib directory.
   
   @kkrugler thx for replying
   - things I tired
   I tried to add `hive-exec-2.3.1`jar to Flink/lib directory, the reason i use 2.3.1 is because the default hive version in `pom.xml` is 2.3.1, then when I try to insert data into a hudi table, some exception happened. 
   Then I change `hive-exec-2.3.1` to `hive-exec-3.1.3`, still encountered same mistake when insert data into hudi table.
   
   - stacktrace
   2023-03-21 13:26:35
   java.lang.NoSuchMethodError: org.apache.parquet.schema.Types$PrimitiveBuilder.as(Lorg/apache/parquet/schema/LogicalTypeAnnotation;)Lorg/apache/parquet/schema/Types$Builder;
   	at org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:177)
   	at org.apache.parquet.avro.AvroSchemaConverter.convertUnion(AvroSchemaConverter.java:242)
   	at org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:199)
   	at org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:152)
   	at org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:260)
   	at org.apache.parquet.avro.AvroSchemaConverter.convertFields(AvroSchemaConverter.java:146)
   	at org.apache.parquet.avro.AvroSchemaConverter.convert(AvroSchemaConverter.java:137)
   	at org.apache.hudi.io.storage.HoodieAvroFileWriterFactory.newParquetFileWriter(HoodieAvroFileWriterFactory.java:53)
   	at org.apache.hudi.io.storage.HoodieFileWriterFactory.getFileWriterByFormat(HoodieFileWriterFactory.java:80)
   	at org.apache.hudi.io.storage.HoodieFileWriterFactory.getFileWriter(HoodieFileWriterFactory.java:67)
   	at org.apache.hudi.io.HoodieCreateHandle.<init>(HoodieCreateHandle.java:103)
   	at org.apache.hudi.io.HoodieCreateHandle.<init>(HoodieCreateHandle.java:82)
   	at org.apache.hudi.io.FlinkCreateHandle.<init>(FlinkCreateHandle.java:66)
   	at org.apache.hudi.io.FlinkCreateHandle.<init>(FlinkCreateHandle.java:59)
   	at org.apache.hudi.io.FlinkWriteHandleFactory$BaseCommitWriteHandleFactory.create(FlinkWriteHandleFactory.java:114)
   	at org.apache.hudi.client.HoodieFlinkWriteClient.getOrCreateWriteHandle(HoodieFlinkWriteClient.java:463)
   	at org.apache.hudi.client.HoodieFlinkWriteClient.access$000(HoodieFlinkWriteClient.java:75)
   	at org.apache.hudi.client.HoodieFlinkWriteClient$AutoCloseableWriteHandle.<init>(HoodieFlinkWriteClient.java:512)
   	at org.apache.hudi.client.HoodieFlinkWriteClient.upsert(HoodieFlinkWriteClient.java:146)
   	at org.apache.hudi.sink.StreamWriteFunction.lambda$initWriteFunction$1(StreamWriteFunction.java:191)
   	at org.apache.hudi.sink.StreamWriteFunction.lambda$flushRemaining$7(StreamWriteFunction.java:478)
   	at java.util.LinkedHashMap$LinkedValues.forEach(LinkedHashMap.java:608)
   	at org.apache.hudi.sink.StreamWriteFunction.flushRemaining(StreamWriteFunction.java:468)
   	at org.apache.hudi.sink.StreamWriteFunction.endInput(StreamWriteFunction.java:158)
   	at org.apache.hudi.sink.common.AbstractWriteOperator.endInput(AbstractWriteOperator.java:48)
   	at org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.endOperatorInput(StreamOperatorWrapper.java:91)
   	at org.apache.flink.streaming.runtime.tasks.OperatorChain.endInput(OperatorChain.java:428)
   	at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:69)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:423)
   	at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:204)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:684)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.executeInvoke(StreamTask.java:639)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:650)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:623)
   	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:779)
   	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566)
   	at java.lang.Thread.run(Thread.java:748)


-- 
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] danny0405 commented on issue #8147: [SUPPORT] Missing dependency on hive-exec (core)

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

   I think we can close the issue, this is a known behavior of flink bundle jar packaging.


-- 
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] danny0405 commented on issue #8147: [SUPPORT] Missing dependency on hive-exec (core)

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

   > WriteProfiles.getCommitMetadata
   
   I see, in thie PR https://github.com/apache/hudi/pull/7055, I have moved the utilities method into another class which is located in `hudi-common`, so this should not be a problem anymore.


-- 
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] danny0405 commented on issue #8147: [SUPPORT] Missing dependency on hive-exec (core)

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

   The error indicates some parquet class conflict.


-- 
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] danny0405 commented on issue #8147: [SUPPORT] Missing dependency on hive-exec (core)

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

   On cluster, you should use the bundle jar instead, and yeah, the default bundler jar does not package the hive-exec, which should be fixed: https://issues.apache.org/jira/browse/HUDI-5916
   
   `hudi-flink` pom already includes the `hive-exec` dependency: https://github.com/apache/hudi/blob/2675118d95c7a087cd9222a05cd7376eb0a31aad/hudi-flink-datasource/hudi-flink/pom.xml#L287, but it does not package into the released jar, that is by design, we only introduce the hive jar into the bundle jars.


-- 
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] kkrugler commented on issue #8147: [SUPPORT] Missing dependency on hive-exec (core)

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

   The `hudi-flink-bundle` pom has what seems like a very long list of transitive dependencies (from running `mvn dependency:tree` in the `packaging/flink-hudi-bundle/` directory). I'm wondering why you don't think this would pull in jars that create conflicts with other jars being used in a workflow...
   
   ```
   [INFO] org.apache.hudi:hudi-flink1.16-bundle:jar:0.14.0-SNAPSHOT
   [INFO] +- org.apache.hudi:hudi-common:jar:0.14.0-SNAPSHOT:compile
   [INFO] |  +- org.openjdk.jol:jol-core:jar:0.16:compile
   [INFO] |  +- com.github.ben-manes.caffeine:caffeine:jar:2.9.1:compile
   [INFO] |  |  +- org.checkerframework:checker-qual:jar:3.10.0:compile
   [INFO] |  |  \- com.google.errorprone:error_prone_annotations:jar:2.5.1:compile
   [INFO] |  +- org.apache.httpcomponents:fluent-hc:jar:4.4.1:compile
   [INFO] |  |  \- commons-logging:commons-logging:jar:1.2:compile
   [INFO] |  +- org.apache.httpcomponents:httpclient:jar:4.4.1:compile
   [INFO] |  +- org.apache.hbase:hbase-client:jar:2.4.9:compile
   [INFO] |  |  +- org.apache.hbase.thirdparty:hbase-shaded-protobuf:jar:3.5.1:compile
   [INFO] |  |  +- org.apache.hbase:hbase-common:jar:2.4.9:compile
   [INFO] |  |  |  +- org.apache.hbase:hbase-logging:jar:2.4.9:compile
   [INFO] |  |  |  \- org.apache.hbase.thirdparty:hbase-shaded-gson:jar:3.5.1:compile
   [INFO] |  |  +- org.apache.hbase:hbase-hadoop-compat:jar:2.4.9:compile
   [INFO] |  |  +- org.apache.hbase:hbase-hadoop2-compat:jar:2.4.9:compile
   [INFO] |  |  |  \- javax.activation:javax.activation-api:jar:1.2.0:runtime
   [INFO] |  |  +- org.apache.hbase:hbase-protocol-shaded:jar:2.4.9:compile
   [INFO] |  |  +- org.apache.hbase:hbase-protocol:jar:2.4.9:compile
   [INFO] |  |  +- org.apache.hbase.thirdparty:hbase-shaded-miscellaneous:jar:3.5.1:compile
   [INFO] |  |  +- org.apache.hbase.thirdparty:hbase-shaded-netty:jar:3.5.1:compile
   [INFO] |  |  +- org.apache.htrace:htrace-core4:jar:4.2.0-incubating:compile
   [INFO] |  |  +- org.jruby.jcodings:jcodings:jar:1.0.55:compile
   [INFO] |  |  +- org.jruby.joni:joni:jar:2.1.31:compile
   [INFO] |  |  +- org.apache.commons:commons-crypto:jar:1.0.0:compile
   [INFO] |  |  \- org.apache.hadoop:hadoop-auth:jar:2.10.1:provided
   [INFO] |  |     +- com.nimbusds:nimbus-jose-jwt:jar:7.9:provided
   [INFO] |  |     |  \- com.github.stephenc.jcip:jcip-annotations:jar:1.0-1:provided
   [INFO] |  |     \- org.apache.directory.server:apacheds-kerberos-codec:jar:2.0.0-M15:provided
   [INFO] |  |        +- org.apache.directory.server:apacheds-i18n:jar:2.0.0-M15:provided
   [INFO] |  |        +- org.apache.directory.api:api-asn1-api:jar:1.0.0-M20:provided
   [INFO] |  |        \- org.apache.directory.api:api-util:jar:1.0.0-M20:provided
   [INFO] |  +- org.apache.hbase:hbase-server:jar:2.4.9:compile
   [INFO] |  |  +- org.apache.hbase:hbase-http:jar:2.4.9:compile
   [INFO] |  |  |  +- org.apache.hbase.thirdparty:hbase-shaded-jetty:jar:3.5.1:compile
   [INFO] |  |  |  +- org.apache.hbase.thirdparty:hbase-shaded-jersey:jar:3.5.1:compile
   [INFO] |  |  |  |  +- jakarta.ws.rs:jakarta.ws.rs-api:jar:2.1.6:compile
   [INFO] |  |  |  |  +- jakarta.annotation:jakarta.annotation-api:jar:1.3.5:compile
   [INFO] |  |  |  |  +- jakarta.validation:jakarta.validation-api:jar:2.0.2:compile
   [INFO] |  |  |  |  \- org.glassfish.hk2.external:jakarta.inject:jar:2.6.1:compile
   [INFO] |  |  |  \- javax.ws.rs:javax.ws.rs-api:jar:2.1.1:compile
   [INFO] |  |  +- org.apache.hbase:hbase-procedure:jar:2.4.9:compile
   [INFO] |  |  +- org.apache.hbase:hbase-zookeeper:jar:2.4.9:compile
   [INFO] |  |  +- org.apache.hbase:hbase-replication:jar:2.4.9:compile
   [INFO] |  |  +- org.apache.hbase:hbase-metrics-api:jar:2.4.9:compile
   [INFO] |  |  +- org.apache.hbase:hbase-metrics:jar:2.4.9:compile
   [INFO] |  |  +- org.apache.hbase:hbase-asyncfs:jar:2.4.9:compile
   [INFO] |  |  +- org.glassfish.web:javax.servlet.jsp:jar:2.3.2:compile
   [INFO] |  |  |  \- org.glassfish:javax.el:jar:3.0.1-b12:provided
   [INFO] |  |  +- javax.servlet.jsp:javax.servlet.jsp-api:jar:2.3.1:compile
   [INFO] |  |  +- org.apache.commons:commons-math3:jar:3.6.1:compile
   [INFO] |  |  +- org.apache.hadoop:hadoop-distcp:jar:2.10.0:compile
   [INFO] |  |  \- org.apache.hadoop:hadoop-annotations:jar:2.10.0:compile
   [INFO] |  +- commons-io:commons-io:jar:2.11.0:compile
   [INFO] |  +- org.lz4:lz4-java:jar:1.8.0:compile
   [INFO] |  \- com.lmax:disruptor:jar:3.4.2:compile
   [INFO] +- org.apache.hudi:hudi-client-common:jar:0.14.0-SNAPSHOT:compile
   [INFO] |  +- com.github.davidmoten:hilbert-curve:jar:0.2.2:compile
   [INFO] |  |  \- com.github.davidmoten:guava-mini:jar:0.1.3:compile
   [INFO] |  +- io.dropwizard.metrics:metrics-graphite:jar:4.1.1:compile
   [INFO] |  +- io.dropwizard.metrics:metrics-core:jar:4.1.1:compile
   [INFO] |  +- io.dropwizard.metrics:metrics-jmx:jar:4.1.1:compile
   [INFO] |  +- io.prometheus:simpleclient:jar:0.8.0:compile
   [INFO] |  +- io.prometheus:simpleclient_httpserver:jar:0.8.0:compile
   [INFO] |  |  \- io.prometheus:simpleclient_common:jar:0.8.0:compile
   [INFO] |  +- io.prometheus:simpleclient_dropwizard:jar:0.8.0:compile
   [INFO] |  +- io.prometheus:simpleclient_pushgateway:jar:0.8.0:compile
   [INFO] |  +- org.apache.curator:curator-framework:jar:2.7.1:compile
   [INFO] |  +- org.apache.curator:curator-client:jar:2.7.1:compile
   [INFO] |  \- org.apache.curator:curator-recipes:jar:2.7.1:compile
   [INFO] +- org.apache.hudi:hudi-aws:jar:0.14.0-SNAPSHOT:compile
   [INFO] |  +- com.amazonaws:dynamodb-lock-client:jar:1.1.0:compile
   [INFO] |  +- com.amazonaws:aws-java-sdk-cloudwatch:jar:1.12.22:compile
   [INFO] |  |  \- com.amazonaws:jmespath-java:jar:1.12.22:compile
   [INFO] |  +- com.amazonaws:aws-java-sdk-dynamodb:jar:1.12.22:compile
   [INFO] |  |  \- com.amazonaws:aws-java-sdk-s3:jar:1.12.22:compile
   [INFO] |  |     \- com.amazonaws:aws-java-sdk-kms:jar:1.12.22:compile
   [INFO] |  +- com.amazonaws:aws-java-sdk-core:jar:1.12.22:compile
   [INFO] |  |  +- software.amazon.ion:ion-java:jar:1.0.2:compile
   [INFO] |  |  \- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:jar:2.12.3:compile
   [INFO] |  +- com.amazonaws:aws-java-sdk-glue:jar:1.12.22:compile
   [INFO] |  \- com.amazonaws:aws-java-sdk-sqs:jar:1.12.22:compile
   [INFO] +- org.apache.hudi:hudi-flink-client:jar:0.14.0-SNAPSHOT:compile
   [INFO] +- org.apache.hudi:hudi-flink1.16.x:jar:0.14.0-SNAPSHOT:compile
   [INFO] +- org.apache.hudi:hudi-flink:jar:0.14.0-SNAPSHOT:compile
   [INFO] |  +- org.apache.hudi:hudi-sync-common:jar:0.14.0-SNAPSHOT:compile
   [INFO] |  +- org.apache.flink:flink-streaming-java:jar:1.16.0:provided
   [INFO] |  |  +- org.apache.flink:flink-core:jar:1.16.0:provided
   [INFO] |  |  |  +- org.apache.flink:flink-annotations:jar:1.16.0:provided
   [INFO] |  |  |  +- org.apache.flink:flink-metrics-core:jar:1.16.0:provided
   [INFO] |  |  |  +- org.apache.flink:flink-shaded-asm-9:jar:9.2-15.0:provided
   [INFO] |  |  |  \- org.apache.flink:flink-shaded-jackson:jar:2.12.4-15.0:provided
   [INFO] |  |  +- org.apache.flink:flink-file-sink-common:jar:1.16.0:provided
   [INFO] |  |  +- org.apache.flink:flink-runtime:jar:1.16.0:provided
   [INFO] |  |  |  +- org.apache.flink:flink-rpc-core:jar:1.16.0:provided
   [INFO] |  |  |  +- org.apache.flink:flink-rpc-akka-loader:jar:1.16.0:provided
   [INFO] |  |  |  +- org.apache.flink:flink-queryable-state-client-java:jar:1.16.0:provided
   [INFO] |  |  |  +- org.apache.flink:flink-hadoop-fs:jar:1.16.0:provided
   [INFO] |  |  |  +- org.apache.flink:flink-shaded-netty:jar:4.1.70.Final-15.0:provided
   [INFO] |  |  |  +- org.apache.flink:flink-shaded-zookeeper-3:jar:3.5.9-15.0:provided
   [INFO] |  |  |  \- org.javassist:javassist:jar:3.24.0-GA:compile
   [INFO] |  |  +- org.apache.flink:flink-java:jar:1.16.0:provided
   [INFO] |  |  |  \- com.twitter:chill-java:jar:0.7.6:provided
   [INFO] |  |  \- org.apache.flink:flink-shaded-guava:jar:30.1.1-jre-15.0:provided
   [INFO] |  +- org.apache.flink:flink-clients:jar:1.16.0:provided
   [INFO] |  |  \- org.apache.flink:flink-optimizer:jar:1.16.0:provided
   [INFO] |  +- org.apache.flink:flink-connector-kafka:jar:1.16.0:provided
   [INFO] |  |  \- org.apache.flink:flink-connector-base:jar:1.16.0:provided
   [INFO] |  +- org.apache.kafka:kafka-clients:jar:2.0.0:compile
   [INFO] |  |  \- org.xerial.snappy:snappy-java:jar:1.1.8.3:compile
   [INFO] |  +- org.apache.hadoop:hadoop-mapreduce-client-core:jar:2.10.1:provided
   [INFO] |  |  +- org.apache.hadoop:hadoop-yarn-client:jar:2.10.1:provided
   [INFO] |  |  |  \- org.apache.hadoop:hadoop-yarn-api:jar:2.10.1:provided
   [INFO] |  |  +- org.apache.hadoop:hadoop-yarn-common:jar:2.10.1:provided
   [INFO] |  |  |  +- org.mortbay.jetty:jetty-util:jar:6.1.26:provided
   [INFO] |  |  |  +- com.sun.jersey:jersey-client:jar:1.9:provided
   [INFO] |  |  |  +- org.codehaus.jackson:jackson-jaxrs:jar:1.9.13:provided
   [INFO] |  |  |  +- org.codehaus.jackson:jackson-xc:jar:1.9.13:provided
   [INFO] |  |  |  +- com.sun.jersey:jersey-json:jar:1.9:provided
   [INFO] |  |  |  |  \- com.sun.xml.bind:jaxb-impl:jar:2.2.3-1:provided
   [INFO] |  |  |  \- com.sun.jersey.contribs:jersey-guice:jar:1.9:provided
   [INFO] |  |  +- com.google.inject.extensions:guice-servlet:jar:3.0:provided
   [INFO] |  |  \- io.netty:netty:jar:3.10.6.Final:compile
   [INFO] |  \- com.beust:jcommander:jar:1.78:compile
   [INFO] +- org.apache.hudi:hudi-hadoop-mr:jar:0.14.0-SNAPSHOT:compile
   [INFO] +- org.apache.hudi:hudi-hive-sync:jar:0.14.0-SNAPSHOT:compile
   [INFO] +- org.apache.hudi:hudi-timeline-service:jar:0.14.0-SNAPSHOT:compile
   [INFO] |  \- io.javalin:javalin:jar:4.6.7:compile
   [INFO] |     +- org.eclipse.jetty.websocket:websocket-server:jar:9.4.48.v20220622:compile
   [INFO] |     |  +- org.eclipse.jetty.websocket:websocket-common:jar:9.4.48.v20220622:compile
   [INFO] |     |  |  \- org.eclipse.jetty.websocket:websocket-api:jar:9.4.48.v20220622:compile
   [INFO] |     |  +- org.eclipse.jetty.websocket:websocket-client:jar:9.4.48.v20220622:compile
   [INFO] |     |  |  \- org.eclipse.jetty:jetty-client:jar:9.4.48.v20220622:compile
   [INFO] |     |  \- org.eclipse.jetty.websocket:websocket-servlet:jar:9.4.48.v20220622:compile
   [INFO] |     \- org.jetbrains.kotlin:kotlin-stdlib-jdk8:jar:1.5.32:compile
   [INFO] |        +- org.jetbrains.kotlin:kotlin-stdlib:jar:1.5.32:compile
   [INFO] |        |  +- org.jetbrains:annotations:jar:13.0:compile
   [INFO] |        |  \- org.jetbrains.kotlin:kotlin-stdlib-common:jar:1.5.32:compile
   [INFO] |        \- org.jetbrains.kotlin:kotlin-stdlib-jdk7:jar:1.5.32:compile
   [INFO] +- com.esotericsoftware:kryo-shaded:jar:4.0.2:compile
   [INFO] |  +- com.esotericsoftware:minlog:jar:1.3.0:compile
   [INFO] |  \- org.objenesis:objenesis:jar:2.5.1:compile
   [INFO] +- javax.servlet:javax.servlet-api:jar:3.1.0:compile
   [INFO] +- org.apache.flink:flink-hadoop-compatibility_2.12:jar:1.16.0:compile
   [INFO] |  \- org.apache.flink:flink-shaded-force-shading:jar:15.0:compile
   [INFO] +- org.apache.flink:flink-parquet:jar:1.16.0:compile
   [INFO] |  +- com.google.protobuf:protobuf-java:jar:3.21.5:compile
   [INFO] |  \- com.google.code.findbugs:jsr305:jar:1.3.9:compile
   [INFO] +- org.apache.flink:flink-json:jar:1.16.0:compile
   [INFO] +- org.apache.parquet:parquet-hadoop:jar:1.12.2:compile
   [INFO] |  +- commons-pool:commons-pool:jar:1.6:compile
   [INFO] |  \- com.github.luben:zstd-jni:jar:1.4.9-1:compile
   [INFO] +- org.apache.parquet:parquet-avro:jar:1.12.2:compile
   [INFO] +- org.apache.parquet:parquet-column:jar:1.12.2:compile
   [INFO] +- org.apache.parquet:parquet-common:jar:1.12.2:compile
   [INFO] |  +- org.slf4j:slf4j-api:jar:1.7.36:provided
   [INFO] |  \- org.apache.yetus:audience-annotations:jar:0.12.0:compile
   [INFO] +- org.apache.parquet:parquet-encoding:jar:1.12.2:compile
   [INFO] +- org.apache.parquet:parquet-format-structures:jar:1.12.2:compile
   [INFO] |  \- javax.annotation:javax.annotation-api:jar:1.3.2:compile
   [INFO] +- org.apache.parquet:parquet-jackson:jar:1.12.2:compile
   [INFO] +- org.apache.avro:avro:jar:1.10.0:compile
   [INFO] |  \- org.apache.commons:commons-compress:jar:1.20:compile
   [INFO] +- org.apache.hive:hive-service:jar:2.3.1:provided
   [INFO] |  +- org.apache.hive:hive-service-rpc:jar:2.3.1:provided
   [INFO] |  +- org.apache.hive:hive-llap-server:jar:2.3.1:provided
   [INFO] |  |  +- org.apache.hive:hive-llap-common:jar:2.3.1:provided
   [INFO] |  |  +- org.apache.hive:hive-llap-client:jar:2.3.1:provided
   [INFO] |  |  +- org.apache.slider:slider-core:jar:0.90.2-incubating:provided
   [INFO] |  |  |  +- org.apache.hadoop:hadoop-client:jar:2.10.1:provided
   [INFO] |  |  |  |  +- org.apache.hadoop:hadoop-mapreduce-client-app:jar:2.10.1:provided
   [INFO] |  |  |  |  |  +- org.apache.hadoop:hadoop-mapreduce-client-common:jar:2.10.1:provided
   [INFO] |  |  |  |  |  \- org.apache.hadoop:hadoop-mapreduce-client-shuffle:jar:2.10.1:provided
   [INFO] |  |  |  |  \- org.apache.hadoop:hadoop-mapreduce-client-jobclient:jar:2.10.1:provided
   [INFO] |  |  |  +- org.apache.hadoop:hadoop-yarn-registry:jar:2.7.1:provided
   [INFO] |  |  |  +- org.mortbay.jetty:jetty:jar:6.1.26:provided
   [INFO] |  |  |  \- javax.servlet.jsp:jsp-api:jar:2.1:provided
   [INFO] |  |  +- org.codehaus.jettison:jettison:jar:1.1:provided
   [INFO] |  |  \- org.apache.hive:hive-llap-common:test-jar:tests:2.3.1:provided
   [INFO] |  +- commons-codec:commons-codec:jar:1.4:compile
   [INFO] |  +- commons-cli:commons-cli:jar:1.2:provided
   [INFO] |  +- net.sf.jpam:jpam:jar:1.1:provided
   [INFO] |  +- commons-lang:commons-lang:jar:2.6:provided
   [INFO] |  +- tomcat:jasper-compiler:jar:5.5.23:provided
   [INFO] |  |  +- javax.servlet:jsp-api:jar:2.0:provided
   [INFO] |  |  \- ant:ant:jar:1.6.5:provided
   [INFO] |  +- tomcat:jasper-runtime:jar:5.5.23:provided
   [INFO] |  |  +- javax.servlet:servlet-api:jar:2.4:provided
   [INFO] |  |  \- commons-el:commons-el:jar:1.0:provided
   [INFO] |  +- org.apache.thrift:libthrift:jar:0.9.3:provided
   [INFO] |  \- org.jamon:jamon-runtime:jar:2.3.1:compile
   [INFO] +- org.apache.hive:hive-exec:jar:2.3.1:provided
   [INFO] |  +- org.apache.hive:hive-vector-code-gen:jar:2.3.1:provided
   [INFO] |  |  \- org.apache.velocity:velocity:jar:1.5:provided
   [INFO] |  |     \- oro:oro:jar:2.0.8:provided
   [INFO] |  +- org.apache.hive:hive-llap-tez:jar:2.3.1:provided
   [INFO] |  +- org.apache.hive:hive-shims:jar:2.3.1:provided
   [INFO] |  |  +- org.apache.hive.shims:hive-shims-common:jar:2.3.1:provided
   [INFO] |  |  +- org.apache.hive.shims:hive-shims-0.23:jar:2.3.1:provided
   [INFO] |  |  |  \- org.apache.hadoop:hadoop-yarn-server-resourcemanager:jar:2.7.2:provided
   [INFO] |  |  |     +- org.apache.hadoop:hadoop-yarn-server-common:jar:2.7.2:provided
   [INFO] |  |  |     +- org.apache.hadoop:hadoop-yarn-server-applicationhistoryservice:jar:2.7.2:provided
   [INFO] |  |  |     +- org.apache.hadoop:hadoop-yarn-server-web-proxy:jar:2.7.2:provided
   [INFO] |  |  |     \- org.fusesource.leveldbjni:leveldbjni-all:jar:1.8:provided
   [INFO] |  |  \- org.apache.hive.shims:hive-shims-scheduler:jar:2.3.1:provided
   [INFO] |  +- commons-httpclient:commons-httpclient:jar:3.0.1:provided
   [INFO] |  |  \- junit:junit:jar:3.8.1:provided
   [INFO] |  +- org.antlr:antlr-runtime:jar:3.5.2:provided
   [INFO] |  +- org.antlr:ST4:jar:4.0.4:provided
   [INFO] |  +- org.apache.ant:ant:jar:1.9.1:provided
   [INFO] |  |  \- org.apache.ant:ant-launcher:jar:1.9.1:provided
   [INFO] |  +- org.apache.ivy:ivy:jar:2.4.0:provided
   [INFO] |  +- org.apache.curator:apache-curator:pom:2.7.1:provided
   [INFO] |  +- org.codehaus.groovy:groovy-all:jar:2.4.4:provided
   [INFO] |  +- org.apache.calcite:calcite-core:jar:1.10.0:provided
   [INFO] |  |  +- org.apache.calcite:calcite-linq4j:jar:1.10.0:provided
   [INFO] |  |  +- net.hydromatic:eigenbase-properties:jar:1.1.5:provided
   [INFO] |  |  +- org.codehaus.janino:janino:jar:2.7.6:provided
   [INFO] |  |  \- org.codehaus.janino:commons-compiler:jar:2.7.6:provided
   [INFO] |  +- org.apache.calcite:calcite-druid:jar:1.10.0:provided
   [INFO] |  +- org.apache.calcite.avatica:avatica:jar:1.8.0:provided
   [INFO] |  |  \- org.apache.calcite.avatica:avatica-metrics:jar:1.8.0:provided
   [INFO] |  +- com.google.code.gson:gson:jar:2.2.4:provided
   [INFO] |  \- stax:stax-api:jar:1.0.1:provided
   [INFO] +- org.apache.hive:hive-metastore:jar:2.3.1:provided
   [INFO] |  +- org.apache.hive:hive-serde:jar:2.3.1:provided
   [INFO] |  |  +- net.sf.opencsv:opencsv:jar:2.3:provided
   [INFO] |  |  \- org.apache.parquet:parquet-hadoop-bundle:jar:1.8.1:provided
   [INFO] |  +- javolution:javolution:jar:5.5.1:provided
   [INFO] |  +- com.jolbox:bonecp:jar:0.8.0.RELEASE:provided
   [INFO] |  +- com.zaxxer:HikariCP:jar:2.5.1:provided
   [INFO] |  +- org.apache.derby:derby:jar:10.10.2.0:provided
   [INFO] |  +- org.datanucleus:datanucleus-rdbms:jar:4.1.19:provided
   [INFO] |  +- commons-dbcp:commons-dbcp:jar:1.4:provided
   [INFO] |  +- javax.jdo:jdo-api:jar:3.0.1:provided
   [INFO] |  +- org.datanucleus:javax.jdo:jar:3.2.0-m3:provided
   [INFO] |  +- co.cask.tephra:tephra-api:jar:0.6.0:provided
   [INFO] |  +- co.cask.tephra:tephra-core:jar:0.6.0:provided
   [INFO] |  |  +- com.google.inject:guice:jar:3.0:provided
   [INFO] |  |  |  +- javax.inject:javax.inject:jar:1:provided
   [INFO] |  |  |  \- aopalliance:aopalliance:jar:1.0:provided
   [INFO] |  |  +- com.google.inject.extensions:guice-assistedinject:jar:3.0:provided
   [INFO] |  |  +- it.unimi.dsi:fastutil:jar:6.5.6:provided
   [INFO] |  |  +- org.apache.twill:twill-common:jar:0.6.0-incubating:provided
   [INFO] |  |  +- org.apache.twill:twill-core:jar:0.6.0-incubating:provided
   [INFO] |  |  |  \- org.apache.twill:twill-api:jar:0.6.0-incubating:provided
   [INFO] |  |  +- org.apache.twill:twill-discovery-api:jar:0.6.0-incubating:provided
   [INFO] |  |  +- org.apache.twill:twill-discovery-core:jar:0.6.0-incubating:provided
   [INFO] |  |  \- org.apache.twill:twill-zookeeper:jar:0.6.0-incubating:provided
   [INFO] |  \- co.cask.tephra:tephra-hbase-compat-1.0:jar:0.6.0:provided
   [INFO] +- org.apache.hive:hive-jdbc:jar:2.3.1:provided
   [INFO] |  +- org.apache.httpcomponents:httpcore:jar:4.4.1:compile
   [INFO] |  \- org.apache.zookeeper:zookeeper:jar:3.4.6:compile
   [INFO] +- org.apache.hive:hive-common:jar:2.3.1:provided
   [INFO] |  +- org.apache.hive:hive-storage-api:jar:2.3.1:provided
   [INFO] |  +- org.apache.commons:commons-lang3:jar:3.1:compile
   [INFO] |  +- jline:jline:jar:2.12:compile
   [INFO] |  +- com.tdunning:json:jar:1.8:provided
   [INFO] |  +- io.dropwizard.metrics:metrics-jvm:jar:3.1.0:provided
   [INFO] |  +- io.dropwizard.metrics:metrics-json:jar:3.1.0:provided
   [INFO] |  \- com.github.joshelser:dropwizard-metrics-hadoop-metrics2-reporter:jar:0.1.2:provided
   [INFO] +- org.datanucleus:datanucleus-core:jar:5.0.1:provided
   [INFO] +- org.datanucleus:datanucleus-api-jdo:jar:5.0.1:provided
   [INFO] +- joda-time:joda-time:jar:2.9.9:compile
   [INFO] +- org.eclipse.jetty:jetty-server:jar:9.4.48.v20220622:compile
   [INFO] |  \- org.eclipse.jetty:jetty-io:jar:9.4.48.v20220622:compile
   [INFO] +- org.eclipse.jetty:jetty-util:jar:9.4.48.v20220622:compile
   [INFO] +- org.eclipse.jetty:jetty-webapp:jar:9.4.48.v20220622:compile
   [INFO] |  +- org.eclipse.jetty:jetty-xml:jar:9.4.48.v20220622:compile
   [INFO] |  \- org.eclipse.jetty:jetty-servlet:jar:9.4.48.v20220622:compile
   [INFO] |     +- org.eclipse.jetty:jetty-security:jar:9.4.48.v20220622:compile
   [INFO] |     \- org.eclipse.jetty:jetty-util-ajax:jar:9.4.48.v20220622:compile
   [INFO] +- org.eclipse.jetty:jetty-http:jar:9.4.48.v20220622:compile
   [INFO] +- org.apache.thrift:libfb303:jar:0.9.3:provided
   [INFO] +- org.apache.orc:orc-core:jar:1.5.6:provided
   [INFO] |  +- org.apache.orc:orc-shims:jar:1.5.6:provided
   [INFO] |  +- io.airlift:aircompressor:jar:0.10:provided
   [INFO] |  +- javax.xml.bind:jaxb-api:jar:2.2.11:compile
   [INFO] |  +- org.apache.hadoop:hadoop-common:jar:2.10.1:provided
   [INFO] |  |  +- com.google.guava:guava:jar:11.0.2:provided
   [INFO] |  |  +- xmlenc:xmlenc:jar:0.52:provided
   [INFO] |  |  +- commons-net:commons-net:jar:3.1:provided
   [INFO] |  |  +- commons-collections:commons-collections:jar:3.2.2:provided
   [INFO] |  |  +- org.mortbay.jetty:jetty-sslengine:jar:6.1.26:provided
   [INFO] |  |  +- com.sun.jersey:jersey-core:jar:1.9:provided
   [INFO] |  |  +- com.sun.jersey:jersey-server:jar:1.9:provided
   [INFO] |  |  |  \- asm:asm:jar:3.1:provided
   [INFO] |  |  +- commons-configuration:commons-configuration:jar:1.6:provided
   [INFO] |  |  +- commons-beanutils:commons-beanutils:jar:1.9.4:provided
   [INFO] |  |  +- org.codehaus.jackson:jackson-core-asl:jar:1.9.13:provided
   [INFO] |  |  +- org.codehaus.jackson:jackson-mapper-asl:jar:1.9.13:provided
   [INFO] |  |  +- com.jcraft:jsch:jar:0.1.55:provided
   [INFO] |  |  +- org.codehaus.woodstox:stax2-api:jar:3.1.4:provided
   [INFO] |  |  \- com.fasterxml.woodstox:woodstox-core:jar:5.0.3:provided
   [INFO] |  \- org.apache.hadoop:hadoop-hdfs:jar:2.10.1:provided
   [INFO] |     +- org.apache.hadoop:hadoop-hdfs-client:jar:2.10.1:provided
   [INFO] |     |  \- com.squareup.okhttp:okhttp:jar:2.7.5:provided
   [INFO] |     |     \- com.squareup.okio:okio:jar:1.6.0:provided
   [INFO] |     \- io.netty:netty-all:jar:4.1.50.Final:provided
   [INFO] +- com.fasterxml.jackson.core:jackson-core:jar:2.6.7:compile
   [INFO] +- com.fasterxml.jackson.core:jackson-databind:jar:2.6.7.3:compile
   [INFO] \- com.fasterxml.jackson.core:jackson-annotations:jar:2.6.7:compile
   ```


-- 
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


Re: [I] [SUPPORT] Missing dependency on hive-exec (core) [hudi]

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

   I have the same issue when I'm trying to use hudi-flink1.17-bundle-0.14.1.jar with Flink and sync to Hive. Is there any way to fix it? Adding hive-exec like above results in the same error as above.


-- 
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] danny0405 closed issue #8147: [SUPPORT] Missing dependency on hive-exec (core)

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 closed issue #8147: [SUPPORT] Missing dependency on hive-exec (core)
URL: https://github.com/apache/hudi/issues/8147


-- 
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] gamblewin commented on issue #8147: [SUPPORT] Missing dependency on hive-exec (core)

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

   hi, I meet same problem when trying to query data from a hudi table in Flink SQL console.
   I compile hudi-0.13.0 source code by myself, so what should I do to avoid this mistake?
   should I add `hive-exec` dependency in the `pom.xml` file of hudi-0.13.0 and recompile it?


-- 
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] kkrugler commented on issue #8147: [SUPPORT] Missing dependency on hive-exec (core)

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

   @gamblewin - if you can, I think the easiest solution is to ad the `hive-exec` jar to your Flink /lib directory.


-- 
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] kkrugler commented on issue #8147: [SUPPORT] Missing dependency on hive-exec (core)

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

   I was also confused by that. I think when the `HoodieInputFormatUtils` class is loaded via the call from `WriteProfiles. getCommitMetadata()` to `HoodieInputFormatUtils.getCommitMetadata()`, this indirectly triggers a reference to `MapredParquetInputFormat` (e.g. maybe through a static class reference?).


-- 
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] danny0405 commented on issue #8147: [SUPPORT] Missing dependency on hive-exec (core)

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

   You are right, the bundle jar should keep more slim.


-- 
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