Skip to content
This repository has been archived by the owner on Jan 24, 2024. It is now read-only.

[FEATURE] Support multiple listeners of one protocol #574

Closed
BewareMyPower opened this issue Jun 15, 2021 · 3 comments · Fixed by #742
Closed

[FEATURE] Support multiple listeners of one protocol #574

BewareMyPower opened this issue Jun 15, 2021 · 3 comments · Fixed by #742
Assignees
Labels
type/feature Indicates new functionality

Comments

@BewareMyPower
Copy link
Collaborator

BewareMyPower commented Jun 15, 2021

Is your feature request related to a problem? Please describe.

Currently for KoP we can configure kafkaListeners as what interfaces KoP binds to and kafkaAdvertisedListeners as how clients connect. However, currently KoP can only bind to one interface for each protocol.

For example, if KoP was configured with

kafkaListeners=PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093

it would work well.

But if KoP was configured with

kafkaListeners=PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093,SSL://127.0.0.1:9094

KoP would fail to start with following logs.

15:56:29.408 [main] ERROR io.streamnative.pulsar.handlers.kop.KafkaProtocolHandler - KafkaProtocolHandler newChannelInitializers failed with 
java.lang.IllegalStateException: PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093,SSL://127.0.0.1:9094 has multiple listeners whose protocol is SSL
	at io.streamnative.pulsar.handlers.kop.EndPoint.parseListeners(EndPoint.java:77) ~[pulsar-protocol-handler-kafka-2.8.0-rc-202106091215.nar-unpacked/:?]
	at io.streamnative.pulsar.handlers.kop.KafkaProtocolHandler.newChannelInitializers(KafkaProtocolHandler.java:313) [pulsar-protocol-handler-kafka-2.8.0-rc-202106091215.nar-unpacked/:?]
	at org.apache.pulsar.broker.protocol.ProtocolHandlerWithClassLoader.newChannelInitializers(ProtocolHandlerWithClassLoader.java:71) [io.streamnative-pulsar-broker-2.8.0-rc-202106071430.jar:2.8.0-rc-202106071430]
	at org.apache.pulsar.broker.protocol.ProtocolHandlers.newChannelInitializers(ProtocolHandlers.java:124) [io.streamnative-pulsar-broker-2.8.0-rc-202106071430.jar:2.8.0-rc-202106071430]
	at org.apache.pulsar.broker.PulsarService.start(PulsarService.java:787) [io.streamnative-pulsar-broker-2.8.0-rc-202106071430.jar:2.8.0-rc-202106071430]
	at org.apache.pulsar.PulsarStandalone.start(PulsarStandalone.java:296) [io.streamnative-pulsar-broker-2.8.0-rc-202106071430.jar:2.8.0-rc-202106071430]
	at org.apache.pulsar.PulsarStandaloneStarter.main(PulsarStandaloneStarter.java:121) [io.streamnative-pulsar-broker-2.8.0-rc-202106071430.jar:2.8.0-rc-202106071430]

Describe the solution you'd like
Kafka (as of KIP-103) has the listener.security.protocol.map config to map your own protocol define a set of listeners, each with a name and a security protocol (PLAINTEXT, SSL), so that a protocol like SSL could bind multiple ports or IPs.

We can also see more details in this blog. Here's an example to advertise multiple ports of PLAINTEXT protocol:

advertised.listeners=PLAINTEXT://broker:9092,CONNECTIONS_FROM_HOST://localhost:19092
listener.security.protocol.map=PLAINTEXT:PLAINTEXT,CONNECTIONS_FROM_HOST:PLAINTEXT

Describe alternatives you've considered
A clear and concise description of any alternative solutions or features you've considered.

Additional context
Add any other context or screenshots about the feature request here.

@BewareMyPower BewareMyPower added the type/feature Indicates new functionality label Jun 15, 2021
@BewareMyPower BewareMyPower changed the title [FEATURE] Support multiple listeners [FEATURE] Support multiple listeners of one protocol Jun 15, 2021
@BewareMyPower BewareMyPower mentioned this issue Jun 18, 2021
9 tasks
@EronWright
Copy link
Contributor

EronWright commented Jul 27, 2021

Please don't use the phrase "define your own protocol", use "define your own listener". The description in KIP-103 is:

Kafka brokers should be able to define multiple listeners for the same security protocol for binding (i.e. listeners) and sharing (i.e. advertised.listeners) so that internal, external and replication traffic can be separated if required.

@EronWright
Copy link
Contributor

EronWright commented Jul 27, 2021

Another important point in KIP-103 is that the Kafka metadata response varies depending on which listener received the traffic:

As stated previously, clients never see listener names and will make metadata requests exactly as before. The difference is that the list of endpoints they get back is restricted to the listener name of the endpoint where they made the request. In the example above, let's assume that all brokers are configured similarly and that a client sends a metadata request to cluster1.foo.com:9092 and it reaches broker1's 192.1.1.8:9092 interface via a load balancer. The security protocol would be SASL_PLAINTEXT and the metadata response would contain host=cluster1.foo.com,port=9092 for each broker returned.

The broker creates a socket server per listener, with the corresponding bind address. Earlier I said that I thought the broker compared the incoming host header against the advertised addresses, but that was incorrect.

@EronWright
Copy link
Contributor

EronWright commented Sep 2, 2021

I am working on a related enhancement for the Pulsar broker, and would like to help with this aspect too.

Here's some relevant information in a related design doc: Expose Brokers via Istio Gateway, and Implementation Details.

BewareMyPower pushed a commit that referenced this issue Sep 26, 2021
Fixes #669
Fixes #574

To support multiple listeners with multiple protocols, we need to set follow configration, and deprecated `kafkaAdvertisedListeners` and `kafkaListenerName`.

> kafkaListeners=internal://0.0.0.0:9092,internal_ssl://0.0.0.0:9093,external://0.0.0.0:19002,external_ssl:0.0.0.0:19003
> kafkaProtocalMap=internal:PLAINTEXT,internal_ssl:SSL,external:PLAINTEXT,external_ssl:SSL
> advertisedListeners={pulsar's listeners},internal:pulsar//192.168.1.10:9092,internal_ssl:pulsar//192.168.1.10:9093,external:pulsar//172.16.1.10:19002,external_ssl:pulsar://172.16.1.10:19003

1. Define the listenerNames and port of each listenerName in `kafkaListeners`
2. Define the listenerNames and protocol of each listenerName in `kafkaProtocalMap`
3. Add listenerNames and advertised address of each listenerName into `advertisedListeners`

Kafka client should connect to the port of the listenerName according to KIP-103.

Co-authored-by: wangjialing <wangjialing@cmss.chinamobile.com>
BewareMyPower pushed a commit that referenced this issue Sep 30, 2021
Fixes #669
Fixes #574

To support multiple listeners with multiple protocols, we need to set follow configration, and deprecated `kafkaAdvertisedListeners` and `kafkaListenerName`.

> kafkaListeners=internal://0.0.0.0:9092,internal_ssl://0.0.0.0:9093,external://0.0.0.0:19002,external_ssl:0.0.0.0:19003
> kafkaProtocalMap=internal:PLAINTEXT,internal_ssl:SSL,external:PLAINTEXT,external_ssl:SSL
> advertisedListeners={pulsar's listeners},internal:pulsar//192.168.1.10:9092,internal_ssl:pulsar//192.168.1.10:9093,external:pulsar//172.16.1.10:19002,external_ssl:pulsar://172.16.1.10:19003

1. Define the listenerNames and port of each listenerName in `kafkaListeners`
2. Define the listenerNames and protocol of each listenerName in `kafkaProtocalMap`
3. Add listenerNames and advertised address of each listenerName into `advertisedListeners`

Kafka client should connect to the port of the listenerName according to KIP-103.

Co-authored-by: wangjialing <wangjialing@cmss.chinamobile.com>
Sign up for free to subscribe to this conversation on GitHub. Already have an account? Sign in.
Labels
type/feature Indicates new functionality
Projects
None yet
Development

Successfully merging a pull request may close this issue.

2 participants