[ 
https://issues.apache.org/jira/browse/KAFKA-9225?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

jiamei xie updated KAFKA-9225:
------------------------------
    Description: 
Steps to reproduce:

1. Download Kafka latest source code

2. Build it with gradle

3. Run [stream quick 
start|[https://kafka.apache.org/23/documentation/streams/quickstart]]

when running

bin/kafka-run-class.sh 
org.apache.kafka.streams.examples.wordcount.WordCountDemo, it crashed with the 
following error message:

SLF4J: Class path contains multiple SLF4J bindings.

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/core/build/dependant-libs-2.12.10/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/tools/build/dependant-libs-2.12.10/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/api/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/transforms/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/runtime/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/file/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/mirror/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/mirror-client/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/json/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/basic-auth-extension/build/dependant-libs/slf4j-log4j12-1.7.28.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.slf4j.impl.Log4jLoggerFactory]

[2019-11-19 15:42:23,277] WARN The configuration 'admin.retries' was supplied 
but isn't a known config. (org.apache.kafka.clients.consumer.ConsumerConfig)

[2019-11-19 15:42:23,278] WARN The configuration 'admin.retry.backoff.ms' was 
supplied but isn't a known config. 
(org.apache.kafka.clients.consumer.ConsumerConfig)

[2019-11-19 15:42:24,278] ERROR stream-client 
[streams-wordcount-0f3cf88b-e2c4-4fb6-b7a3-9754fad5cd48] All stream threads 
have died. The instance will be in error state and should be closed. (org.apach 
                                e.kafka.streams.KafkaStreams)

Exception in thread 
"streams-wordcount-0f3cf88b-e2c4-4fb6-b7a3-9754fad5cd48-StreamThread-1" 
java.lang.UnsatisfiedLinkError: /tmp/librocksdbjni1377754636857652484.so: 
/tmp/librocksdbjni13777546368576524                                 84.so: 
cannot open shared object file: No such file or directory (Possible cause: 
can't load AMD 64-bit .so on a AARCH64-bit platform)

Analyze:

This issue is caused by rocksdbjni-5.18.3.jar which doesn't come with aarch64 
native support. Replace rocksdbjni-5.18.3.jar with rocksdbjni-6.3.6.jar from 
[https://mvnrepository.com/artifact/org.rocksdb/rocksdbjni/6.3.6] can fix this 
problem.

Attached is the binary compatibility report of rocksdbjni.jar between 5.18.3 
and 6.3.6. The result is 81.8%. So is it possible to upgrade rocksdbjni to 
6.3.6 in upstream? Should there be any kind of tests to execute, please kindly 
point me. Thanks a lot.

  was:
Steps to reproduce:

1. Download Kafka latest source code

2. Build it with gradle

3. Run [stream quick 
start|[https://kafka.apache.org/23/documentation/streams/quickstart]]

when running bin/kafka-run-class.sh 
org.apache.kafka.streams.examples.wordcount.WordCountDemo, it crashed with the 
following error message:

SLF4J: Class path contains multiple SLF4J bindings.

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/core/build/dependant-libs-2.12.10/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/tools/build/dependant-libs-2.12.10/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/api/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/transforms/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/runtime/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/file/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/mirror/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/mirror-client/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/json/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in 
[jar:file:/home/xjm/kafka/connect/basic-auth-extension/build/dependant-libs/slf4j-log4j12-1.7.28.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.slf4j.impl.Log4jLoggerFactory]

[2019-11-19 15:42:23,277] WARN The configuration 'admin.retries' was supplied 
but isn't a known config. (org.apache.kafka.clients.consumer.ConsumerConfig)

[2019-11-19 15:42:23,278] WARN The configuration 'admin.retry.backoff.ms' was 
supplied but isn't a known config. 
(org.apache.kafka.clients.consumer.ConsumerConfig)

[2019-11-19 15:42:24,278] ERROR stream-client 
[streams-wordcount-0f3cf88b-e2c4-4fb6-b7a3-9754fad5cd48] All stream threads 
have died. The instance will be in error state and should be closed. (org.apach 
                                e.kafka.streams.KafkaStreams)

Exception in thread 
"streams-wordcount-0f3cf88b-e2c4-4fb6-b7a3-9754fad5cd48-StreamThread-1" 
java.lang.UnsatisfiedLinkError: /tmp/librocksdbjni1377754636857652484.so: 
/tmp/librocksdbjni13777546368576524                                 84.so: 
cannot open shared object file: No such file or directory (Possible cause: 
can't load AMD 64-bit .so on a AARCH64-bit platform)

Analyze:

This issue is caused by rocksdbjni-5.18.3.jar which doesn't come with aarch64 
native support. Replace rocksdbjni-5.18.3.jar with rocksdbjni-6.3.6.jar from 
[https://mvnrepository.com/artifact/org.rocksdb/rocksdbjni/6.3.6] can fix this 
problem.

Attached is the binary compatibility report of rocksdbjni.jar between 5.18.3 
and 6.3.6. The result is 81.8%. So is it possible to upgrade rocksdbjni to 
6.3.6 in upstream? Should there be any kind of tests to execute, please kindly 
point me. Thanks a lot.


> kafka fail to run on linux-aarch64
> ----------------------------------
>
>                 Key: KAFKA-9225
>                 URL: https://issues.apache.org/jira/browse/KAFKA-9225
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>            Reporter: jiamei xie
>            Priority: Major
>              Labels: incompatible
>         Attachments: compat_report.html
>
>
> Steps to reproduce:
> 1. Download Kafka latest source code
> 2. Build it with gradle
> 3. Run [stream quick 
> start|[https://kafka.apache.org/23/documentation/streams/quickstart]]
> when running
> bin/kafka-run-class.sh 
> org.apache.kafka.streams.examples.wordcount.WordCountDemo, it crashed with 
> the following error message:
> SLF4J: Class path contains multiple SLF4J bindings.
> SLF4J: Found binding in 
> [jar:file:/home/xjm/kafka/core/build/dependant-libs-2.12.10/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in 
> [jar:file:/home/xjm/kafka/tools/build/dependant-libs-2.12.10/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in 
> [jar:file:/home/xjm/kafka/connect/api/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in 
> [jar:file:/home/xjm/kafka/connect/transforms/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in 
> [jar:file:/home/xjm/kafka/connect/runtime/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in 
> [jar:file:/home/xjm/kafka/connect/file/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in 
> [jar:file:/home/xjm/kafka/connect/mirror/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in 
> [jar:file:/home/xjm/kafka/connect/mirror-client/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in 
> [jar:file:/home/xjm/kafka/connect/json/build/dependant-libs/slf4j-log4j12-1.7.28.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in 
> [jar:file:/home/xjm/kafka/connect/basic-auth-extension/build/dependant-libs/slf4j-log4j12-1.7.28.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.slf4j.impl.Log4jLoggerFactory]
> [2019-11-19 15:42:23,277] WARN The configuration 'admin.retries' was supplied 
> but isn't a known config. (org.apache.kafka.clients.consumer.ConsumerConfig)
> [2019-11-19 15:42:23,278] WARN The configuration 'admin.retry.backoff.ms' was 
> supplied but isn't a known config. 
> (org.apache.kafka.clients.consumer.ConsumerConfig)
> [2019-11-19 15:42:24,278] ERROR stream-client 
> [streams-wordcount-0f3cf88b-e2c4-4fb6-b7a3-9754fad5cd48] All stream threads 
> have died. The instance will be in error state and should be closed. 
> (org.apach                                 e.kafka.streams.KafkaStreams)
> Exception in thread 
> "streams-wordcount-0f3cf88b-e2c4-4fb6-b7a3-9754fad5cd48-StreamThread-1" 
> java.lang.UnsatisfiedLinkError: /tmp/librocksdbjni1377754636857652484.so: 
> /tmp/librocksdbjni13777546368576524                                 84.so: 
> cannot open shared object file: No such file or directory (Possible cause: 
> can't load AMD 64-bit .so on a AARCH64-bit platform)
> Analyze:
> This issue is caused by rocksdbjni-5.18.3.jar which doesn't come with aarch64 
> native support. Replace rocksdbjni-5.18.3.jar with rocksdbjni-6.3.6.jar from 
> [https://mvnrepository.com/artifact/org.rocksdb/rocksdbjni/6.3.6] can fix 
> this problem.
> Attached is the binary compatibility report of rocksdbjni.jar between 5.18.3 
> and 6.3.6. The result is 81.8%. So is it possible to upgrade rocksdbjni to 
> 6.3.6 in upstream? Should there be any kind of tests to execute, please 
> kindly point me. Thanks a lot.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to