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

Conversation

@BewareMyPower
Copy link
Collaborator

@BewareMyPower BewareMyPower commented Oct 28, 2021

Fixes #818

Motivation

#742 adds the multiple listeners support for KoP 2.8.x. However, there're some flaws.

First, the KoP advertised listeners must be added to advertisedListeners config. This config should be treated as the advertised listeners for broker, not protocol handlers. What KafkaProtocolHandler#getProtocolDataToAdvertise returns is not used though it has been written to ZK. It also makes kafkaAdvertisedListeners meaningless and it was marked as deprecated.

Modification

Benefit by PIP 95, Pulsar client doesn't need to configure listener name for topic lookup. So this PR simplifies the implementation of LookupClient by using a shared PulsarClient without configuring listener name.

Then this PR uses the MetadataStoreCacheLoader introduced from #820 to get the kafkaAdvertisedListeners from ZK. Since it has already been cached, this PR removes the KOP_ADDRESS_CACHE.

To verify the feature, this PR adds a test testMetadataRequestForMultiListeners. The KoP config is like

kafkaListeners=PLAINTEXT://0.0.0.0:<port1>,GW://0.0.0.0:<port2>
kafkaProtocolMap=PLAINTEXT:PLAINTEXT,GW:PLAINTEXT
kafkaAdvertisedListeners=PLAINTEXT://192.168.0.1:<port3>,GW://192.168.0.1:<port4>

And verify that each KafkaRequestHandler can handle the METADATA requests well.

@BewareMyPower BewareMyPower changed the title [WIP] Correct topic lookup implementation Support multiple listeners without configuring broker's advertisedListeners Oct 28, 2021
@BewareMyPower BewareMyPower force-pushed the bewaremypower/multi-listener branch from 323cbbe to d0b09fe Compare October 28, 2021 13:33
@BewareMyPower BewareMyPower self-assigned this Oct 28, 2021
@BewareMyPower BewareMyPower added the type/feature Indicates new functionality label Oct 28, 2021
Copy link
Contributor

@EronWright EronWright left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

All seems good to me.

Copy link
Member

@Demogorgon314 Demogorgon314 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

@BewareMyPower BewareMyPower merged commit 50699b3 into streamnative:master Oct 29, 2021
@BewareMyPower BewareMyPower deleted the bewaremypower/multi-listener branch October 29, 2021 14:03
BewareMyPower added a commit that referenced this pull request Oct 29, 2021
…teners (#864)

Fixes #818

### Motivation

#742 adds the multiple listeners support for KoP 2.8.x. However, there're some flaws. 

First, the KoP advertised listeners must be added to `advertisedListeners` config. This config should be treated as the advertised listeners for broker, not protocol handlers. What `KafkaProtocolHandler#getProtocolDataToAdvertise` returns is not used though it has been written to ZK. It also makes `kafkaAdvertisedListeners` meaningless and it was marked as deprecated.

### Modification

Benefit by [PIP 95](apache/pulsar#12040), Pulsar client doesn't need to configure listener name for topic lookup. So this PR simplifies the implementation of `LookupClient` by using a shared `PulsarClient` without configuring listener name.

Then this PR uses the `MetadataStoreCacheLoader` introduced from #820 to get the `kafkaAdvertisedListeners` from ZK. Since it has already been cached, this PR removes the `KOP_ADDRESS_CACHE`.

To verify the feature, this PR adds a test `testMetadataRequestForMultiListeners`. The KoP config is like

```properties
kafkaListeners=PLAINTEXT://0.0.0.0:<port1>,GW://0.0.0.0:<port2>
kafkaProtocolMap=PLAINTEXT:PLAINTEXT,GW:PLAINTEXT
kafkaAdvertisedListeners=PLAINTEXT://192.168.0.1:<port3>,GW://192.168.0.1:<port4>
```

And verify that each `KafkaRequestHandler` can handle the METADATA requests well.
BewareMyPower added a commit that referenced this pull request Nov 17, 2021
Cherry-picking #901 relies on some
changes of #864, while #864 cannot be
cherry-picked into branch-2.8.1 because it relies on PIP 95, which was
introduced from Pulsar 2.9.0
gaozhangmin pushed a commit to gaozhangmin/kop that referenced this pull request Nov 29, 2021
…teners (streamnative#864)

Fixes streamnative#818

### Motivation

streamnative#742 adds the multiple listeners support for KoP 2.8.x. However, there're some flaws.

First, the KoP advertised listeners must be added to `advertisedListeners` config. This config should be treated as the advertised listeners for broker, not protocol handlers. What `KafkaProtocolHandler#getProtocolDataToAdvertise` returns is not used though it has been written to ZK. It also makes `kafkaAdvertisedListeners` meaningless and it was marked as deprecated.

### Modification

Benefit by [PIP 95](apache/pulsar#12040), Pulsar client doesn't need to configure listener name for topic lookup. So this PR simplifies the implementation of `LookupClient` by using a shared `PulsarClient` without configuring listener name.

Then this PR uses the `MetadataStoreCacheLoader` introduced from streamnative#820 to get the `kafkaAdvertisedListeners` from ZK. Since it has already been cached, this PR removes the `KOP_ADDRESS_CACHE`.

To verify the feature, this PR adds a test `testMetadataRequestForMultiListeners`. The KoP config is like

```properties
kafkaListeners=PLAINTEXT://0.0.0.0:<port1>,GW://0.0.0.0:<port2>
kafkaProtocolMap=PLAINTEXT:PLAINTEXT,GW:PLAINTEXT
kafkaAdvertisedListeners=PLAINTEXT://192.168.0.1:<port3>,GW://192.168.0.1:<port4>
```

And verify that each `KafkaRequestHandler` can handle the METADATA requests well.

(cherry picked from commit 50699b3)
BewareMyPower pushed a commit that referenced this pull request Dec 3, 2021
…brokers (#941)

**Motivation**
When you enable JWT token authentication transactions do not work because transactions need broker-to-broker communication but there is currently no support for the SASL handshake.

**Changes**
Now when you enable authentication and you configure Token Authentication for Broker-To-Broker communications in Pulsar, we perform SASL PLAIN authentication while preparing the connection to other brokers.

**Tests**
I have extended existing SASL tests in order to use Transactions. I have copy/pasted some parts of the TransactionTest, but there was not enough code to make sense to share the test code.

Fix conflicts made by #864,
which changes the signature of `lookupBroker`.
BewareMyPower added a commit that referenced this pull request Feb 16, 2022
…with kafkaListeners (#1092)

This PR is a follow up bug fix of #1042 for branch-2.8.2. It also migrate some util methods and tests from #864.
BewareMyPower added a commit that referenced this pull request May 7, 2022
…espace (#1274)

### Motivation

Currently in KoP tests, the metadata namespace configured by
`kafkaMetadataNamespace` is "default", which is same with the default
namespace of Pulsar and KoP (`kafkaNamespace`).

https://github.com/streamnative/kop/blob/64001937c92d80ed4f652becd8d6b9768918b875/tests/src/test/java/io/streamnative/pulsar/handlers/kop/KopProtocolHandlerTestBase.java#L176-L177

It's a terrible config because some topic policies might be changed with
the metadata namespace and the offset topic is created during starting.
We should avoid the namespace for normal topics being affected by these
implicit operations.

### Modifications

Remove the configs for `kafkaMetadataTenant` and
`kafkaMetadataNamespace`, then fix the following tests:

- `MultiLedgerTest`: set the infinite retention to avoid rollover
  ledgers being removed automatically.
- `KafkaTopicConsumerManagerTest`: trigger the topic lookup before
  creating `PersistentTopic` objects in `BrokerService` so that broker
  can acquire the ownership of namespace bundles.
- `CacheInvalidatorTest`: unload the metadata namespace instead of the
  default namespace.

(cherry picked from commit 9e402f8)

Resolve the conflicts by printing the KOP_ADDRESS_CACHE, which was
removed in branch-2.9.x and higher since
#864.
@BewareMyPower BewareMyPower removed the doc-required This pr needs a document label Jan 10, 2023
public void testMetadataRequestForMultiListeners() throws Exception {
final Map<Integer, InetSocketAddress> bindPortToAdvertisedAddress = new HashMap<>();
final int anotherKafkaPort = PortManager.nextFreePort();
bindPortToAdvertisedAddress.put(kafkaBrokerPort,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I encountered an error Listener 'PLAINTEXT://192.168.0.1/<unresolved>:15007' is invalid when calling super.internalSetup(). Did you have the same issue when you run this test?

Sign up for free to subscribe to this conversation on GitHub. Already have an account? Sign in.

Labels

release/2.9.0 type/feature Indicates new functionality

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[FEATURE] Smart listener support (KIP-103 compatibility)

4 participants