Kafka Connect: fix Hadoop dependency exclusion#11516
Conversation
| I asked the user to test this PR on MSK (where the issue happens), so just waiting on that. |
singhpk234 left a comment
There was a problem hiding this comment.
LGTM as well, Thanks @bryanck !
I'm deploying it since MSK Connect it's a black box completely. I'll know in a minute. |
| exclude group: 'org.slf4j' | ||
| exclude group: 'ch.qos.reload4j' | ||
| exclude group: 'org.apache.avro', module: 'avro' | ||
| exclude group: 'com.fasterxml.woodstox' |
There was a problem hiding this comment.
I observed the same issue recently when I copied this block of dependency exclusion for REST TCK runtime image
https://github.com/apache/iceberg/pull/11283/files#diff-49a96e7eea8a94af862798a45174e6ac43eb4f8b4bd40759b5da63ba31ec3ef7R975
I too included it after that.
And when I test the same thing with HadoopFileIO,
a) hadoop-auth dependency was also needed.
b) commons-configuration2 was also needed.
So, may be better you can test it out that too. (line 104, 105)
Callstack for me.
java.lang.NoClassDefFoundError: org/apache/hadoop/util/PlatformName at org.apache.hadoop.security.UserGroupInformation.getOSLoginModuleName(UserGroupInformation.java:433) at org.apache.hadoop.security.UserGroupInformation.<clinit>(UserGroupInformation.java:461) at org.apache.hadoop.fs.FileSystem$Cache$Key.<init>(FileSystem.java:3888) at org.apache.hadoop.fs.FileSystem$Cache$Key.<init>(FileSystem.java:3878) at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3666) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:557) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:289) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:541) at org.apache.hadoop.fs.Path.getFileSystem(Path.java:366) at org.apache.iceberg.hadoop.Util.getFs(Util.java:56) at org.apache.iceberg.hadoop.HadoopOutputFile.fromPath(HadoopOutputFile.java:53) at org.apache.iceberg.hadoop.HadoopFileIO.newOutputFile(HadoopFileIO.java:97) at org.apache.iceberg.BaseMetastoreTableOperations.writeNewMetadata(BaseMetastoreTableOperations.java:155) at org.apache.iceberg.BaseMetastoreTableOperations.writeNewMetadataIfRequired(BaseMetastoreTableOperations.java:150) at org.apache.iceberg.jdbc.JdbcTableOperations.doCommit(JdbcTableOperations.java:106) at org.apache.iceberg.BaseMetastoreTableOperations.commit(BaseMetastoreTableOperations.java:125) at org.apache.iceberg.BaseMetastoreCatalog$BaseMetastoreCatalogTableBuilder.create(BaseMetastoreCatalog.java:201) at org.apache.iceberg.rest.CatalogHandlers.createTable(CatalogHandlers.java:274) at org.apache.iceberg.rest.RESTCatalogAdapter.handleRequest(RESTCatalogAdapter.java:377) at org.apache.iceberg.rest.RESTServerCatalogAdapter.handleRequest(RESTServerCatalogAdapter.java:42) at org.apache.iceberg.rest.RESTCatalogAdapter.execute(RESTCatalogAdapter.java:558) at org.apache.iceberg.rest.RESTCatalogServlet.execute(RESTCatalogServlet.java:100) at org.apache.iceberg.rest.RESTCatalogServlet.doPost(RESTCatalogServlet.java:78) There was a problem hiding this comment.
Thanks @ajantha-bhat , the default connector distribution doesn't package the Hadoop client libs, so HadoopFileIO won't work with it.
There was a problem hiding this comment.
Thanks @bryanck !
Unfortunately it's not working. I didn't update the issue with my most recent findings but I had to include as well woodstox only dependency, stax2-api, to make it work.
Now with the changes applied in this pull request the error is:
ERROR [REDACTED|task-0] WorkerSinkTask{id=REDACTED-0} Task threw an uncaught and unrecoverable exception. Task is being killed and will not recover until manually restarted (org.apache.kafka.connect.runtime.WorkerTask:193) java.lang.NoClassDefFoundError: org/codehaus/stax2/XMLInputFactory2 at java.base/java.lang.ClassLoader.defineClass1(Native Method) at java.base/java.lang.ClassLoader.defineClass(ClassLoader.java:1022) at java.base/java.security.SecureClassLoader.defineClass(SecureClassLoader.java:174) at java.base/java.net.URLClassLoader.defineClass(URLClassLoader.java:555) at java.base/java.net.URLClassLoader$1.run(URLClassLoader.java:458) at java.base/java.net.URLClassLoader$1.run(URLClassLoader.java:452) at java.base/java.security.AccessController.doPrivileged(Native Method) at java.base/java.net.URLClassLoader.findClass(URLClassLoader.java:451) at org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:96) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:527) at org.apache.hadoop.conf.Configuration.<clinit>(Configuration.java:360) at java.base/java.lang.Class.forName0(Native Method) at java.base/java.lang.Class.forName(Class.java:398) at org.apache.iceberg.common.DynClasses$Builder.impl(DynClasses.java:68) at org.apache.iceberg.connect.CatalogUtils.loadHadoopConfig(CatalogUtils.java:53) at org.apache.iceberg.connect.CatalogUtils.loadCatalog(CatalogUtils.java:45) at org.apache.iceberg.connect.IcebergSinkTask.open(IcebergSinkTask.java:56) at org.apache.kafka.connect.runtime.WorkerSinkTask.openPartitions(WorkerSinkTask.java:641) at org.apache.kafka.connect.runtime.WorkerSinkTask.access$1100(WorkerSinkTask.java:71) at org.apache.kafka.connect.runtime.WorkerSinkTask$HandleRebalance.onPartitionsAssigned(WorkerSinkTask.java:706) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.invokePartitionsAssigned(ConsumerCoordinator.java:293) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onJoinComplete(ConsumerCoordinator.java:430) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:449) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:365) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:508) at org.apache.kafka.clients.consumer.KafkaConsumer.updateAssignmentMetadataIfNeeded(KafkaConsumer.java:1257) at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1226) at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1206) at org.apache.kafka.connect.runtime.WorkerSinkTask.pollConsumer(WorkerSinkTask.java:458) at org.apache.kafka.connect.runtime.WorkerSinkTask.poll(WorkerSinkTask.java:325) at org.apache.kafka.connect.runtime.WorkerSinkTask.iteration(WorkerSinkTask.java:232) at org.apache.kafka.connect.runtime.WorkerSinkTask.execute(WorkerSinkTask.java:201) at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:191) at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:240) at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) at java.base/java.lang.Thread.run(Thread.java:829) Caused by: java.lang.ClassNotFoundException: org.codehaus.stax2.XMLInputFactory2 at java.base/java.net.URLClassLoader.findClass(URLClassLoader.java:476) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:594) at org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:104) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:527) ... 39 more | I'm including here my build scripts to make it work for now if it's of any usefulness.
SHELL=/bin/bash SCHEMA_REGISTRY_CONVERTER_VERSION := 1.1.20 SCHEMA_REGISTRY_CONVERTER_ARTIFACT := software.amazon.glue:schema-registry-kafkaconnect-converter:${SCHEMA_REGISTRY_CONVERTER_VERSION} SCHEMA_REGISTRY_CONVERTER_JAR := schema-registry-kafkaconnect-converter-${SCHEMA_REGISTRY_CONVERTER_VERSION}.jar # This JAR and its dependency won't be necessary when this issue is fixed # https://github.com/apache/iceberg/issues/11489 WOODSTOX_VERSION := 6.7.0 WOODSTOX_ARTIFACT := com.fasterxml.woodstox:woodstox-core:${WOODSTOX_VERSION} WOODSTOX_JAR := woodstox-core-${WOODSTOX_VERSION}.jar STAX_VERSION := 4.2.2 STAX_ARTIFACT := org.codehaus.woodstox:stax2-api:${STAX_VERSION} STAX_JAR := stax2-api-${STAX_VERSION}.jar CONNECTOR_ZIP := iceberg-sink-*-*.zip .DEFAULT_GOAL := all .PHONY: all create-aws-plugin clean all: $(CONNECTOR_ZIP) $(SCHEMA_REGISTRY_CONVERTER_JAR) $(WOODSTOX_JAR) $(STAX_JAR) zip $(CONNECTOR_ZIP) $(SCHEMA_REGISTRY_CONVERTER_JAR) $(WOODSTOX_JAR) $(STAX_JAR) $(CONNECTOR_ZIP): ./build.sh $(SCHEMA_REGISTRY_CONVERTER_JAR): # You can run this inside a `maven` docker image if you don't have `mvn` locally mvn dependency:copy -Dartifact=$(SCHEMA_REGISTRY_CONVERTER_ARTIFACT) -DoutputDirectory=$(pwd) $(WOODSTOX_JAR): # You can run this inside a `maven` docker image if you don't have `mvn` locally mvn dependency:copy -Dartifact=$(WOODSTOX_ARTIFACT) -DoutputDirectory=$(pwd) $(STAX_JAR): # You can run this inside a `maven` docker image if you don't have `mvn` locally mvn dependency:copy -Dartifact=$(STAX_ARTIFACT) -DoutputDirectory=$(pwd) clean: @rm -f $(CONNECTOR_ZIP) $(SCHEMA_REGISTRY_CONVERTER_JAR) $(WOODSTOX_JAR) $(STAX_JAR)
#!/usr/bin/env bash set -eo pipefail ICEBERG_VERSION=1.8.0 [[ -n $ICEBERG_LOCATION ]] || { echo "ICEBERG_LOCATION environment variable not defined" exit 1 } [[ -d $ICEBERG_LOCATION ]] || { echo "No directory exists at $ICEBERG_LOCATION" exit 1 } pushd "$ICEBERG_LOCATION" ./gradlew -x test -x integrationTest build GIT_COMMIT=$(git rev-parse --short HEAD) popd ARTIFACT_FOLDER="kafka-connect/kafka-connect-runtime/build/distributions" ARTIFACT_PATH="$ICEBERG_LOCATION/$ARTIFACT_FOLDER/iceberg-kafka-connect-runtime-$ICEBERG_VERSION-SNAPSHOT.zip" cp "$ARTIFACT_PATH" "iceberg-sink-$ICEBERG_VERSION-$GIT_COMMIT.zip" |
| Thanks for testing @josepanguera , any chance you could test one more time? I removed another exclude. If that doesn't work I'll debug on MSK myself. |
josepanguera left a comment
There was a problem hiding this comment.
Now it's working, thanks again Bryan for the time!
Just a little comment, that I guess should be fixed at repository level and not here. The version included now of woodstox-core is 5.4.0 (instead of the previous 6.7.0). And 5.4.0 have a security vulnerability https://mvnrepository.com/artifact/com.fasterxml.woodstox/woodstox-core/5.4.0.
| The unfortunate fact is many Hadoop/Hive dependencies have security vulnerabilities, which is why we have 2 connector distributions (one with Hive, one without). I pushed an update to force the version to 6.7.0, if you want to try again... |
Oh, I see. I cannot try it until later, I'll update the issue as soon as I check it. |
| I'm a little lost on this, why is this difficult on MSK? If we don't know the classpath is there ever a way we can really test things? |
| The full classpath depends on the Connect framework being used to run the connector (MSK, Strimzi, Confluent, etc), so in this case we need to deploy to MSK to really test this. Josep was kind enough to do that. We use the Confluent image for the integration tests, and that already has this dependency so the tests were passing. |
If this is the case, can ever actually stay compatible? Aren't we at the mercy of these other builds? |
| To some extent we are at their mercy. But in this case, it is really our bug, because we were relying on a library to be on the classpath that we shouldn't have relied on. We got "lucky" before on MSK because the Azure dependency happened to also require this library so it was being included in the runtime. When Azure was updated, it was no longer included and caused the error. The library is actually required by our Hadoop dependency, not Connect. |
josepanguera left a comment
There was a problem hiding this comment.
Working as well with the commit forcing the version dc284da
🚀
| Awesome, thanks @josepanguera for reporting this and testing the fix, and thanks @nastra @Fokko @RussellSpitzer @ajantha-bhat and @singhpk234 for the review! |
This PR removes the exclusion of the woodstox libraries from the Hadoop transitive dependencies when building the Kafka Connect distribution, as they are needed to load Hadoop's
Configuration. Previously the required libraries were brought in by the Azure dependencies until that was upgraded in this commit, so the connector did not have any issues.This addresses #11489.