Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Implement the static membership feature to Kafka input #135

Merged
merged 23 commits into from
Jan 25, 2023

Conversation

andsel
Copy link
Contributor

@andsel andsel commented Jan 13, 2023

Release notes

Added config group_instance_id to use the Kafka's consumer static membership feature

What does this PR do?

Static membership is reflected in the Kafka property group.instance.id, which has to be a unique identifier of the consumer instance provided by end user.
If consumer_threads settings is 1 the value is passed directly down to the Kafka's consumer configuration, but if threads count is more than 1, as per KIP-345 it would clash, so in that case, a postfix -<thread-index> is added.

Why is it important/What is the impact to the user?

The static membership feature offered by Kafka client's consumer is intended to bind a consumer to a partition, this is needed in cases where the cost of state replication between consumers during a rebalance, is high. This PR exposes the feature, which is optional, to Logstash's users.

Checklist

  • My code follows the style guidelines of this project
  • I have commented my code, particularly in hard-to-understand areas
  • I have made corresponding changes to the documentation
  • [ ] I have made corresponding change to the default configuration files (and/or docker env variables)
  • I have added tests that prove my fix is effective or that my feature works

Author's Checklist

  • test locally on rel Kafka with 1 thread and more threads
  • test locally with another client that kicks off the Kafka input plugin
  • obtain a docs review.

How to test this PR locally

Runs a local Kafka cluster

Fro the clone of this repository launch the test Kafka script

./kafka_test_setup.sh

Connect a producer

  • start the producer with
bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic "logstash_integration_static_membership_topic"

Setup Logstash Kafka input & run

  • Adds the plugin definition in Gemfile
gem "logstash-integration-kafka", :path => "/path/to/logstash_plugins/logstash-integration-kafka"
  • Install the plugin in development mode
bin/logstash-plugin install --no-verify
  • edit the pipeline test_pipeline.conf file
input {
  kafka {
    bootstrap_servers => ["localhost:9092"]
    topics => ["logstash_integration_static_membership_topic"]
    group_id => "logstash"
    consumer_threads => 1

    group_instance_id => "test_static_group_id"
  }
}


output {
  stdout {
    codec => rubydebug
  }
}

Verify Logstash is receiving data

From the producer's console send some data and verify on the Logstash console the message is received.

Verify another consumer kicks off Logstash

Start another consumer with same group.instance.id in same consumer group

  • In the folder build/kafka the file client_config.properties` with content
group.instance.id=test_static_group_id
bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic "logstash_integration_static_membership_topic" --from-beginning --group logstash --consumer.config "${PWD}/client_config.properties"

Test with multiple threads

  • Edit the pipeline to have consumer_threads > 1 and verify that another consumer doesn't kicks off because the group.instance.id has now been suffixed with -n for each Logstash Kafka consumer threads.

Related issues

Use cases

As a Kafka uses that want to avoid that consumers spend a lot of time during rebalances, specially when they have heavy state, I want to be able to assign a "static membership" id to every consumer.

@andsel andsel self-assigned this Jan 13, 2023
@andsel andsel changed the title Implement the static membership feature cotnained in KIP-345 Implement the static membership feature to Kafka input Jan 18, 2023
@andsel andsel added the enhancement New feature or request label Jan 18, 2023
…all the clients belogning to the same consumer group
@andsel andsel marked this pull request as ready for review January 18, 2023 10:35
Comment on lines 210 to 229
it "input plugin disconnects from the broker when another client with same static membership connects" do
queue = java.util.concurrent.ArrayBlockingQueue.new(10)
kafka_input = LogStash::Inputs::Kafka.new(consumer_config)
kafka_input.register

expect(logger).to receive(:error).with("Another consumer with same group.instance.id has connected")

input_worker = java.lang.Thread.new { kafka_input.run(queue) }
begin
input_worker.start
wait_kafka_input_is_ready("logstash_integration_static_membership_topic", queue)
saboteur_kafka_consumer = create_consumer_and_start_consuming("test_static_group_id")
saboteur_kafka_consumer.run # ask to be scheduled
saboteur_kafka_consumer.join

expect(saboteur_kafka_consumer.value).to eq("saboteur exited")
ensure
input_worker.join(30_000)
end
end
Copy link
Member

Choose a reason for hiding this comment

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

We can move some things outside of the test by introducing a context:

Suggested change
it "input plugin disconnects from the broker when another client with same static membership connects" do
queue = java.util.concurrent.ArrayBlockingQueue.new(10)
kafka_input = LogStash::Inputs::Kafka.new(consumer_config)
kafka_input.register
expect(logger).to receive(:error).with("Another consumer with same group.instance.id has connected")
input_worker = java.lang.Thread.new { kafka_input.run(queue) }
begin
input_worker.start
wait_kafka_input_is_ready("logstash_integration_static_membership_topic", queue)
saboteur_kafka_consumer = create_consumer_and_start_consuming("test_static_group_id")
saboteur_kafka_consumer.run # ask to be scheduled
saboteur_kafka_consumer.join
expect(saboteur_kafka_consumer.value).to eq("saboteur exited")
ensure
input_worker.join(30_000)
end
end
context "when another client with same static membership connects" do
let(:queue) { java.util.concurrent.ArrayBlockingQueue.new(10) }
let(:kafka_input) { LogStash::Inputs::Kafka.new(consumer_config) }
before(:each) { kafka_input.register }
it "input plugin disconnects from the broker" do
expect(logger).to receive(:error).with("Another consumer with same group.instance.id has connected")
input_worker = java.lang.Thread.new { kafka_input.run(queue) }
begin
input_worker.start
wait_kafka_input_is_ready("logstash_integration_static_membership_topic", queue)
saboteur_kafka_consumer = create_consumer_and_start_consuming("test_static_group_id")
saboteur_kafka_consumer.run # ask to be scheduled
saboteur_kafka_consumer.join
expect(saboteur_kafka_consumer.value).to eq("saboteur exited")
ensure
input_worker.join(30_000)
end
end
end

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Nice suggestion thanks, integrated with commit 97f35b3.

@andsel andsel requested a review from jsvd January 18, 2023 14:07
Co-authored-by: João Duarte <[email protected]>
Copy link
Member

@jsvd jsvd left a comment

Choose a reason for hiding this comment

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

LGTM

@roaksoax roaksoax requested a review from karenzone January 18, 2023 15:52
Copy link
Contributor

@karenzone karenzone left a comment

Choose a reason for hiding this comment

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

I left some comments for your consideration. In places, I made some guesses rather than asking questions to help us reduce the back-and-forth of multiple review cycles. Please keep me honest, and let me know if you'd like to discuss. :-)

Comment on lines 362 to 363
NOTE: It has to be unique across all the clients belonging to the same <<plugins-{type}s-{plugin}-group_id>>, in case another client connects
with same `group.instance.id` value then the oldest is kicked off.
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
NOTE: It has to be unique across all the clients belonging to the same <<plugins-{type}s-{plugin}-group_id>>, in case another client connects
with same `group.instance.id` value then the oldest is kicked off.
NOTE: The `group_instance_id` setting must be unique across all the clients belonging to the same <<plugins-{type}s-{plugin}-group_id>>. Otherwise, another client connecting
with same `group.instance.id` value would cause the oldest instance to be removed.

Copy link
Contributor

Choose a reason for hiding this comment

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

I'm not sure I got all the details right. I'm trying to add precise language by restating the name of the option rather than just saying "it".
Also, "kicked off" is ambiguous in English. It can mean to start something (like kicking off a process" or deleting/removing something. I'm guessing you mean the latter.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes you guessed right :-) I thinks that instead of

instance to be removed

would be more appropriate

instance to be disconnected

Copy link
Contributor

@karenzone karenzone left a comment

Choose a reason for hiding this comment

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

LGTM!

@andsel andsel merged commit 7f34902 into logstash-plugins:main Jan 25, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
enhancement New feature or request
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Add static membership Kafka feature to clients
4 participants