librdkafka
librdkafka copied to clipboard
ssl: support libssls with no ENGINE implementation
OpenSSL can be built without ENGINE support, and some libssl-compatible forks (e.g. BoringSSL) don't contain any ENGINE implementation at all - guard all references to the ENGINE API using OPENSSL_NO_ENGINE
so these libssls can be used with librdkafka.
Please rebase this on latest master, and verify that it builds correctly with LibreSSL.
Thanks!
While rebasing on master, I also refactored the changes to src/rdkafka_ssl.c
- now there's an OPENSSL_HAS_ENGINE
macro that eliminates the need to repeat the conditional every time.
The following table lists the result of the 0124_openssl_invalid_engine
test with various libssls before and after applying this PR:
libssl | master | #3535 |
---|---|---|
OpenSSL 1.0.2u | SKIPPED | SKIPPED |
OpenSSL 1.1.1m with ENGINE | PASSED | PASSED |
OpenSSL 1.1.1m without ENGINE | Build failure | SKIPPED |
LibreSSL 3.3.4 | SKIPPED | PASSED |
SKIPPED indicates that the test was skipped with the reason Configuration property "ssl.engine.location" not supported in this build: OpenSSL with ENGINE support not available at build time
.
Note that the test isn't skipped on LibreSSL any more - LibreSSL 3.3 does contain an ENGINE component, and it should probably be used if enabled (e.g. so it can make use of AES-NI).
Some style-fixes still needed (run make style-fix
)
Unfortunately it seems that clang-format bounces between
.unsupported = "OpenSSL with ENGINE support not available at build " \
"time"
and
.unsupported = \
"OpenSSL with ENGINE support not available at build " \
"time"
as recommendations - using one always causes it to suggest the other. There's no way that's not a bug in clang-format, but I'll just reword that string so it fits on a single line.
Tried to get clever by using a string that was unambiguously over the 80-column limit, but clang-format wasn't having any of that either. Reverted to a shorter, single-line message and it looks like we're there now. Sorry for all the noise!
Thanks for all your work on this @chrisnovakovic!
We're a bit too close to the 1.9.0 release to merge this, but we'll make sure to include it in a 1.9.1 release (scheduled for June).
Rebased on master and edited to account for the changes made to accommodate OpenSSL 3.0 - #4026 defines WITH_SSL_ENGINE
(making most of this PR redundant), but it doesn't consider the value of OPENSSL_NO_ENGINE
when defining WITH_SSL_ENGINE
, so librdkafka still fails to link to libssls that don't have ENGINE support due to missing symbols.
We need this change as well in order to build librdkafka.