Skip to content

Add support for config entries in the topic creation API #540

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

Merged
merged 1 commit into from
Feb 19, 2018
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,8 @@ Changes and additions to the library will be listed here.

## Unreleased

- Add support for config entries in the topic creation API.

## v0.5.3

- Add support for the topic deletion API (#528).
Expand Down
10 changes: 8 additions & 2 deletions lib/kafka/client.rb
Original file line number Diff line number Diff line change
Expand Up @@ -457,10 +457,16 @@ def each_message(topic:, start_from_beginning: true, max_wait_time: 5, min_bytes
# @param replication_factor [Integer] the replication factor of the topic.
# @param timeout [Integer] a duration of time to wait for the topic to be
# completely created.
# @param config_entries [Hash] topic-level configs to use for the topic.
# See https://kafka.apache.org/documentation/#topicconfigs.
# @raise [Kafka::TopicAlreadyExists] if the topic already exists.
# @return [nil]
def create_topic(name, num_partitions: 1, replication_factor: 1, timeout: 30)
@cluster.create_topic(name, num_partitions: num_partitions, replication_factor: replication_factor, timeout: timeout)
def create_topic(name, num_partitions: 1, replication_factor: 1, timeout: 30, config_entries: {})
@cluster.create_topic(name,
num_partitions: num_partitions,
replication_factor: replication_factor,
timeout: timeout,
config_entries: config_entries)
end

# Delete a topic in the cluster.
Expand Down
3 changes: 2 additions & 1 deletion lib/kafka/cluster.rb
Original file line number Diff line number Diff line change
Expand Up @@ -156,12 +156,13 @@ def partitions_for(topic)
raise
end

def create_topic(name, num_partitions:, replication_factor:, timeout:)
def create_topic(name, num_partitions:, replication_factor:, timeout:, config_entries:)
options = {
topics: {
name => {
num_partitions: num_partitions,
replication_factor: replication_factor,
config_entries: config_entries,
}
},
timeout: timeout,
Expand Down
5 changes: 4 additions & 1 deletion lib/kafka/protocol/create_topics_request.rb
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,10 @@ def encode(encoder)
encoder.write_array([])

# Config entries. We don't care.
encoder.write_array([])
encoder.write_array(config.fetch(:config_entries)) do |config_name, config_value|
encoder.write_string(config_name)
encoder.write_string(config_value)
end
end

# Timeout is in ms.
Expand Down
14 changes: 14 additions & 0 deletions spec/functional/topic_management_spec.rb
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,20 @@
expect(partitions).to eq 3
end

example "creating a topic with config entries" do
unless kafka.supports_api?(Kafka::Protocol::DESCRIBE_CONFIGS_API)
skip("This Kafka version not support ")
end

topic = generate_topic_name
expect(kafka.topics).not_to include(topic)

kafka.create_topic(topic, num_partitions: 3, config_entries: { 'cleanup.policy' => 'compact' })

configs = kafka.describe_topic(topic, %w(cleanup.policy))
expect(configs['cleanup.policy']).to eq('compact')
end

example "deleting topics" do
topic = generate_topic_name

Expand Down
7 changes: 5 additions & 2 deletions spec/spec_helper.rb
Original file line number Diff line number Diff line change
Expand Up @@ -59,8 +59,11 @@ def create_random_topic(*args)
topic
end

def create_topic(name, num_partitions: 1, num_replicas: 1)
kafka.create_topic(name, num_partitions: num_partitions, replication_factor: num_replicas)
def create_topic(name, num_partitions: 1, num_replicas: 1, config_entries: {})
kafka.create_topic(name,
num_partitions: num_partitions,
replication_factor: num_replicas,
config_entries: config_entries)
end
end

Expand Down