From a88ea57b7ad5e0310c5a5e579c577adaadc930c3 Mon Sep 17 00:00:00 2001 From: David Cuddeback Date: Thu, 2 Feb 2017 16:56:06 -0800 Subject: [PATCH 01/47] implements consumer groups --- lib/kafka_ex.ex | 50 ++- lib/kafka_ex/consumer_group.ex | 309 ++++++++++++++ lib/kafka_ex/gen_consumer.ex | 474 ++++++++++++++++++++++ lib/kafka_ex/protocol/heartbeat.ex | 22 +- lib/kafka_ex/protocol/join_group.ex | 14 +- lib/kafka_ex/protocol/leave_group.ex | 21 +- lib/kafka_ex/protocol/sync_group.ex | 37 +- lib/kafka_ex/server.ex | 32 +- lib/kafka_ex/server_0_p_8_p_0.ex | 8 +- lib/kafka_ex/server_0_p_8_p_2.ex | 8 +- lib/kafka_ex/server_0_p_9_p_0.ex | 32 +- test/integration/server0_p_9_p_0_test.exs | 78 +++- test/protocol/heartbeat_test.exs | 8 +- test/protocol/join_group_test.exs | 4 +- test/protocol/leave_group_test.exs | 7 +- test/protocol/sync_group_test.exs | 20 +- 16 files changed, 1036 insertions(+), 88 deletions(-) create mode 100644 lib/kafka_ex/consumer_group.ex create mode 100644 lib/kafka_ex/gen_consumer.ex diff --git a/lib/kafka_ex.ex b/lib/kafka_ex.ex index ea6241a0..24f1b29a 100644 --- a/lib/kafka_ex.ex +++ b/lib/kafka_ex.ex @@ -6,6 +6,12 @@ defmodule KafkaEx do alias KafkaEx.Protocol.ConsumerMetadata.Response, as: ConsumerMetadataResponse alias KafkaEx.Protocol.Fetch.Response, as: FetchResponse alias KafkaEx.Protocol.Fetch.Request, as: FetchRequest + alias KafkaEx.Protocol.Heartbeat.Request, as: HeartbeatRequest + alias KafkaEx.Protocol.Heartbeat.Response, as: HeartbeatResponse + alias KafkaEx.Protocol.JoinGroup.Request, as: JoinGroupRequest + alias KafkaEx.Protocol.JoinGroup.Response, as: JoinGroupResponse + alias KafkaEx.Protocol.LeaveGroup.Request, as: LeaveGroupRequest + alias KafkaEx.Protocol.LeaveGroup.Response, as: LeaveGroupResponse alias KafkaEx.Protocol.Metadata.Response, as: MetadataResponse alias KafkaEx.Protocol.Offset.Response, as: OffsetResponse alias KafkaEx.Protocol.OffsetCommit.Request, as: OffsetCommitRequest @@ -14,6 +20,8 @@ defmodule KafkaEx do alias KafkaEx.Protocol.OffsetFetch.Request, as: OffsetFetchRequest alias KafkaEx.Protocol.Produce.Request, as: ProduceRequest alias KafkaEx.Protocol.Produce.Message + alias KafkaEx.Protocol.SyncGroup.Request, as: SyncGroupRequest + alias KafkaEx.Protocol.SyncGroup.Response, as: SyncGroupResponse alias KafkaEx.Server @type uri() :: [{binary|char_list, number}] @@ -74,6 +82,46 @@ defmodule KafkaEx do Server.call(worker, :consumer_group) end + @doc """ + Sends a request to join a consumer group. + """ + @spec join_group(JoinGroupRequest.t, Keyword.t) :: JoinGroupResponse.t + def join_group(request, opts \\ []) do + worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + timeout = Keyword.get(opts, :timeout) + Server.call(worker_name, {:join_group, request, timeout}, opts) + end + + @doc """ + Sends a request to synchronize with a consumer group. + """ + @spec sync_group(SyncGroupRequest.t, Keyword.t) :: SyncGroupResponse.t + def sync_group(request, opts \\ []) do + worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + timeout = Keyword.get(opts, :timeout) + Server.call(worker_name, {:sync_group, request, timeout}, opts) + end + + @doc """ + Sends a request to leave a consumer group. + """ + @spec leave_group(LeaveGroupRequest.t, Keyword.t) :: LeaveGroupResponse.t + def leave_group(request, opts \\ []) do + worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + timeout = Keyword.get(opts, :timeout) + Server.call(worker_name, {:leave_group, request, timeout}, opts) + end + + @doc """ + Sends a heartbeat to maintain membership in a consumer group. + """ + @spec heartbeat(HeartbeatRequest.t, Keyword.t) :: HeartbeatResponse.t + def heartbeat(request, opts \\ []) do + worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + timeout = Keyword.get(opts, :timeout) + Server.call(worker_name, {:heartbeat, request, timeout}, opts) + end + @doc """ Return metadata for the given topic; returns for all topics if topic is empty string @@ -196,7 +244,7 @@ defmodule KafkaEx do }, opts) end - @spec offset_commit(atom, OffsetCommitRequest.t) :: OffsetCommitResponse.t + @spec offset_commit(atom, OffsetCommitRequest.t) :: [OffsetCommitResponse.t] def offset_commit(worker_name, offset_commit_request) do Server.call(worker_name, {:offset_commit, offset_commit_request}) end diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex new file mode 100644 index 00000000..4115a7b7 --- /dev/null +++ b/lib/kafka_ex/consumer_group.ex @@ -0,0 +1,309 @@ +defmodule KafkaEx.ConsumerGroup do + @moduledoc """ + A process that manages membership in a Kafka consumer group. + + Consumers in a consumer group coordinate with each other through a Kafka broker to distribute the + work of consuming one or several topics without any overlap. This is facilitated by the [Kafka + client-side assignment + protocol](https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Client-side+Assignment+Proposal). + + Any time group membership changes (a member joins or leaves the group), a Kafka broker initiates + group synchronization by asking one of the group members (the leader) to provide partition + assignments for the whole group. Partition assignment is handled by the + `c:KafkaEx.GenConsumer.assign_partitions/2` callback of the provided consumer module. + + A `ConsumerGroup` process is responsible for: + + 1. Maintaining membership in a Kafka consumer group. + 2. Determining partition assignments if elected as the group leader. + 3. Launching and terminating `GenConsumer` processes based on its assigned partitions. + + To use a `ConsumerGroup`, a developer must define a module that implements the + `KafkaEx.GenConsumer` behaviour and start a `ConsumerGroup` with that module. + + ## Example + + The following consumer prints each message with the name of the node that's consuming the message: + + ``` + defmodule DistributedConsumer do + use KafkaEx.GenConsumer + + def handle_message(%Message{value: message}, state) do + IO.puts(to_string(node()) <> ": " <> inspect(message)) + {:ack, state} + end + end + + # use DistributedConsumer in a consumer group + {:ok, pid} = KafkaEx.ConsumerGroup.start_link(DistributedConsumer, "test_group", ["test_topic"]) + ``` + + Running this on multiple nodes might display the following: + + ```txt + node1@host: "messages" + node2@host: "on" + node2@host: "multiple" + node1@host: "nodes" + ``` + + It is not necessary for the nodes to be connected, because `ConsumerGroup` uses Kafka's built-in + group coordination protocol. + """ + + use GenServer + + alias KafkaEx.Config + alias KafkaEx.Protocol.JoinGroup.Request, as: JoinGroupRequest + alias KafkaEx.Protocol.Heartbeat.Request, as: HeartbeatRequest + alias KafkaEx.Protocol.Heartbeat.Response, as: HeartbeatResponse + alias KafkaEx.Protocol.LeaveGroup.Request, as: LeaveGroupRequest + alias KafkaEx.Protocol.LeaveGroup.Response, as: LeaveGroupResponse + alias KafkaEx.Protocol.Metadata.Response, as: MetadataResponse + alias KafkaEx.Protocol.Metadata.TopicMetadata + alias KafkaEx.Protocol.Metadata.PartitionMetadata + alias KafkaEx.Protocol.SyncGroup.Request, as: SyncGroupRequest + alias KafkaEx.Protocol.SyncGroup.Response, as: SyncGroupResponse + + require Logger + + defmodule State do + @moduledoc false + defstruct [ + :worker_name, + :heartbeat_interval, + :session_timeout, + :consumer_module, + :consumer_opts, + :group_name, + :topics, + :partitions, + :member_id, + :generation_id, + :assignments, + :consumer_pid, + ] + end + + @heartbeat_interval 5_000 + @session_timeout 30_000 + + # Client API + + @doc """ + Starts a `ConsumerGroup` process linked to the current process. + + This can be used to start a `ConsumerGroup` as part of a supervision tree. + + `module` is a module that implements the `KafkaEx.GenConsumer` behaviour. `group_name` is the name + of the consumer group. `topics` is a list of topics that the consumer group should consume from. + `opts` can be any options accepted by `GenConsumer` or `GenServer`. + + ### Return Values + + This function has the same return values as `GenServer.start_link/3`. + + If the consumer group is successfully created and initialized, this function returns `{:ok, pid}`, + where `pid` is the PID of the consumer group process. + """ + @spec start_link(module, binary, [binary], KafkaEx.GenConsumer.options) :: GenServer.on_start + def start_link(consumer_module, group_name, topics, opts \\ []) do + {server_opts, consumer_opts} = Keyword.split(opts, [:debug, :name, :timeout, :spawn_opt]) + + GenServer.start_link(__MODULE__, {consumer_module, group_name, topics, consumer_opts}, server_opts) + end + + # GenServer callbacks + + def init({consumer_module, group_name, topics, opts}) do + worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + heartbeat_interval = Keyword.get(opts, :heartbeat_interval, Application.get_env(:kafka_ex, :heartbeat_interval, @heartbeat_interval)) + session_timeout = Keyword.get(opts, :session_timeout, Application.get_env(:kafka_ex, :session_timeout, @session_timeout)) + + consumer_opts = Keyword.drop(opts, [:heartbeat_interval, :session_timeout]) + + state = %State{ + worker_name: worker_name, + heartbeat_interval: heartbeat_interval, + session_timeout: session_timeout, + consumer_module: consumer_module, + consumer_opts: consumer_opts, + group_name: group_name, + topics: topics, + member_id: "", + } + + Process.flag(:trap_exit, true) + + {:ok, state, 0} + end + + def handle_info(:timeout, %State{generation_id: nil, member_id: ""} = state) do + new_state = join(state) + + {:noreply, new_state, new_state.heartbeat_interval} + end + + def handle_info(:timeout, %State{} = state) do + new_state = heartbeat(state) + + {:noreply, new_state, new_state.heartbeat_interval} + end + + def handle_info({:EXIT, pid, reason}, %State{consumer_pid: pid} = state) do + new_state = %State{state | consumer_pid: nil} + + {:stop, reason, new_state} + end + + def handle_info({:EXIT, _pid, _reason}, %State{} = state) do + {:noreply, state, state.heartbeat_interval} + end + + def terminate(_reason, %State{} = state) do + leave(state) + end + + # Helpers + + defp join(%State{worker_name: worker_name, session_timeout: session_timeout, group_name: group_name, topics: topics, member_id: member_id} = state) do + join_request = %JoinGroupRequest{ + group_name: group_name, + member_id: member_id, + topics: topics, + session_timeout: session_timeout, + } + + join_response = KafkaEx.join_group(join_request, worker_name: worker_name, timeout: session_timeout + 5000) + new_state = %State{state | member_id: join_response.member_id, generation_id: join_response.generation_id} + + Logger.debug("Joined consumer group #{group_name}") + + if join_response.member_id == join_response.leader_id do + sync_leader(new_state, join_response.members) + else + sync_follower(new_state) + end + end + + defp sync_leader(%State{worker_name: worker_name, topics: topics, partitions: nil} = state, members) do + %MetadataResponse{topic_metadatas: topic_metadatas} = KafkaEx.metadata(worker_name: worker_name) + + partitions = Enum.flat_map(topics, fn (topic) -> + %TopicMetadata{error_code: :no_error, partition_metadatas: partition_metadatas} = Enum.find(topic_metadatas, &(&1.topic == topic)) + + Enum.map(partition_metadatas, fn (%PartitionMetadata{error_code: :no_error, partition_id: partition_id}) -> + {topic, partition_id} + end) + end) + + sync_leader(%State{state | partitions: partitions}, members) + end + + defp sync_leader(%State{worker_name: worker_name, session_timeout: session_timeout, + group_name: group_name, generation_id: generation_id, member_id: member_id} = state, members) do + assignments = assign_partitions(state, members) + + sync_request = %SyncGroupRequest{ + group_name: group_name, + member_id: member_id, + generation_id: generation_id, + assignments: assignments, + } + + sync_request + |> KafkaEx.sync_group(worker_name: worker_name, timeout: session_timeout + 5000) + |> update_assignments(state) + end + + defp sync_follower(%State{worker_name: worker_name, session_timeout: session_timeout, + group_name: group_name, generation_id: generation_id, member_id: member_id} = state) do + sync_request = %SyncGroupRequest{ + group_name: group_name, + member_id: member_id, + generation_id: generation_id, + assignments: [], + } + + sync_request + |> KafkaEx.sync_group(timeout: session_timeout + 5000, worker_name: worker_name) + |> update_assignments(state) + end + + defp update_assignments(%SyncGroupResponse{error_code: :rebalance_in_progress}, %State{} = state), do: rebalance(state) + defp update_assignments(%SyncGroupResponse{error_code: :no_error, assignments: assignments}, %State{} = state) do + start_consumer(state, assignments) + end + + defp heartbeat(%State{worker_name: worker_name, group_name: group_name, generation_id: generation_id, member_id: member_id} = state) do + heartbeat_request = %HeartbeatRequest{ + group_name: group_name, + member_id: member_id, + generation_id: generation_id, + } + + case KafkaEx.heartbeat(heartbeat_request, worker_name: worker_name) do + %HeartbeatResponse{error_code: :no_error} -> + state + + %HeartbeatResponse{error_code: :rebalance_in_progress} -> + rebalance(state) + end + end + + defp rebalance(%State{} = state) do + state + |> stop_consumer() + |> join() + end + + defp leave(%State{worker_name: worker_name, group_name: group_name, member_id: member_id} = state) do + stop_consumer(state) + + leave_request = %LeaveGroupRequest{ + group_name: group_name, + member_id: member_id, + } + + %LeaveGroupResponse{error_code: :no_error} = KafkaEx.leave_group(leave_request, worker_name: worker_name) + + Logger.debug("Left consumer group #{group_name}") + end + + defp start_consumer(%State{consumer_module: consumer_module, consumer_opts: consumer_opts, + group_name: group_name, consumer_pid: nil} = state, assignments) do + assignments = + Enum.flat_map(assignments, fn ({topic, partition_ids}) -> + Enum.map(partition_ids, &({topic, &1})) + end) + + {:ok, pid} = KafkaEx.GenConsumer.Supervisor.start_link(consumer_module, group_name, assignments, consumer_opts) + + %State{state | assignments: assignments, consumer_pid: pid} + end + + defp stop_consumer(%State{consumer_pid: nil} = state), do: state + defp stop_consumer(%State{consumer_pid: pid} = state) when is_pid(pid) do + :ok = Supervisor.stop(pid) + %State{state | consumer_pid: nil} + end + + defp assign_partitions(%State{consumer_module: consumer_module, partitions: partitions}, members) do + assignments = + consumer_module.assign_partitions(members, partitions) + |> Enum.map(fn ({member, topic_partitions}) -> + assigns = + topic_partitions + |> Enum.group_by(&(elem(&1, 0)), &(elem(&1, 1))) + |> Enum.into([]) + + {member, assigns} + end) + |> Map.new + + Enum.map(members, fn (member) -> + {member, Map.get(assignments, member, [])} + end) + end +end diff --git a/lib/kafka_ex/gen_consumer.ex b/lib/kafka_ex/gen_consumer.ex new file mode 100644 index 00000000..2f1c4ced --- /dev/null +++ b/lib/kafka_ex/gen_consumer.ex @@ -0,0 +1,474 @@ +defmodule KafkaEx.GenConsumer do + @moduledoc """ + A behaviour module for implementing a Kafka consumer. + + A `GenConsumer` is an Elixir process that consumes messages from Kafka. A single `GenConsumer` + process consumes from a single partition of a Kafka topic. Several `GenConsumer` processes can be + used to consume from multiple partitions or even multiple topics. Partition assignments for a + group of `GenConsumer`s can be defined manually using `KafkaEx.GenConsumer.Supervisor` or + coordinated across a cluster of nodes using `KafkaEx.ConsumerGroup`. + + ## Example + + The `GenConsumer` behaviour abstracts common Kafka consumer interactions. `GenConsumer` will take + care of the details of determining a starting offset, fetching messages from a Kafka broker, and + committing offsets for consumed messages. Developers are only required to implement + `c:handle_message/2` to process messages from the queue. + + The following is a minimal example that logs each message as it's consumed: + + ``` + defmodule ExampleGenConsumer do + use KafkaEx.GenConsumer + + require Logger + + def handle_message(%Message{value: message}, state) do + Logger.debug("message: " <> inspect(message)) + {:ack, state} + end + end + ``` + + `c:handle_message/2` will be called for each message that's fetched from a Kafka broker. In this + example, since `c:handle_message/2` always returns `{:ack, new_state}`, the message offsets will + be auto-committed. + + ## Auto-Committing Offsets + + `GenConsumer` manages a consumer's offsets by committing the offsets of acknowledged messages. + Messages are acknowledged by returning `{:ack, new_state}` from `c:handle_message/2`. + Acknowledged messages are not committed immediately. To avoid excessive network calls, + acknowledged messages may be batched and committed periodically. Offsets are also committed when a + `GenServer` is terminated. + + How often a `GenConsumer` auto-commits offsets is controlled by the two configuration values + `:commit_interval` and `:commit_threshold`. These can be set globally in the `:kafka_ex` app's + environment or on a per-consumer basis by passing options to `start_link/5`: + + ``` + # In config/config.exs + config :kafka_ex, + commit_interval: 5000, + commit_threshold: 100 + + # As options to start_link/5 + KafkaEx.GenConsumer.start_link(MyConsumer, "my_group", "topic", 0, + commit_interval: 5000, + commit_threshold: 100) + ``` + + * `:commit_interval` is the maximum time (in milliseconds) that a `GenConsumer` will delay + committing the offset for an acknowledged message. + * `:commit_threshold` is the maximum number of acknowledged messages that a `GenConsumer` will + allow to be uncommitted before triggering an auto-commit. + + For low-volume topics, `:commit_interval` is the dominant factor for how often a `GenConsumer` + auto-commits. For high-volume topics, `:commit_threshold` is the dominant factor. + + ## Callbacks + + There are three callbacks that are required to be implemented in a `GenConsumer`. By adding `use + KafkaEx.GenServer` to a module, two of the callbacks will be defined with default behavior, + leaving you to implement `c:handle_message/2`. + + ## Integration with OTP + + A `GenConsumer` is a specialized `GenServer`. It can be supervised, registered, and debugged the + same as any other `GenServer`. However, its arguments for `c:GenServer.init/1` are unspecified, so + `start_link/5` should be used to start a `GenConsumer` process instead of `GenServer` primitives. + + ## Testing + + A `GenConsumer` can be unit-tested without a running Kafka broker by sending messages directly to + its `c:handle_message/2` function. The following recipe can be used as a starting point when + testing a `GenConsumer`: + + ``` + defmodule ExampleGenConsumerTest do + use ExUnit.Case, async: true + + alias KafkaEx.Protocol.Fetch.Message + + @topic "topic" + @partition 0 + + setup do + {:ok, state} = ExampleGenConsumer.init(@topic, @partition) + {:ok, %{state: state}} + end + + test "it acks a message", %{state: state} do + message = %Message{offset: 0, value: "hello"} + {response, _new_state} = ExampleGenConsumer.handle_message(message, state) + assert response == :ack + end + end + ``` + """ + + use GenServer + + alias KafkaEx.Config + alias KafkaEx.Protocol.OffsetCommit.Request, as: OffsetCommitRequest + alias KafkaEx.Protocol.OffsetCommit.Response, as: OffsetCommitResponse + alias KafkaEx.Protocol.OffsetFetch.Request, as: OffsetFetchRequest + alias KafkaEx.Protocol.OffsetFetch.Response, as: OffsetFetchResponse + alias KafkaEx.Protocol.Offset.Response, as: OffsetResponse + alias KafkaEx.Protocol.Fetch.Response, as: FetchResponse + alias KafkaEx.Protocol.Fetch.Message + + require Logger + + @typedoc """ + The ID of a member of a consumer group, assigned by a Kafka broker. + """ + @type member_id :: binary + + @typedoc """ + The name of a Kafka topic. + """ + @type topic :: binary + + @typedoc """ + The ID of a partition of a Kafka topic. + """ + @type partition_id :: integer + + @typedoc """ + A partition of a particular Kafka topic. + """ + @type partition :: {topic, partition_id} + + @typedoc """ + Option values used when starting a `GenConsumer`. + """ + @type option :: {:worker_name, atom | pid} + | {:commit_interval, non_neg_integer} + | {:commit_threshold, non_neg_integer} + + @typedoc """ + Options used when starting a `GenConsumer`. + """ + @type options :: [option | GenServer.option] + + @doc """ + Invoked when the server is started. `start_link/5` will block until it returns. + + `topic` and `partition` are the arguments passed to `start_link/5`. They identify the Kafka + partition that the `GenConsumer` will consume from. + + Returning `{:ok, state}` will cause `start_link/5` to return `{:ok, pid}` and the process to start + consuming from its assigned partition. `state` becomes the consumer's state. + + Any other return value will cause the `start_link/5` to return `{:error, error}` and the process + to exit. + """ + @callback init(topic :: topic, partition :: partition_id) :: {:ok, state :: term} + + @doc """ + Invoked for each message consumed from a Kafka queue. + + `message` is a message fetched from a Kafka broker and `state` is the current state of the + `GenConsumer`. + + Returning `{:ack, new_state}` acknowledges `message` and continues to consume from the Kafka queue + with new state `new_state`. Acknowledged messages will be auto-committed (possibly at a later + time) based on the `:commit_interval` and `:commit_threshold` options. + + Returning `{:commit, new_state}` commits `message` synchronously before continuing to consume from + the Kafka queue with new state `new_state`. Committing a message synchronously means that no more + messages will be consumed until the message's offset is committed. `:commit` should be used + sparingly, since committing every message synchronously would impact a consumer's performance and + could result in excessive network traffic. + """ + @callback handle_message(message :: Message.t, state :: term) :: {:ack, new_state :: term} + | {:commit, new_state :: term} + + @doc """ + Invoked to determine partition assignments for a coordinated consumer group. + + `members` is a list of member IDs and `partitions` is a list of partitions that need to be + assigned to a group member. + + The return value must be a map with member IDs as keys and a list of partition assignments as + values. For each member ID in the returned map, the assigned partitions will become the + `assignments` argument to `KafkaEx.GenConsumer.Supervisor.start_link/4` in the corresponding + member process. Any member that's omitted from the return value will not be assigned any + partitions. + + If this callback is not implemented, the default implementation by `use KafkaEx.GenConsumer` + implements a simple round-robin assignment. + + ### Example + + Given the following `members` and `partitions` to be assigned: + + ``` + members = ["member1", "member2", "member3"] + partitions = [{"topic", 0}, {"topic", 1}, {"topic", 2}] + ``` + + One possible assignment is as follows: + + ``` + ExampleGenConsumer.assign_partitions(members, partitions) + #=> %{"member1" => [{"topic", 0}, {"topic", 2}], "member2" => [{"topic", 1}]} + ``` + + In this case, the consumer group process for `"member1"` will launch two `GenConsumer` processes + (one for each of its assigned partitions), `"member2"` will launch one `GenConsumer` process, and + `"member3"` will launch no processes. + """ + @callback assign_partitions(members :: [member_id], partitions :: [partition]) :: %{member_id => [partition]} + + defmacro __using__(_opts) do + quote do + @behaviour KafkaEx.GenConsumer + alias KafkaEx.Protocol.Fetch.Message + + def init(_topic, _partition) do + {:ok, nil} + end + + def assign_partitions(members, partitions) do + Stream.cycle(members) + |> Enum.zip(partitions) + |> Enum.group_by(&(elem(&1, 0)), &(elem(&1, 1))) + end + + defoverridable [init: 2, assign_partitions: 2] + end + end + + defmodule Supervisor do + @moduledoc """ + A supervisor for managing `GenConsumer` processes that are part of a consumer group. + + The supervisor will launch individual `GenConsumer` processes for each partition given by the + `partitions` argument to `start_link/4`. When terminated, each of the supervisor's child + processes will commit its latest offset before terminating. + + This module manages a static list of consumer processes. For dynamically distributing consumers + in a consumer group across a cluster of nodes, see `KafkaEx.ConsumerGroup`. + """ + + use Elixir.Supervisor + + @doc """ + Starts a `GenConsumer.Supervisor` process linked to the current process. + + `module` is a module that implements the `GenConsumer` behaviour. `group_name` is the name of a + consumer group, and `assignments` is a list of partitions for the `GenConsumer`s to consume. + `opts` accepts the same options as `KafkaEx.GenConsumer.start_link/5`. + + ### Return Values + + This function has the same return values as `Supervisor.start_link/3`. + + If the supervisor and its consumers are successfully created, this function returns `{:ok, + pid}`, where `pid` is the PID of the supervisor. + """ + @spec start_link(module, binary, [KafkaEx.GenConsumer.partition], KafkaEx.GenConsumer.options) :: Elixir.Supervisor.on_start + def start_link(consumer_module, group_name, assignments, opts \\ []) do + case Elixir.Supervisor.start_link(__MODULE__, {consumer_module, group_name, assignments, opts}) do + {:ok, pid} -> + Enum.each(assignments, fn ({topic, partition}) -> + case Elixir.Supervisor.start_child(pid, [topic, partition, opts]) do + {:ok, _child} -> nil + {:ok, _child, _info} -> nil + end + end) + + {:ok, pid} + + error -> + error + end + end + + def init({consumer_module, group_name, _assignments, _opts}) do + children = [ + worker(KafkaEx.GenConsumer, [consumer_module, group_name]) + ] + + supervise(children, strategy: :simple_one_for_one) + end + end + + defmodule State do + @moduledoc false + defstruct [ + :consumer_module, + :consumer_state, + :commit_interval, + :commit_threshold, + :worker_name, + :group, + :topic, + :partition, + :current_offset, + :committed_offset, + :acked_offset, + :last_commit, + ] + end + + @commit_interval 5_000 + @commit_threshold 100 + + # Client API + + @doc """ + Starts a `GenConsumer` process linked to the current process. + + This can be used to start the `GenConsumer` as part of a supervision tree. + + Once the consumer has been started, the `c:init/2` function of the given `consumer_module` is + called with the given `topic` and `partition`. `group_name` is the consumer group name that will + be used for managing consumer offsets. + + ### Options + + * `:commit_interval` - the interval in milliseconds that the consumer will wait to commit + acknowledged messages. If not present, the `:commit_interval` environment value is used. + * `:commit_threshold` - the maximum number of messages that can be acknowledged without being + committed. If not present, the `:commit_threshold` environment value is used. + * `:worker_name` - the name of the `KafkaEx.Server` process to use for communicating with the + Kafka brokers. If not present, the default worker is used. + + Any valid options for `GenServer.start_link/3` can also be specified. + + ### Return Values + + This function has the same return values as `GenServer.start_link/3`. + + If the consumer is successfully created and initialized, this function returns `{:ok, pid}`, where + `pid` is the PID of the consumer process. + """ + @spec start_link(module, binary, topic, partition_id, options) :: GenServer.on_start + def start_link(consumer_module, group_name, topic, partition, opts \\ []) do + {server_opts, consumer_opts} = Keyword.split(opts, [:debug, :name, :timeout, :spawn_opt]) + + GenServer.start_link(__MODULE__, {consumer_module, group_name, topic, partition, consumer_opts}, server_opts) + end + + # GenServer callbacks + + def init({consumer_module, group_name, topic, partition, opts}) do + worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + commit_interval = Keyword.get(opts, :commit_interval, Application.get_env(:kafka_ex, :commit_interval, @commit_interval)) + commit_threshold = Keyword.get(opts, :commit_threshold, Application.get_env(:kafka_ex, :commit_threshold, @commit_threshold)) + + {:ok, consumer_state} = consumer_module.init(topic, partition) + + state = %State{ + consumer_module: consumer_module, + consumer_state: consumer_state, + commit_interval: commit_interval, + commit_threshold: commit_threshold, + worker_name: worker_name, + group: group_name, + topic: topic, + partition: partition, + } + + Process.flag(:trap_exit, true) + + {:ok, state, 0} + end + + def handle_info(:timeout, %State{current_offset: nil, last_commit: nil} = state) do + new_state = %State{load_offsets(state) | last_commit: :erlang.monotonic_time(:milli_seconds)} + + {:noreply, new_state, 0} + end + + def handle_info(:timeout, %State{} = state) do + new_state = consume(state) + + {:noreply, new_state, 0} + end + + def terminate(_reason, %State{} = state) do + commit(state) + end + + # Helpers + + defp consume(%State{worker_name: worker_name, topic: topic, partition: partition, current_offset: offset} = state) do + [%FetchResponse{topic: ^topic, partitions: [response = %{error_code: :no_error, partition: ^partition}]}] = + KafkaEx.fetch(topic, partition, offset: offset, auto_commit: false, worker_name: worker_name) + + case response do + %{last_offset: nil, message_set: []} -> + auto_commit(state) + + %{last_offset: _, message_set: messages} -> + Enum.reduce(messages, state, &handle_message/2) + end + end + + defp handle_message(%Message{offset: offset} = message, %State{consumer_module: consumer_module, consumer_state: consumer_state} = state) do + case consumer_module.handle_message(message, consumer_state) do + {:ack, new_state} -> + auto_commit %State{state | consumer_state: new_state, acked_offset: offset + 1, current_offset: offset + 1} + + {:commit, new_state} -> + commit %State{state | consumer_state: new_state, acked_offset: offset + 1, current_offset: offset + 1} + end + end + + defp auto_commit(%State{acked_offset: acked, committed_offset: committed, commit_threshold: threshold, + last_commit: last_commit, commit_interval: interval} = state) do + case acked - committed do + 0 -> + %State{state | last_commit: :erlang.monotonic_time(:milli_seconds)} + + n when n >= threshold -> + commit(state) + + _ -> + if :erlang.monotonic_time(:milli_seconds) - last_commit >= interval do + commit(state) + else + state + end + end + end + + defp commit(%State{acked_offset: offset, committed_offset: offset} = state), do: state + defp commit(%State{worker_name: worker_name, group: group, topic: topic, partition: partition, acked_offset: offset} = state) do + request = %OffsetCommitRequest{ + consumer_group: group, + topic: topic, + partition: partition, + offset: offset, + } + + [%OffsetCommitResponse{topic: ^topic, partitions: [^partition]}] = + KafkaEx.offset_commit(worker_name, request) + + Logger.debug("Committed offset #{topic}/#{partition}@#{offset} for #{group}") + + %State{state | committed_offset: offset, last_commit: :erlang.monotonic_time(:milli_seconds)} + end + + defp load_offsets(%State{worker_name: worker_name, group: group, topic: topic, partition: partition} = state) do + request = %OffsetFetchRequest{consumer_group: group, topic: topic, partition: partition} + + [%OffsetFetchResponse{topic: ^topic, partitions: [%{partition: ^partition, error_code: error_code, offset: offset}]}] = + KafkaEx.offset_fetch(worker_name, request) + + case error_code do + :no_error -> + %State{state | current_offset: offset, committed_offset: offset, acked_offset: offset} + + :unknown_topic_or_partition -> + [%OffsetResponse{topic: ^topic, partition_offsets: [%{partition: ^partition, error_code: :no_error, offset: [offset]}]}] = + KafkaEx.earliest_offset(topic, partition, worker_name) + + %State{state | current_offset: offset, committed_offset: offset, acked_offset: offset} + end + end +end diff --git a/lib/kafka_ex/protocol/heartbeat.ex b/lib/kafka_ex/protocol/heartbeat.ex index 4e8ca91c..6fb81590 100644 --- a/lib/kafka_ex/protocol/heartbeat.ex +++ b/lib/kafka_ex/protocol/heartbeat.ex @@ -3,6 +3,17 @@ defmodule KafkaEx.Protocol.Heartbeat do Implementation of the Kafka Hearbeat request and response APIs """ + defmodule Request do + @moduledoc false + defstruct group_name: nil, member_id: nil, generation_id: nil + + @type t :: %Request{ + group_name: binary, + member_id: binary, + generation_id: integer, + } + end + defmodule Response do @moduledoc false # We could just return the error code instead of having the struct, but this @@ -11,17 +22,16 @@ defmodule KafkaEx.Protocol.Heartbeat do @type t :: %Response{error_code: atom | integer} end - @spec create_request(integer, binary, binary, binary, integer) :: binary - def create_request(correlation_id, client_id, member_id, group_id, generation_id) do + @spec create_request(integer, binary, Request.t) :: binary + def create_request(correlation_id, client_id, request) do KafkaEx.Protocol.create_request(:heartbeat, correlation_id, client_id) <> - << byte_size(group_id) :: 16-signed, group_id :: binary, - generation_id :: 32-signed, - byte_size(member_id) :: 16-signed, member_id :: binary >> + << byte_size(request.group_name) :: 16-signed, request.group_name :: binary, + request.generation_id :: 32-signed, + byte_size(request.member_id) :: 16-signed, request.member_id :: binary >> end @spec parse_response(binary) :: Response.t def parse_response(<< _correlation_id :: 32-signed, error_code :: 16-signed >>) do %Response{error_code: KafkaEx.Protocol.error(error_code)} end - end diff --git a/lib/kafka_ex/protocol/join_group.ex b/lib/kafka_ex/protocol/join_group.ex index e9fa432d..06e1564a 100644 --- a/lib/kafka_ex/protocol/join_group.ex +++ b/lib/kafka_ex/protocol/join_group.ex @@ -4,19 +4,17 @@ defmodule KafkaEx.Protocol.JoinGroup do @moduledoc """ Implementation of the Kafka JoinGroup request and response APIs """ + @protocol_type "consumer" @strategy_name "assign" @metadata_version 0 defmodule Request do @moduledoc false - defstruct correlation_id: nil, - client_id: nil, member_id: nil, + defstruct member_id: nil, group_name: nil, topics: nil, session_timeout: nil @type t :: %Request{ - correlation_id: integer, - client_id: binary, member_id: binary, group_name: binary, topics: [binary], @@ -31,17 +29,15 @@ defmodule KafkaEx.Protocol.JoinGroup do leader_id: binary, member_id: binary, members: [binary]} end - @spec create_request(Request.t) :: binary - def create_request(join_group_req) do + @spec create_request(integer, binary, Request.t) :: binary + def create_request(correlation_id, client_id, %Request{} = join_group_req) do metadata = << @metadata_version :: 16-signed, length(join_group_req.topics) :: 32-signed, topic_data(join_group_req.topics) :: binary, 0 :: 32-signed >> - KafkaEx.Protocol.create_request( - :join_group, join_group_req.correlation_id, join_group_req.client_id - ) <> + KafkaEx.Protocol.create_request(:join_group, correlation_id, client_id) <> << byte_size(join_group_req.group_name) :: 16-signed, join_group_req.group_name :: binary, join_group_req.session_timeout :: 32-signed, byte_size(join_group_req.member_id) :: 16-signed, join_group_req.member_id :: binary, diff --git a/lib/kafka_ex/protocol/leave_group.ex b/lib/kafka_ex/protocol/leave_group.ex index 11576e81..6a4b9ba5 100644 --- a/lib/kafka_ex/protocol/leave_group.ex +++ b/lib/kafka_ex/protocol/leave_group.ex @@ -1,12 +1,27 @@ defmodule KafkaEx.Protocol.LeaveGroup do + defmodule Request do + @moduledoc false + defstruct group_name: nil, member_id: nil + + @type t :: %Request{ + group_name: binary, + member_id: binary, + } + end + defmodule Response do defstruct error_code: nil + + @type t :: %Response{ + error_code: atom | integer, + } end - def create_request(correlation_id, client_id, group_id, member_id) do + @spec create_request(integer, binary, Request.t) :: binary + def create_request(correlation_id, client_id, request) do KafkaEx.Protocol.create_request(:leave_group, correlation_id, client_id) <> - << byte_size(group_id) :: 16-signed, group_id :: binary, - byte_size(member_id) :: 16-signed, member_id :: binary >> + << byte_size(request.group_name) :: 16-signed, request.group_name :: binary, + byte_size(request.member_id) :: 16-signed, request.member_id :: binary >> end def parse_response(<< _correlation_id :: 32-signed, error_code :: 16-signed >>) do diff --git a/lib/kafka_ex/protocol/sync_group.ex b/lib/kafka_ex/protocol/sync_group.ex index efe5c06e..a94be427 100644 --- a/lib/kafka_ex/protocol/sync_group.ex +++ b/lib/kafka_ex/protocol/sync_group.ex @@ -4,6 +4,18 @@ defmodule KafkaEx.Protocol.SyncGroup do """ @member_assignment_version 0 + defmodule Request do + @moduledoc false + defstruct member_id: nil, group_name: nil, generation_id: nil, assignments: [] + + @type t :: %Request{ + member_id: binary, + group_name: binary, + generation_id: integer, + assignments: [{member :: binary, [{topic :: binary, partitions :: [integer]}]}], + } + end + defmodule Assignment do @moduledoc false defstruct topic: nil, partitions: [] @@ -16,23 +28,21 @@ defmodule KafkaEx.Protocol.SyncGroup do @type t :: %Response{error_code: atom | integer, assignments: [Assignment.t]} end - @spec create_request(integer, binary, binary, integer, binary, [{binary, [Assignment.t]}]) :: binary - def create_request(correlation_id, client_id, group_name, generation_id, member_id, assignments) do + @spec create_request(integer, binary, Request.t) :: binary + def create_request(correlation_id, client_id, %Request{} = request) do KafkaEx.Protocol.create_request(:sync_group, correlation_id, client_id) <> - << byte_size(group_name) :: 16-signed, group_name :: binary, - generation_id :: 32-signed, - byte_size(member_id) :: 16-signed, member_id :: binary, - length(assignments) :: 32-signed, group_assignment_data(assignments, "") :: binary + << byte_size(request.group_name) :: 16-signed, request.group_name :: binary, + request.generation_id :: 32-signed, + byte_size(request.member_id) :: 16-signed, request.member_id :: binary, + length(request.assignments) :: 32-signed, group_assignment_data(request.assignments, "") :: binary >> end @spec parse_response(binary) :: Response.t def parse_response(<< _correlation_id :: 32-signed, error_code :: 16-signed, - _member_assignment_len :: 32-signed, - @member_assignment_version :: 16-signed, - assignments_size :: 32-signed, rest :: binary >>) do - assignments = parse_assignments(assignments_size, rest, []) - %Response{error_code: KafkaEx.Protocol.error(error_code), assignments: assignments} + member_assignment_len :: 32-signed, + member_assignment :: size(member_assignment_len)-binary >>) do + %Response{error_code: KafkaEx.Protocol.error(error_code), assignments: parse_member_assignment(member_assignment)} end # Helper functions to create assignment data structure @@ -65,6 +75,11 @@ defmodule KafkaEx.Protocol.SyncGroup do # Helper functions to parse assignments + defp parse_member_assignment(<<>>), do: [] + defp parse_member_assignment(<< @member_assignment_version :: 16-signed, assignments_size :: 32-signed, rest :: binary >>) do + parse_assignments(assignments_size, rest, []) + end + defp parse_assignments(0, _rest, assignments), do: assignments defp parse_assignments(size, << topic_len :: 16-signed, topic :: size(topic_len)-binary, partition_len :: 32-signed, diff --git a/lib/kafka_ex/server.ex b/lib/kafka_ex/server.ex index eb69c1e4..8785e63f 100644 --- a/lib/kafka_ex/server.ex +++ b/lib/kafka_ex/server.ex @@ -4,6 +4,9 @@ defmodule KafkaEx.Server do """ alias KafkaEx.Protocol.ConsumerMetadata + alias KafkaEx.Protocol.Heartbeat.Request, as: HeartbeatRequest + alias KafkaEx.Protocol.JoinGroup.Request, as: JoinGroupRequest + alias KafkaEx.Protocol.LeaveGroup.Request, as: LeaveGroupRequest alias KafkaEx.Protocol.Metadata alias KafkaEx.Protocol.Metadata.Broker alias KafkaEx.Protocol.Metadata.Response, as: MetadataResponse @@ -12,6 +15,7 @@ defmodule KafkaEx.Server do alias KafkaEx.Protocol.Fetch.Request, as: FetchRequest alias KafkaEx.Protocol.Produce alias KafkaEx.Protocol.Produce.Request, as: ProduceRequest + alias KafkaEx.Protocol.SyncGroup.Request, as: SyncGroupRequest alias KafkaEx.Socket defmodule State do @@ -108,28 +112,28 @@ defmodule KafkaEx.Server do {:noreply, new_state, timeout | :hibernate} | {:stop, reason, reply, new_state} | {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_join_group(topics :: [binary], session_timeout :: integer, state :: State.t) :: + @callback kafka_server_join_group(JoinGroupRequest.t, network_timeout :: integer, state :: State.t) :: {:reply, reply, new_state} | {:reply, reply, new_state, timeout | :hibernate} | {:noreply, new_state} | {:noreply, new_state, timeout | :hibernate} | {:stop, reason, reply, new_state} | {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_sync_group(group_name :: binary, generation_id :: integer, member_id :: binary, assignments :: [binary] , state :: State.t) :: + @callback kafka_server_sync_group(SyncGroupRequest.t, network_timeout :: integer, state :: State.t) :: {:reply, reply, new_state} | {:reply, reply, new_state, timeout | :hibernate} | {:noreply, new_state} | {:noreply, new_state, timeout | :hibernate} | {:stop, reason, reply, new_state} | {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_leave_group(group_name :: binary, member_id :: binary, state :: State.t) :: + @callback kafka_server_leave_group(LeaveGroupRequest.t, network_timeout :: integer, state :: State.t) :: {:reply, reply, new_state} | {:reply, reply, new_state, timeout | :hibernate} | {:noreply, new_state} | {:noreply, new_state, timeout | :hibernate} | {:stop, reason, reply, new_state} | {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_heartbeat(group_name :: binary, generation_id :: integer, member_id :: binary, state :: State.t) :: + @callback kafka_server_heartbeat(HeartbeatRequest.t, network_timeout :: integer, state :: State.t) :: {:reply, reply, new_state} | {:reply, reply, new_state, timeout | :hibernate} | {:noreply, new_state} | @@ -237,20 +241,20 @@ defmodule KafkaEx.Server do kafka_server_metadata(topic, state) end - def handle_call({:join_group, topics, session_timeout}, _from, state) do - kafka_server_join_group(topics, session_timeout,state) + def handle_call({:join_group, request, network_timeout}, _from, state) do + kafka_server_join_group(request, network_timeout, state) end - def handle_call({:sync_group, group_name, generation_id, member_id, assignments}, _from, state) do - kafka_server_sync_group(group_name, generation_id, member_id, assignments, state) + def handle_call({:sync_group, request, network_timeout}, _from, state) do + kafka_server_sync_group(request, network_timeout, state) end - def handle_call({:leave_group, group_name, member_id}, _from, state) do - kafka_server_leave_group(group_name, member_id, state) + def handle_call({:leave_group, request, network_timeout}, _from, state) do + kafka_server_leave_group(request, network_timeout, state) end - def handle_call({:heartbeat, group_name, generation_id, member_id}, _from, state) do - kafka_server_heartbeat(group_name, generation_id, member_id, state) + def handle_call({:heartbeat, request, network_timeout}, _from, state) do + kafka_server_heartbeat(request, network_timeout, state) end def handle_call({:create_stream, handler, handler_init}, _from, state) do @@ -454,8 +458,8 @@ defmodule KafkaEx.Server do end) end - defp sync_timeout do - Application.get_env(:kafka_ex, :sync_timeout, @sync_timeout) + defp sync_timeout(timeout \\ nil) do + timeout || Application.get_env(:kafka_ex, :sync_timeout, @sync_timeout) end end end diff --git a/lib/kafka_ex/server_0_p_8_p_0.ex b/lib/kafka_ex/server_0_p_8_p_0.ex index 83c82c93..b24fceb0 100644 --- a/lib/kafka_ex/server_0_p_8_p_0.ex +++ b/lib/kafka_ex/server_0_p_8_p_0.ex @@ -5,8 +5,8 @@ defmodule KafkaEx.Server0P8P0 do # these functions aren't implemented for 0.8.0 @dialyzer [ - {:nowarn_function, kafka_server_heartbeat: 4}, - {:nowarn_function, kafka_server_sync_group: 5}, + {:nowarn_function, kafka_server_heartbeat: 3}, + {:nowarn_function, kafka_server_sync_group: 3}, {:nowarn_function, kafka_server_join_group: 3}, {:nowarn_function, kafka_server_leave_group: 3}, {:nowarn_function, kafka_server_update_consumer_metadata: 1}, @@ -62,9 +62,9 @@ defmodule KafkaEx.Server0P8P0 do def kafka_server_consumer_group(_state), do: raise "Consumer Group is not supported in 0.8.0 version of kafka" def kafka_server_consumer_group_metadata(_state), do: raise "Consumer Group Metadata is not supported in 0.8.0 version of kafka" def kafka_server_join_group(_, _, _state), do: raise "Join Group is not supported in 0.8.0 version of kafka" - def kafka_server_sync_group(_, _, _, _, _state), do: raise "Sync Group is not supported in 0.8.0 version of kafka" + def kafka_server_sync_group(_, _, _state), do: raise "Sync Group is not supported in 0.8.0 version of kafka" def kafka_server_leave_group(_, _, _state), do: raise "Leave Group is not supported in 0.8.0 version of Kafka" - def kafka_server_heartbeat(_, _, _, _state), do: raise "Heartbeat is not supported in 0.8.0 version of kafka" + def kafka_server_heartbeat(_, _, _state), do: raise "Heartbeat is not supported in 0.8.0 version of kafka" def kafka_server_update_consumer_metadata(_state), do: raise "Consumer Group Metadata is not supported in 0.8.0 version of kafka" def kafka_server_start_streaming(_, state = %State{event_pid: nil}) do diff --git a/lib/kafka_ex/server_0_p_8_p_2.ex b/lib/kafka_ex/server_0_p_8_p_2.ex index 2677f792..8b4832d4 100644 --- a/lib/kafka_ex/server_0_p_8_p_2.ex +++ b/lib/kafka_ex/server_0_p_8_p_2.ex @@ -5,8 +5,8 @@ defmodule KafkaEx.Server0P8P2 do # these functions aren't implemented for 0.8.2 @dialyzer [ - {:nowarn_function, kafka_server_heartbeat: 4}, - {:nowarn_function, kafka_server_sync_group: 5}, + {:nowarn_function, kafka_server_heartbeat: 3}, + {:nowarn_function, kafka_server_sync_group: 3}, {:nowarn_function, kafka_server_join_group: 3}, {:nowarn_function, kafka_server_leave_group: 3} ] @@ -145,9 +145,9 @@ defmodule KafkaEx.Server0P8P2 do end def kafka_server_join_group(_, _, _state), do: raise "Join Group is not supported in 0.8.0 version of kafka" - def kafka_server_sync_group(_, _, _, _, _state), do: raise "Sync Group is not supported in 0.8.0 version of kafka" + def kafka_server_sync_group(_, _, _state), do: raise "Sync Group is not supported in 0.8.0 version of kafka" def kafka_server_leave_group(_, _, _state), do: raise "Leave Group is not supported in 0.8.0 version of Kafka" - def kafka_server_heartbeat(_, _, _, _state), do: raise "Heartbeat is not supported in 0.8.0 version of kafka" + def kafka_server_heartbeat(_, _, _state), do: raise "Heartbeat is not supported in 0.8.0 version of kafka" defp update_consumer_metadata(state), do: update_consumer_metadata(state, @retry_count, 0) defp update_consumer_metadata(state = %State{consumer_group: consumer_group}, 0, error_code) do diff --git a/lib/kafka_ex/server_0_p_9_p_0.ex b/lib/kafka_ex/server_0_p_9_p_0.ex index c8ba2041..76a520db 100644 --- a/lib/kafka_ex/server_0_p_9_p_0.ex +++ b/lib/kafka_ex/server_0_p_9_p_0.ex @@ -7,7 +7,6 @@ defmodule KafkaEx.Server0P9P0 do alias KafkaEx.Protocol.ConsumerMetadata.Response, as: ConsumerMetadataResponse alias KafkaEx.Protocol.Heartbeat alias KafkaEx.Protocol.JoinGroup - alias KafkaEx.Protocol.JoinGroup.Request, as: JoinGroupRequest alias KafkaEx.Protocol.LeaveGroup alias KafkaEx.Protocol.Metadata.Broker alias KafkaEx.Protocol.SyncGroup @@ -68,49 +67,42 @@ defmodule KafkaEx.Server0P9P0 do {:ok, state} end - def kafka_server_join_group(topics, session_timeout, state) do + def kafka_server_join_group(join_group_request, network_timeout, state) do true = consumer_group?(state) {broker, state} = broker_for_consumer_group_with_update(state) - request = JoinGroup.create_request( - %JoinGroupRequest{ - correlation_id: state.correlation_id, - client_id: @client_id, member_id: "", - group_name: state.consumer_group, - topics: topics, session_timeout: session_timeout - } - ) + request = JoinGroup.create_request(state.correlation_id, @client_id, join_group_request) response = broker - |> NetworkClient.send_sync_request(request, sync_timeout()) + |> NetworkClient.send_sync_request(request, sync_timeout(network_timeout)) |> JoinGroup.parse_response {:reply, response, %{state | correlation_id: state.correlation_id + 1}} end - def kafka_server_sync_group(group_name, generation_id, member_id, assignments, state) do + def kafka_server_sync_group(sync_group_request, network_timeout, state) do true = consumer_group?(state) {broker, state} = broker_for_consumer_group_with_update(state) - request = SyncGroup.create_request(state.correlation_id, @client_id, group_name, generation_id, member_id, assignments) + request = SyncGroup.create_request(state.correlation_id, @client_id, sync_group_request) response = broker - |> NetworkClient.send_sync_request(request, sync_timeout()) + |> NetworkClient.send_sync_request(request, sync_timeout(network_timeout)) |> SyncGroup.parse_response {:reply, response, %{state | correlation_id: state.correlation_id + 1}} end - def kafka_server_leave_group(group_name, member_id, state) do + def kafka_server_leave_group(request, network_timeout, state) do true = consumer_group?(state) {broker, state} = broker_for_consumer_group_with_update(state) - request = LeaveGroup.create_request(state.correlation_id, @client_id, group_name, member_id) + request = LeaveGroup.create_request(state.correlation_id, @client_id, request) response = broker - |> NetworkClient.send_sync_request(request, sync_timeout()) + |> NetworkClient.send_sync_request(request, sync_timeout(network_timeout)) |> LeaveGroup.parse_response {:reply, response, %{state | correlation_id: state.correlation_id + 1}} end - def kafka_server_heartbeat(group_name, generation_id, member_id, state) do + def kafka_server_heartbeat(request, network_timeout, state) do true = consumer_group?(state) {broker, state} = broker_for_consumer_group_with_update(state) - request = Heartbeat.create_request(state.correlation_id, @client_id, member_id, group_name, generation_id) + request = Heartbeat.create_request(state.correlation_id, @client_id, request) response = broker - |> NetworkClient.send_sync_request(request, sync_timeout()) + |> NetworkClient.send_sync_request(request, sync_timeout(network_timeout)) |> Heartbeat.parse_response {:reply, response, %{state | correlation_id: state.correlation_id + 1}} end diff --git a/test/integration/server0_p_9_p_0_test.exs b/test/integration/server0_p_9_p_0_test.exs index aa32ac27..265cbf50 100644 --- a/test/integration/server0_p_9_p_0_test.exs +++ b/test/integration/server0_p_9_p_0_test.exs @@ -2,14 +2,25 @@ defmodule KafkaEx.Server0P9P0.Test do use ExUnit.Case import TestHelper + alias KafkaEx.Protocol.Heartbeat.Request, as: HeartbeatRequest + alias KafkaEx.Protocol.JoinGroup.Request, as: JoinGroupRequest + alias KafkaEx.Protocol.LeaveGroup.Request, as: LeaveGroupRequest + alias KafkaEx.Protocol.SyncGroup.Request, as: SyncGroupRequest + @moduletag :server_0_p_9_p_0 test "can join a consumer group" do random_group = generate_random_string() KafkaEx.create_worker(:join_group, [uris: uris(), consumer_group: random_group]) - # No wrapper in kafka_ex yet as long as the 0.9 functionality is in progress - answer = GenServer.call(:join_group, {:join_group, ["foo", "bar"], 6000}) + request = %JoinGroupRequest{ + group_name: random_group, + member_id: "", + topics: ["foo", "bar"], + session_timeout: 6000, + } + + answer = KafkaEx.join_group(request, worker_name: :join_group) assert answer.error_code == :no_error assert answer.generation_id == 1 # We should be the leader @@ -21,7 +32,15 @@ defmodule KafkaEx.Server0P9P0.Test do # how this pans out eventually as we add more and more 0.9 consumer group code random_group = generate_random_string() KafkaEx.create_worker(:sync_group, [uris: uris(), consumer_group: random_group]) - answer = GenServer.call(:sync_group, {:join_group, ["foo", "bar"], 6000}) + + request = %JoinGroupRequest{ + group_name: random_group, + member_id: "", + topics: ["foo", "bar"], + session_timeout: 6000, + } + + answer = KafkaEx.join_group(request, worker_name: :sync_group) assert answer.error_code == :no_error member_id = answer.member_id @@ -29,7 +48,14 @@ defmodule KafkaEx.Server0P9P0.Test do my_assignments = [{"foo", [1]}, {"bar", [2]}] assignments = [{member_id, my_assignments}] - answer = GenServer.call(:sync_group, {:sync_group, random_group, generation_id, member_id, assignments}) + request = %SyncGroupRequest{ + group_name: random_group, + member_id: member_id, + generation_id: generation_id, + assignments: assignments, + } + + answer = KafkaEx.sync_group(request, worker_name: :sync_group) assert answer.error_code == :no_error # Parsing happens to return the assignments reversed, which is fine as there's no # ordering. Just reverse what we expect to match @@ -41,12 +67,25 @@ defmodule KafkaEx.Server0P9P0.Test do # how this pans out eventually as we add more and more 0.9 consumer group code random_group = generate_random_string() KafkaEx.create_worker(:leave_group, [uris: uris(), consumer_group: random_group]) - answer = GenServer.call(:leave_group, {:join_group, ["foo", "bar"], 6000}) + + request = %JoinGroupRequest{ + group_name: random_group, + member_id: "", + topics: ["foo", "bar"], + session_timeout: 6000, + } + + answer = KafkaEx.join_group(request, worker_name: :leave_group) assert answer.error_code == :no_error member_id = answer.member_id - answer = GenServer.call(:leave_group, {:leave_group, random_group, member_id}) + request = %LeaveGroupRequest{ + group_name: random_group, + member_id: member_id, + } + + answer = KafkaEx.leave_group(request, worker_name: :leave_group) assert answer.error_code == :no_error end @@ -54,7 +93,15 @@ defmodule KafkaEx.Server0P9P0.Test do # See sync test. Removing repetition in the next iteration random_group = generate_random_string() KafkaEx.create_worker(:heartbeat, [uris: uris(), consumer_group: random_group]) - answer = GenServer.call(:heartbeat, {:join_group, ["foo", "bar"], 6000}) + + request = %JoinGroupRequest{ + group_name: random_group, + member_id: "", + topics: ["foo", "bar"], + session_timeout: 6000, + } + + answer = KafkaEx.join_group(request, worker_name: :heartbeat) assert answer.error_code == :no_error member_id = answer.member_id @@ -62,10 +109,23 @@ defmodule KafkaEx.Server0P9P0.Test do my_assignments = [{"foo", [1]}, {"bar", [2]}] assignments = [{member_id, my_assignments}] - answer = GenServer.call(:heartbeat, {:sync_group, random_group, generation_id, member_id, assignments}) + request = %SyncGroupRequest{ + group_name: random_group, + member_id: member_id, + generation_id: generation_id, + assignments: assignments, + } + + answer = KafkaEx.sync_group(request, worker_name: :heartbeat) assert answer.error_code == :no_error - answer = GenServer.call(:heartbeat, {:heartbeat, random_group, generation_id, member_id}) + request = %HeartbeatRequest{ + group_name: random_group, + member_id: member_id, + generation_id: generation_id, + } + + answer = KafkaEx.heartbeat(request, worker_name: :heartbeat) assert answer.error_code == :no_error end end diff --git a/test/protocol/heartbeat_test.exs b/test/protocol/heartbeat_test.exs index efeb16a2..7ad16aaa 100644 --- a/test/protocol/heartbeat_test.exs +++ b/test/protocol/heartbeat_test.exs @@ -8,7 +8,13 @@ defmodule KafkaEx.Protocol.Heartbeat.Test do 1234 :: 32, # GenerationId 9 :: 16, "member_id" :: binary # MemberId >> - request = KafkaEx.Protocol.Heartbeat.create_request(42, "client_id", "member_id", "group_id", 1234) + heartbeat_request = %KafkaEx.Protocol.Heartbeat.Request{ + group_name: "group_id", + member_id: "member_id", + generation_id: 1234, + } + + request = KafkaEx.Protocol.Heartbeat.create_request(42, "client_id", heartbeat_request) assert request == good_request end diff --git a/test/protocol/join_group_test.exs b/test/protocol/join_group_test.exs index d8fb87fd..f67f0495 100644 --- a/test/protocol/join_group_test.exs +++ b/test/protocol/join_group_test.exs @@ -15,10 +15,8 @@ defmodule KafkaEx.Protocol.JoinGroup.Test do 0 :: 16, # v0 2 :: 32, 9 :: 16, "topic_one" :: binary, 9 :: 16, "topic_two" :: binary, # Topics array 0 :: 32 >> # UserData - request = JoinGroup.create_request( + request = JoinGroup.create_request(42, "client_id", %JoinGroup.Request{ - correlation_id: 42, - client_id: "client_id", member_id: "member_id", group_name: "group", topics: ["topic_one", "topic_two"], diff --git a/test/protocol/leave_group_test.exs b/test/protocol/leave_group_test.exs index 05e0863a..176f5a20 100644 --- a/test/protocol/leave_group_test.exs +++ b/test/protocol/leave_group_test.exs @@ -8,7 +8,12 @@ defmodule KafkaEx.Protocol.LeaveGroup.Test do byte_size("member") :: 16, "member" :: binary, # MemberId >> - request = KafkaEx.Protocol.LeaveGroup.create_request(42, "client_id", "group", "member") + leave_request = %KafkaEx.Protocol.LeaveGroup.Request{ + group_name: "group", + member_id: "member", + } + + request = KafkaEx.Protocol.LeaveGroup.create_request(42, "client_id", leave_request) assert request == good_request end diff --git a/test/protocol/sync_group_test.exs b/test/protocol/sync_group_test.exs index bb3f73c3..aadccddf 100644 --- a/test/protocol/sync_group_test.exs +++ b/test/protocol/sync_group_test.exs @@ -28,8 +28,14 @@ defmodule KafkaEx.Protocol.SyncGroup.Test do byte_size(second_assignments) :: 32, second_assignments :: binary >> - request = KafkaEx.Protocol.SyncGroup.create_request(42, "client_id", "group", 1, "member_one", - [{"member_one", [{"topic1", [1, 3, 5]}]}, {"member_two", [{"topic1", [2, 4, 6]}]}]) + sync_request = %KafkaEx.Protocol.SyncGroup.Request{ + group_name: "group", + member_id: "member_one", + generation_id: 1, + assignments: [{"member_one", [{"topic1", [1, 3, 5]}]}, {"member_two", [{"topic1", [2, 4, 6]}]}], + } + + request = KafkaEx.Protocol.SyncGroup.create_request(42, "client_id", sync_request) assert request == good_request end @@ -44,4 +50,14 @@ defmodule KafkaEx.Protocol.SyncGroup.Test do assignments: [{"topic1", [5, 3, 1]}]} assert KafkaEx.Protocol.SyncGroup.parse_response(response) == expected_response end + + test "parse empty assignments correctly" do + response = << + 42 :: 32, # CorrelationId + 0 :: 16, # ErrorCode + 0 :: 32, # MemberAssignment (empty) + >> + expected_response = %KafkaEx.Protocol.SyncGroup.Response{error_code: :no_error, assignments: []} + assert KafkaEx.Protocol.SyncGroup.parse_response(response) == expected_response + end end From 3cb335e51e60172254922644a744c11c4d55e7ec Mon Sep 17 00:00:00 2001 From: David Cuddeback Date: Wed, 10 May 2017 13:49:18 -0700 Subject: [PATCH 02/47] improves error handling --- lib/kafka_ex/consumer_group.ex | 21 ++++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 4115a7b7..3314e477 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -56,6 +56,7 @@ defmodule KafkaEx.ConsumerGroup do alias KafkaEx.Config alias KafkaEx.Protocol.JoinGroup.Request, as: JoinGroupRequest + alias KafkaEx.Protocol.JoinGroup.Response, as: JoinGroupResponse alias KafkaEx.Protocol.Heartbeat.Request, as: HeartbeatRequest alias KafkaEx.Protocol.Heartbeat.Response, as: HeartbeatResponse alias KafkaEx.Protocol.LeaveGroup.Request, as: LeaveGroupRequest @@ -140,13 +141,13 @@ defmodule KafkaEx.ConsumerGroup do end def handle_info(:timeout, %State{generation_id: nil, member_id: ""} = state) do - new_state = join(state) + {:ok, new_state} = join(state) {:noreply, new_state, new_state.heartbeat_interval} end def handle_info(:timeout, %State{} = state) do - new_state = heartbeat(state) + {:ok, new_state} = heartbeat(state) {:noreply, new_state, new_state.heartbeat_interval} end @@ -161,8 +162,9 @@ defmodule KafkaEx.ConsumerGroup do {:noreply, state, state.heartbeat_interval} end + def terminate(_reason, %State{generation_id: nil, member_id: ""}), do: :ok def terminate(_reason, %State{} = state) do - leave(state) + :ok = leave(state) end # Helpers @@ -175,11 +177,13 @@ defmodule KafkaEx.ConsumerGroup do session_timeout: session_timeout, } - join_response = KafkaEx.join_group(join_request, worker_name: worker_name, timeout: session_timeout + 5000) - new_state = %State{state | member_id: join_response.member_id, generation_id: join_response.generation_id} + join_response = %JoinGroupResponse{error_code: :no_error} = + KafkaEx.join_group(join_request, worker_name: worker_name, timeout: session_timeout + 5000) Logger.debug("Joined consumer group #{group_name}") + new_state = %State{state | member_id: join_response.member_id, generation_id: join_response.generation_id} + if join_response.member_id == join_response.leader_id do sync_leader(new_state, join_response.members) else @@ -245,9 +249,10 @@ defmodule KafkaEx.ConsumerGroup do case KafkaEx.heartbeat(heartbeat_request, worker_name: worker_name) do %HeartbeatResponse{error_code: :no_error} -> - state + {:ok, state} %HeartbeatResponse{error_code: :rebalance_in_progress} -> + Logger.debug("Rebalancing consumer group #{group_name}") rebalance(state) end end @@ -269,6 +274,8 @@ defmodule KafkaEx.ConsumerGroup do %LeaveGroupResponse{error_code: :no_error} = KafkaEx.leave_group(leave_request, worker_name: worker_name) Logger.debug("Left consumer group #{group_name}") + + :ok end defp start_consumer(%State{consumer_module: consumer_module, consumer_opts: consumer_opts, @@ -280,7 +287,7 @@ defmodule KafkaEx.ConsumerGroup do {:ok, pid} = KafkaEx.GenConsumer.Supervisor.start_link(consumer_module, group_name, assignments, consumer_opts) - %State{state | assignments: assignments, consumer_pid: pid} + {:ok, %State{state | assignments: assignments, consumer_pid: pid}} end defp stop_consumer(%State{consumer_pid: nil} = state), do: state From da7ce510745648c3ec267819b5da0781a2f9fc2f Mon Sep 17 00:00:00 2001 From: David Cuddeback Date: Wed, 10 May 2017 14:09:08 -0700 Subject: [PATCH 03/47] makes session timeout padding a module attribute --- lib/kafka_ex/consumer_group.ex | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 3314e477..33362f7d 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -89,6 +89,7 @@ defmodule KafkaEx.ConsumerGroup do @heartbeat_interval 5_000 @session_timeout 30_000 + @session_timeout_padding 5_000 # Client API @@ -178,7 +179,7 @@ defmodule KafkaEx.ConsumerGroup do } join_response = %JoinGroupResponse{error_code: :no_error} = - KafkaEx.join_group(join_request, worker_name: worker_name, timeout: session_timeout + 5000) + KafkaEx.join_group(join_request, worker_name: worker_name, timeout: session_timeout + @session_timeout_padding) Logger.debug("Joined consumer group #{group_name}") @@ -217,7 +218,7 @@ defmodule KafkaEx.ConsumerGroup do } sync_request - |> KafkaEx.sync_group(worker_name: worker_name, timeout: session_timeout + 5000) + |> KafkaEx.sync_group(worker_name: worker_name, timeout: session_timeout + @session_timeout_padding) |> update_assignments(state) end @@ -231,7 +232,7 @@ defmodule KafkaEx.ConsumerGroup do } sync_request - |> KafkaEx.sync_group(timeout: session_timeout + 5000, worker_name: worker_name) + |> KafkaEx.sync_group(timeout: session_timeout + @session_timeout_padding, worker_name: worker_name) |> update_assignments(state) end From 6c279d2e50e75f403d1d15fd273d4ed107bda5ce Mon Sep 17 00:00:00 2001 From: David Cuddeback Date: Wed, 10 May 2017 14:58:59 -0700 Subject: [PATCH 04/47] refactor to combine common code in sync group functions --- lib/kafka_ex/consumer_group.ex | 80 +++++++++++++++------------------- 1 file changed, 34 insertions(+), 46 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 33362f7d..10a9bb50 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -79,7 +79,6 @@ defmodule KafkaEx.ConsumerGroup do :consumer_opts, :group_name, :topics, - :partitions, :member_id, :generation_id, :assignments, @@ -185,31 +184,19 @@ defmodule KafkaEx.ConsumerGroup do new_state = %State{state | member_id: join_response.member_id, generation_id: join_response.generation_id} - if join_response.member_id == join_response.leader_id do - sync_leader(new_state, join_response.members) - else - sync_follower(new_state) - end - end - - defp sync_leader(%State{worker_name: worker_name, topics: topics, partitions: nil} = state, members) do - %MetadataResponse{topic_metadatas: topic_metadatas} = KafkaEx.metadata(worker_name: worker_name) - - partitions = Enum.flat_map(topics, fn (topic) -> - %TopicMetadata{error_code: :no_error, partition_metadatas: partition_metadatas} = Enum.find(topic_metadatas, &(&1.topic == topic)) - - Enum.map(partition_metadatas, fn (%PartitionMetadata{error_code: :no_error, partition_id: partition_id}) -> - {topic, partition_id} - end) - end) - - sync_leader(%State{state | partitions: partitions}, members) + assignments = + if join_response.member_id == join_response.leader_id do + partitions = assignable_partitions(new_state) + assign_partitions(new_state, join_response.members, partitions) + else + [] + end + + sync(new_state, assignments) end - defp sync_leader(%State{worker_name: worker_name, session_timeout: session_timeout, - group_name: group_name, generation_id: generation_id, member_id: member_id} = state, members) do - assignments = assign_partitions(state, members) - + defp sync(%State{group_name: group_name, member_id: member_id, generation_id: generation_id, + worker_name: worker_name, session_timeout: session_timeout} = state, assignments) do sync_request = %SyncGroupRequest{ group_name: group_name, member_id: member_id, @@ -217,28 +204,13 @@ defmodule KafkaEx.ConsumerGroup do assignments: assignments, } - sync_request - |> KafkaEx.sync_group(worker_name: worker_name, timeout: session_timeout + @session_timeout_padding) - |> update_assignments(state) - end + case KafkaEx.sync_group(sync_request, worker_name: worker_name, timeout: session_timeout + @session_timeout_padding) do + %SyncGroupResponse{error_code: :no_error, assignments: assignments} -> + start_consumer(state, assignments) - defp sync_follower(%State{worker_name: worker_name, session_timeout: session_timeout, - group_name: group_name, generation_id: generation_id, member_id: member_id} = state) do - sync_request = %SyncGroupRequest{ - group_name: group_name, - member_id: member_id, - generation_id: generation_id, - assignments: [], - } - - sync_request - |> KafkaEx.sync_group(timeout: session_timeout + @session_timeout_padding, worker_name: worker_name) - |> update_assignments(state) - end - - defp update_assignments(%SyncGroupResponse{error_code: :rebalance_in_progress}, %State{} = state), do: rebalance(state) - defp update_assignments(%SyncGroupResponse{error_code: :no_error, assignments: assignments}, %State{} = state) do - start_consumer(state, assignments) + %SyncGroupResponse{error_code: :rebalance_in_progress} -> + rebalance(state) + end end defp heartbeat(%State{worker_name: worker_name, group_name: group_name, generation_id: generation_id, member_id: member_id} = state) do @@ -279,6 +251,8 @@ defmodule KafkaEx.ConsumerGroup do :ok end + # Consumer Management + defp start_consumer(%State{consumer_module: consumer_module, consumer_opts: consumer_opts, group_name: group_name, consumer_pid: nil} = state, assignments) do assignments = @@ -297,7 +271,21 @@ defmodule KafkaEx.ConsumerGroup do %State{state | consumer_pid: nil} end - defp assign_partitions(%State{consumer_module: consumer_module, partitions: partitions}, members) do + # Partition Assignment + + defp assignable_partitions(%State{worker_name: worker_name, topics: topics}) do + %MetadataResponse{topic_metadatas: topic_metadatas} = KafkaEx.metadata(worker_name: worker_name) + + Enum.flat_map(topics, fn (topic) -> + %TopicMetadata{error_code: :no_error, partition_metadatas: partition_metadatas} = Enum.find(topic_metadatas, &(&1.topic == topic)) + + Enum.map(partition_metadatas, fn (%PartitionMetadata{error_code: :no_error, partition_id: partition_id}) -> + {topic, partition_id} + end) + end) + end + + defp assign_partitions(%State{consumer_module: consumer_module}, members, partitions) do assignments = consumer_module.assign_partitions(members, partitions) |> Enum.map(fn ({member, topic_partitions}) -> From f77c679eb4c080e0980674c61b69308a3e37a6e3 Mon Sep 17 00:00:00 2001 From: David Cuddeback Date: Wed, 10 May 2017 17:47:13 -0700 Subject: [PATCH 05/47] attempt to make ConsumerGroup logic easier to follow --- lib/kafka_ex/consumer_group.ex | 129 ++++++++++++++++++++++------ lib/kafka_ex/protocol/join_group.ex | 4 + 2 files changed, 105 insertions(+), 28 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 10a9bb50..5179556d 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -140,35 +140,54 @@ defmodule KafkaEx.ConsumerGroup do {:ok, state, 0} end + # If `member_id` and `generation_id` aren't set, we haven't yet joined the group. `member_id` and + # `generation_id` are initialized by `JoinGroupResponse`. def handle_info(:timeout, %State{generation_id: nil, member_id: ""} = state) do {:ok, new_state} = join(state) {:noreply, new_state, new_state.heartbeat_interval} end + # After joining the group, a member must periodically send heartbeats to the group coordinator. def handle_info(:timeout, %State{} = state) do {:ok, new_state} = heartbeat(state) {:noreply, new_state, new_state.heartbeat_interval} end + # Stop the consumer group if the consumer supervisor crashes. def handle_info({:EXIT, pid, reason}, %State{consumer_pid: pid} = state) do new_state = %State{state | consumer_pid: nil} {:stop, reason, new_state} end + # Ignore spurious exit signals. These are often from old consumer PIDs, because the exit signal + # can arrive after the `consumer_pid` state has already been updated. def handle_info({:EXIT, _pid, _reason}, %State{} = state) do {:noreply, state, state.heartbeat_interval} end + # When terminating, inform the group coordinator that this member is leaving the group so that the + # group can rebalance without waiting for a session timeout. def terminate(_reason, %State{generation_id: nil, member_id: ""}), do: :ok def terminate(_reason, %State{} = state) do :ok = leave(state) end - # Helpers - + ### Helpers + + # `JoinGroupRequest` is used to set the active members of a group. The response blocks until the + # broker has decided that it has a full list of group members. This requires that all active + # members send a `JoinGroupRequest`. For active members, this is triggered by the broker + # responding to a heartbeat with a `:rebalance_in_progress` error code. If any group members fail + # to send a `JoinGroupRequest` before the session timeout expires, then those group members are + # removed from the group and synchronization continues without them. + # + # `JoinGroupResponse` tells each member its unique member ID as well as the group's current + # generation ID. The broker will pick one group member to be the leader, which is reponsible for + # assigning partitions to all of the group members. Once a `JoinGroupResponse` is received, all + # group members must send a `SyncGroupRequest` (see sync/2). defp join(%State{worker_name: worker_name, session_timeout: session_timeout, group_name: group_name, topics: topics, member_id: member_id} = state) do join_request = %JoinGroupRequest{ group_name: group_name, @@ -185,16 +204,27 @@ defmodule KafkaEx.ConsumerGroup do new_state = %State{state | member_id: join_response.member_id, generation_id: join_response.generation_id} assignments = - if join_response.member_id == join_response.leader_id do + if JoinGroupResponse.leader?(join_response) do + # Leader is responsible for assigning partitions to all group members. partitions = assignable_partitions(new_state) assign_partitions(new_state, join_response.members, partitions) else + # Follower does not assign partitions; must be empty. [] end sync(new_state, assignments) end + # `SyncGroupRequest` is used to distribute partition assignments to all group members. All group + # members must send this request after receiving a response to a `JoinGroupRequest`. The request + # blocks until assignments are provided by the leader. The leader sends partition assignments + # (given by the `assignments` parameter) as part of its `SyncGroupRequest`. For all other members, + # `assignments` must be empty. + # + # `SyncGroupResponse` contains the individual member's partition assignments. Upon receiving a + # successful `SyncGroupResponse`, a group member is free to start consuming from its assigned + # partitions, but must send periodic heartbeats to the coordinating broker. defp sync(%State{group_name: group_name, member_id: member_id, generation_id: generation_id, worker_name: worker_name, session_timeout: session_timeout} = state, assignments) do sync_request = %SyncGroupRequest{ @@ -206,13 +236,24 @@ defmodule KafkaEx.ConsumerGroup do case KafkaEx.sync_group(sync_request, worker_name: worker_name, timeout: session_timeout + @session_timeout_padding) do %SyncGroupResponse{error_code: :no_error, assignments: assignments} -> - start_consumer(state, assignments) + start_consumer(state, unpack_assignments(assignments)) %SyncGroupResponse{error_code: :rebalance_in_progress} -> rebalance(state) end end + # `HeartbeatRequest` is sent periodically by each active group member (after completing the + # join/sync phase) to inform the broker that the member is still alive and participating in the + # group. If a group member fails to send a heartbeat before the group's session timeout expires, + # the coordinator removes that member from the group and initiates a rebalance. + # + # `HeartbeatResponse` allows the coordinating broker to communicate the group's status to each + # member: + # + # * `:no_error` indicates that the group is up to date and no action is needed. + # * `:rebalance_in_progress` instructs each member to rejoin the group by sending a + # `JoinGroupRequest` (see join/1). defp heartbeat(%State{worker_name: worker_name, group_name: group_name, generation_id: generation_id, member_id: member_id} = state) do heartbeat_request = %HeartbeatRequest{ group_name: group_name, @@ -230,12 +271,9 @@ defmodule KafkaEx.ConsumerGroup do end end - defp rebalance(%State{} = state) do - state - |> stop_consumer() - |> join() - end - + # `LeaveGroupRequest` is used to voluntarily leave a group. This tells the broker that the member + # is leaving the group without having to wait for the session timeout to expire. Leaving a group + # triggers a rebalance for the remaining group members. defp leave(%State{worker_name: worker_name, group_name: group_name, member_id: member_id} = state) do stop_consumer(state) @@ -251,28 +289,37 @@ defmodule KafkaEx.ConsumerGroup do :ok end - # Consumer Management + # When instructed that a rebalance is in progress, a group member must rejoin the group with + # `JoinGroupRequest` (see join/1). To keep the state synchronized during the join/sync phase, each + # member pauses its consumers and commits its offsets before rejoining the group. + defp rebalance(%State{} = state) do + state + |> stop_consumer() + |> join() + end + ### Consumer Management + + # Starts consuming from the member's assigned partitions. defp start_consumer(%State{consumer_module: consumer_module, consumer_opts: consumer_opts, group_name: group_name, consumer_pid: nil} = state, assignments) do - assignments = - Enum.flat_map(assignments, fn ({topic, partition_ids}) -> - Enum.map(partition_ids, &({topic, &1})) - end) - {:ok, pid} = KafkaEx.GenConsumer.Supervisor.start_link(consumer_module, group_name, assignments, consumer_opts) {:ok, %State{state | assignments: assignments, consumer_pid: pid}} end + # Stops consuming from the member's assigned partitions and commits offsets. defp stop_consumer(%State{consumer_pid: nil} = state), do: state defp stop_consumer(%State{consumer_pid: pid} = state) when is_pid(pid) do :ok = Supervisor.stop(pid) %State{state | consumer_pid: nil} end - # Partition Assignment + ### Partition Assignment + # Queries the Kafka brokers for a list of partitions for the topics of interest to this consumer + # group. This function returns a list of topic/partition tuples that can be passed to a + # GenConsumer's `assign_partitions` method. defp assignable_partitions(%State{worker_name: worker_name, topics: topics}) do %MetadataResponse{topic_metadatas: topic_metadatas} = KafkaEx.metadata(worker_name: worker_name) @@ -285,21 +332,47 @@ defmodule KafkaEx.ConsumerGroup do end) end + # This function is used by the group leader to determine partition assignments during the + # join/sync phase. `members` is provided to the leader by the coordinating broker in + # `JoinGroupResponse`. `partitions` is a list of topic/partition tuples, obtained from + # `assignable_partitions/1`. The return value is a complete list of member assignments in the + # format needed by `SyncGroupResponse`. defp assign_partitions(%State{consumer_module: consumer_module}, members, partitions) do - assignments = - consumer_module.assign_partitions(members, partitions) - |> Enum.map(fn ({member, topic_partitions}) -> - assigns = - topic_partitions - |> Enum.group_by(&(elem(&1, 0)), &(elem(&1, 1))) - |> Enum.into([]) - - {member, assigns} + # Delegate partition assignment to GenConsumer module. + assignments = consumer_module.assign_partitions(members, partitions) + + # Convert assignments to format expected by Kafka protocol. + packed_assignments = + Enum.map(assignments, fn ({member, topic_partitions}) -> + {member, pack_assignments(topic_partitions)} end) - |> Map.new + assignments_map = Map.new(packed_assignments) + # Fill in empty assignments for missing member IDs. Enum.map(members, fn (member) -> - {member, Map.get(assignments, member, [])} + {member, Map.get(assignments_map, member, [])} end) end + + # Converts assignments from Kafka's protocol format to topic/partition tuples. + # + # Example: + # + # unpack_assignments([{"foo", [0, 1]}]) #=> [{"foo", 0}, {"foo", 1}] + defp unpack_assignments(assignments) do + Enum.flat_map(assignments, fn ({topic, partition_ids}) -> + Enum.map(partition_ids, &({topic, &1})) + end) + end + + # Converts assignments from topic/partition tuples to Kafka's protocol format. + # + # Example: + # + # pack_assignments([{"foo", 0}, {"foo", 1}]) #=> [{"foo", [0, 1]}] + defp pack_assignments(assignments) do + assignments + |> Enum.group_by(&(elem(&1, 0)), &(elem(&1, 1))) + |> Enum.into([]) + end end diff --git a/lib/kafka_ex/protocol/join_group.ex b/lib/kafka_ex/protocol/join_group.ex index 06e1564a..222307d0 100644 --- a/lib/kafka_ex/protocol/join_group.ex +++ b/lib/kafka_ex/protocol/join_group.ex @@ -27,6 +27,10 @@ defmodule KafkaEx.Protocol.JoinGroup do defstruct error_code: nil, generation_id: 0, leader_id: nil, member_id: nil, members: [] @type t :: %Response{error_code: atom | integer, generation_id: integer, leader_id: binary, member_id: binary, members: [binary]} + + def leader?(%__MODULE__{member_id: member_id, leader_id: leader_id}) do + member_id == leader_id + end end @spec create_request(integer, binary, Request.t) :: binary From 32fd4e531921c8291806fb1b693adab9ec53afea Mon Sep 17 00:00:00 2001 From: David Cuddeback Date: Wed, 10 May 2017 20:13:03 -0700 Subject: [PATCH 06/47] replace GenServer timeout with timer --- lib/kafka_ex/consumer_group.ex | 44 +++++++++++++++++++++++++++------- 1 file changed, 36 insertions(+), 8 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 5179556d..d4911971 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -83,6 +83,7 @@ defmodule KafkaEx.ConsumerGroup do :generation_id, :assignments, :consumer_pid, + :heartbeat_timer, ] end @@ -145,14 +146,14 @@ defmodule KafkaEx.ConsumerGroup do def handle_info(:timeout, %State{generation_id: nil, member_id: ""} = state) do {:ok, new_state} = join(state) - {:noreply, new_state, new_state.heartbeat_interval} + {:noreply, new_state} end # After joining the group, a member must periodically send heartbeats to the group coordinator. - def handle_info(:timeout, %State{} = state) do + def handle_info(:heartbeat, %State{} = state) do {:ok, new_state} = heartbeat(state) - {:noreply, new_state, new_state.heartbeat_interval} + {:noreply, new_state} end # Stop the consumer group if the consumer supervisor crashes. @@ -165,7 +166,7 @@ defmodule KafkaEx.ConsumerGroup do # Ignore spurious exit signals. These are often from old consumer PIDs, because the exit signal # can arrive after the `consumer_pid` state has already been updated. def handle_info({:EXIT, _pid, _reason}, %State{} = state) do - {:noreply, state, state.heartbeat_interval} + {:noreply, state} end # When terminating, inform the group coordinator that this member is leaving the group so that the @@ -236,7 +237,11 @@ defmodule KafkaEx.ConsumerGroup do case KafkaEx.sync_group(sync_request, worker_name: worker_name, timeout: session_timeout + @session_timeout_padding) do %SyncGroupResponse{error_code: :no_error, assignments: assignments} -> - start_consumer(state, unpack_assignments(assignments)) + new_state = state + |> start_consumer(unpack_assignments(assignments)) + |> start_heartbeat_timer() + + {:ok, new_state} %SyncGroupResponse{error_code: :rebalance_in_progress} -> rebalance(state) @@ -263,7 +268,9 @@ defmodule KafkaEx.ConsumerGroup do case KafkaEx.heartbeat(heartbeat_request, worker_name: worker_name) do %HeartbeatResponse{error_code: :no_error} -> - {:ok, state} + new_state = start_heartbeat_timer(state) + + {:ok, new_state} %HeartbeatResponse{error_code: :rebalance_in_progress} -> Logger.debug("Rebalancing consumer group #{group_name}") @@ -275,7 +282,9 @@ defmodule KafkaEx.ConsumerGroup do # is leaving the group without having to wait for the session timeout to expire. Leaving a group # triggers a rebalance for the remaining group members. defp leave(%State{worker_name: worker_name, group_name: group_name, member_id: member_id} = state) do - stop_consumer(state) + state + |> stop_heartbeat_timer() + |> stop_consumer() leave_request = %LeaveGroupRequest{ group_name: group_name, @@ -294,10 +303,28 @@ defmodule KafkaEx.ConsumerGroup do # member pauses its consumers and commits its offsets before rejoining the group. defp rebalance(%State{} = state) do state + |> stop_heartbeat_timer() |> stop_consumer() |> join() end + ### Timer Management + + # Starts a timer for the next heartbeat. + defp start_heartbeat_timer(%State{heartbeat_interval: heartbeat_interval} = state) do + {:ok, timer} = :timer.send_after(heartbeat_interval, :heartbeat) + + %State{state | heartbeat_timer: timer} + end + + # Stops any active heartbeat timer. + defp stop_heartbeat_timer(%State{heartbeat_timer: nil} = state), do: state + defp stop_heartbeat_timer(%State{heartbeat_timer: heartbeat_timer} = state) do + {:ok, :cancel} = :timer.cancel(heartbeat_timer) + + %State{state | heartbeat_timer: nil} + end + ### Consumer Management # Starts consuming from the member's assigned partitions. @@ -305,13 +332,14 @@ defmodule KafkaEx.ConsumerGroup do group_name: group_name, consumer_pid: nil} = state, assignments) do {:ok, pid} = KafkaEx.GenConsumer.Supervisor.start_link(consumer_module, group_name, assignments, consumer_opts) - {:ok, %State{state | assignments: assignments, consumer_pid: pid}} + %State{state | assignments: assignments, consumer_pid: pid} end # Stops consuming from the member's assigned partitions and commits offsets. defp stop_consumer(%State{consumer_pid: nil} = state), do: state defp stop_consumer(%State{consumer_pid: pid} = state) when is_pid(pid) do :ok = Supervisor.stop(pid) + %State{state | consumer_pid: nil} end From 6f40c7d43176ba79f8b0c1eba90700e74ba48196 Mon Sep 17 00:00:00 2001 From: David Cuddeback Date: Wed, 10 May 2017 20:36:10 -0700 Subject: [PATCH 07/47] adds Metadata.Response.partitions_for_topic/2 --- lib/kafka_ex/consumer_group.ex | 10 ++++------ lib/kafka_ex/protocol/metadata.ex | 6 ++++++ 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index d4911971..8eecf8bb 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -62,8 +62,6 @@ defmodule KafkaEx.ConsumerGroup do alias KafkaEx.Protocol.LeaveGroup.Request, as: LeaveGroupRequest alias KafkaEx.Protocol.LeaveGroup.Response, as: LeaveGroupResponse alias KafkaEx.Protocol.Metadata.Response, as: MetadataResponse - alias KafkaEx.Protocol.Metadata.TopicMetadata - alias KafkaEx.Protocol.Metadata.PartitionMetadata alias KafkaEx.Protocol.SyncGroup.Request, as: SyncGroupRequest alias KafkaEx.Protocol.SyncGroup.Response, as: SyncGroupResponse @@ -349,13 +347,13 @@ defmodule KafkaEx.ConsumerGroup do # group. This function returns a list of topic/partition tuples that can be passed to a # GenConsumer's `assign_partitions` method. defp assignable_partitions(%State{worker_name: worker_name, topics: topics}) do - %MetadataResponse{topic_metadatas: topic_metadatas} = KafkaEx.metadata(worker_name: worker_name) + metadata = KafkaEx.metadata(worker_name: worker_name) Enum.flat_map(topics, fn (topic) -> - %TopicMetadata{error_code: :no_error, partition_metadatas: partition_metadatas} = Enum.find(topic_metadatas, &(&1.topic == topic)) + partitions = MetadataResponse.partitions_for_topic(metadata, topic) - Enum.map(partition_metadatas, fn (%PartitionMetadata{error_code: :no_error, partition_id: partition_id}) -> - {topic, partition_id} + Enum.map(partitions, fn (partition) -> + {topic, partition} end) end) end diff --git a/lib/kafka_ex/protocol/metadata.ex b/lib/kafka_ex/protocol/metadata.ex index d8854f41..cdc7bcec 100644 --- a/lib/kafka_ex/protocol/metadata.ex +++ b/lib/kafka_ex/protocol/metadata.ex @@ -29,6 +29,12 @@ defmodule KafkaEx.Protocol.Metadata do end end + def partitions_for_topic(metadata, topic) do + topic_metadata = Enum.find(metadata.topic_metadatas, &(&1.topic == topic)) + + Enum.map(topic_metadata.partition_metadatas, &(&1.partition_id)) + end + defp find_lead_broker(metadata_brokers, topic_metadata, brokers, partition) do case Enum.find(topic_metadata.partition_metadatas, &(partition == &1.partition_id)) do nil -> nil From 76093888601b1645c61af21399d61ef7d478340d Mon Sep 17 00:00:00 2001 From: David Cuddeback Date: Tue, 23 May 2017 14:12:27 -0700 Subject: [PATCH 08/47] manage dynamic assignments with ConsumerGroup.Supervisor --- lib/kafka_ex/consumer_group.ex | 140 +++++++++++++++++++++++---------- 1 file changed, 100 insertions(+), 40 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 8eecf8bb..2aea1079 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -36,7 +36,7 @@ defmodule KafkaEx.ConsumerGroup do end # use DistributedConsumer in a consumer group - {:ok, pid} = KafkaEx.ConsumerGroup.start_link(DistributedConsumer, "test_group", ["test_topic"]) + {:ok, pid} = KafkaEx.ConsumerGroup.Supervisor.start_link(DistributedConsumer, "test_group", ["test_topic"]) ``` Running this on multiple nodes might display the following: @@ -67,9 +67,96 @@ defmodule KafkaEx.ConsumerGroup do require Logger + defmodule Supervisor do + @moduledoc """ + A supervisor for managing a consumer group. A `KafkaEx.ConsumerGroup.Supervisor` process will + manage an entire process tree for a single consumer group. Multiple supervisors can be used for + multiple consumer groups within the same application. + + ## Example + + This supervisor can be addeded to an application's supervision tree with a custom `GenConsumer` + implementation with the following child spec: + + ``` + supervisor(KafkaEx.ConsumerGroup.Supervisor, [MyApp.Consumer, "group_name", ["topic1", "topic2"]]) + ``` + """ + + use Elixir.Supervisor + + @typedoc """ + Option values used when starting a `ConsumerGroup.Supervisor`. + """ + @type option :: KafkaEx.GenConsumer.option + | {:name, Elixir.Supervisor.name} + | {:max_restarts, non_neg_integer} + | {:max_seconds, non_neg_integer} + + @typedoc """ + Options used when starting a `ConsumerGroup.Supervisor`. + """ + @type options :: [option] + + @doc """ + Starts a `ConsumerGroup.Supervisor` process linked to the current process. + + This can be used to start a `KafkaEx.ConsumerGroup` as part of a supervision tree. + + `module` is a module that implements the `KafkaEx.GenConsumer` behaviour. `group_name` is the + name of the consumer group. `topics` is a list of topics that the consumer group should consume + from. `opts` can be any options accepted by `KafkaEx.ConsumerGroup` or `Supervisor`. + + ### Return Values + + This function has the same return values as `Supervisor.start_link/3`. + + If the supervisor and consumer group are successfully created and initialized, this function + returns `{:ok, pid}`, where `pid` is the PID of the consumer group supervisor process. + """ + @spec start_link(module, binary, [binary], options) :: Elixir.Supervisor.on_start + def start_link(consumer_module, group_name, topics, opts \\ []) do + {supervisor_opts, module_opts} = Keyword.split(opts, [:name, :strategy, :max_restarts, :max_seconds]) + + Elixir.Supervisor.start_link(__MODULE__, {consumer_module, group_name, topics, module_opts}, supervisor_opts) + end + + @doc false # used by ConsumerGroup to set partition assignments + def start_consumer(pid, consumer_module, group_name, assignments, opts) do + child = supervisor(KafkaEx.GenConsumer.Supervisor, [consumer_module, group_name, assignments, opts], id: :consumer) + + case Elixir.Supervisor.start_child(pid, child) do + {:ok, _child} -> :ok + {:ok, _child, _info} -> :ok + end + end + + @doc false # used by ConsumerGroup to pause consumption during rebalance + def stop_consumer(pid) do + case Elixir.Supervisor.terminate_child(pid, :consumer) do + :ok -> + Elixir.Supervisor.delete_child(pid, :consumer) + + {:error, :not_found} -> + :ok + end + end + + def init({consumer_module, group_name, topics, opts}) do + opts = Keyword.put(opts, :supervisor_pid, self()) + + children = [ + worker(KafkaEx.ConsumerGroup, [consumer_module, group_name, topics, opts]), + ] + + supervise(children, strategy: :one_for_all) + end + end + defmodule State do @moduledoc false defstruct [ + :supervisor_pid, :worker_name, :heartbeat_interval, :session_timeout, @@ -80,7 +167,6 @@ defmodule KafkaEx.ConsumerGroup do :member_id, :generation_id, :assignments, - :consumer_pid, :heartbeat_timer, ] end @@ -92,20 +178,8 @@ defmodule KafkaEx.ConsumerGroup do # Client API @doc """ - Starts a `ConsumerGroup` process linked to the current process. - - This can be used to start a `ConsumerGroup` as part of a supervision tree. - - `module` is a module that implements the `KafkaEx.GenConsumer` behaviour. `group_name` is the name - of the consumer group. `topics` is a list of topics that the consumer group should consume from. - `opts` can be any options accepted by `GenConsumer` or `GenServer`. - - ### Return Values - - This function has the same return values as `GenServer.start_link/3`. - - If the consumer group is successfully created and initialized, this function returns `{:ok, pid}`, - where `pid` is the PID of the consumer group process. + Starts a `ConsumerGroup` process linked to the current process. Client programs should use + `KafkaEx.ConsumerGroup.Supervisor.start_link/4` instead. """ @spec start_link(module, binary, [binary], KafkaEx.GenConsumer.options) :: GenServer.on_start def start_link(consumer_module, group_name, topics, opts \\ []) do @@ -121,9 +195,11 @@ defmodule KafkaEx.ConsumerGroup do heartbeat_interval = Keyword.get(opts, :heartbeat_interval, Application.get_env(:kafka_ex, :heartbeat_interval, @heartbeat_interval)) session_timeout = Keyword.get(opts, :session_timeout, Application.get_env(:kafka_ex, :session_timeout, @session_timeout)) - consumer_opts = Keyword.drop(opts, [:heartbeat_interval, :session_timeout]) + supervisor_pid = Keyword.fetch!(opts, :supervisor_pid) + consumer_opts = Keyword.drop(opts, [:supervisor_pid, :heartbeat_interval, :session_timeout]) state = %State{ + supervisor_pid: supervisor_pid, worker_name: worker_name, heartbeat_interval: heartbeat_interval, session_timeout: session_timeout, @@ -154,19 +230,6 @@ defmodule KafkaEx.ConsumerGroup do {:noreply, new_state} end - # Stop the consumer group if the consumer supervisor crashes. - def handle_info({:EXIT, pid, reason}, %State{consumer_pid: pid} = state) do - new_state = %State{state | consumer_pid: nil} - - {:stop, reason, new_state} - end - - # Ignore spurious exit signals. These are often from old consumer PIDs, because the exit signal - # can arrive after the `consumer_pid` state has already been updated. - def handle_info({:EXIT, _pid, _reason}, %State{} = state) do - {:noreply, state} - end - # When terminating, inform the group coordinator that this member is leaving the group so that the # group can rebalance without waiting for a session timeout. def terminate(_reason, %State{generation_id: nil, member_id: ""}), do: :ok @@ -280,9 +343,7 @@ defmodule KafkaEx.ConsumerGroup do # is leaving the group without having to wait for the session timeout to expire. Leaving a group # triggers a rebalance for the remaining group members. defp leave(%State{worker_name: worker_name, group_name: group_name, member_id: member_id} = state) do - state - |> stop_heartbeat_timer() - |> stop_consumer() + stop_heartbeat_timer(state) leave_request = %LeaveGroupRequest{ group_name: group_name, @@ -327,18 +388,17 @@ defmodule KafkaEx.ConsumerGroup do # Starts consuming from the member's assigned partitions. defp start_consumer(%State{consumer_module: consumer_module, consumer_opts: consumer_opts, - group_name: group_name, consumer_pid: nil} = state, assignments) do - {:ok, pid} = KafkaEx.GenConsumer.Supervisor.start_link(consumer_module, group_name, assignments, consumer_opts) + group_name: group_name, supervisor_pid: pid} = state, assignments) do + :ok = KafkaEx.ConsumerGroup.Supervisor.start_consumer(pid, consumer_module, group_name, assignments, consumer_opts) - %State{state | assignments: assignments, consumer_pid: pid} + state end # Stops consuming from the member's assigned partitions and commits offsets. - defp stop_consumer(%State{consumer_pid: nil} = state), do: state - defp stop_consumer(%State{consumer_pid: pid} = state) when is_pid(pid) do - :ok = Supervisor.stop(pid) + defp stop_consumer(%State{supervisor_pid: pid} = state) do + :ok = KafkaEx.ConsumerGroup.Supervisor.stop_consumer(pid) - %State{state | consumer_pid: nil} + state end ### Partition Assignment From 14793e310bf721eee9548886e85814beeae4652a Mon Sep 17 00:00:00 2001 From: David Cuddeback Date: Tue, 23 May 2017 14:33:03 -0700 Subject: [PATCH 09/47] improve readability of deeply-nested GenConsumer.Supervisor.start_link/4 --- lib/kafka_ex/gen_consumer.ex | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/lib/kafka_ex/gen_consumer.ex b/lib/kafka_ex/gen_consumer.ex index 2f1c4ced..6297a027 100644 --- a/lib/kafka_ex/gen_consumer.ex +++ b/lib/kafka_ex/gen_consumer.ex @@ -273,13 +273,7 @@ defmodule KafkaEx.GenConsumer do def start_link(consumer_module, group_name, assignments, opts \\ []) do case Elixir.Supervisor.start_link(__MODULE__, {consumer_module, group_name, assignments, opts}) do {:ok, pid} -> - Enum.each(assignments, fn ({topic, partition}) -> - case Elixir.Supervisor.start_child(pid, [topic, partition, opts]) do - {:ok, _child} -> nil - {:ok, _child, _info} -> nil - end - end) - + :ok = start_workers(pid, assignments, opts) {:ok, pid} error -> @@ -294,6 +288,17 @@ defmodule KafkaEx.GenConsumer do supervise(children, strategy: :simple_one_for_one) end + + defp start_workers(pid, assignments, opts) do + Enum.each(assignments, fn ({topic, partition}) -> + case Elixir.Supervisor.start_child(pid, [topic, partition, opts]) do + {:ok, _child} -> nil + {:ok, _child, _info} -> nil + end + end) + + :ok + end end defmodule State do From 9d0b4cc7af0e840863685a3005eaaafb7b961254 Mon Sep 17 00:00:00 2001 From: David Cuddeback Date: Tue, 23 May 2017 14:36:46 -0700 Subject: [PATCH 10/47] starts KafkaEx.Server per consumer --- lib/kafka_ex/consumer_group.ex | 4 ++-- lib/kafka_ex/gen_consumer.ex | 8 ++------ 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 2aea1079..629ce7e4 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -54,7 +54,6 @@ defmodule KafkaEx.ConsumerGroup do use GenServer - alias KafkaEx.Config alias KafkaEx.Protocol.JoinGroup.Request, as: JoinGroupRequest alias KafkaEx.Protocol.JoinGroup.Response, as: JoinGroupResponse alias KafkaEx.Protocol.Heartbeat.Request, as: HeartbeatRequest @@ -191,13 +190,14 @@ defmodule KafkaEx.ConsumerGroup do # GenServer callbacks def init({consumer_module, group_name, topics, opts}) do - worker_name = Keyword.get(opts, :worker_name, Config.default_worker) heartbeat_interval = Keyword.get(opts, :heartbeat_interval, Application.get_env(:kafka_ex, :heartbeat_interval, @heartbeat_interval)) session_timeout = Keyword.get(opts, :session_timeout, Application.get_env(:kafka_ex, :session_timeout, @session_timeout)) supervisor_pid = Keyword.fetch!(opts, :supervisor_pid) consumer_opts = Keyword.drop(opts, [:supervisor_pid, :heartbeat_interval, :session_timeout]) + {:ok, worker_name} = KafkaEx.create_worker(:no_name, consumer_group: group_name) + state = %State{ supervisor_pid: supervisor_pid, worker_name: worker_name, diff --git a/lib/kafka_ex/gen_consumer.ex b/lib/kafka_ex/gen_consumer.ex index 6297a027..61d24ce1 100644 --- a/lib/kafka_ex/gen_consumer.ex +++ b/lib/kafka_ex/gen_consumer.ex @@ -109,7 +109,6 @@ defmodule KafkaEx.GenConsumer do use GenServer - alias KafkaEx.Config alias KafkaEx.Protocol.OffsetCommit.Request, as: OffsetCommitRequest alias KafkaEx.Protocol.OffsetCommit.Response, as: OffsetCommitResponse alias KafkaEx.Protocol.OffsetFetch.Request, as: OffsetFetchRequest @@ -143,8 +142,7 @@ defmodule KafkaEx.GenConsumer do @typedoc """ Option values used when starting a `GenConsumer`. """ - @type option :: {:worker_name, atom | pid} - | {:commit_interval, non_neg_integer} + @type option :: {:commit_interval, non_neg_integer} | {:commit_threshold, non_neg_integer} @typedoc """ @@ -339,8 +337,6 @@ defmodule KafkaEx.GenConsumer do acknowledged messages. If not present, the `:commit_interval` environment value is used. * `:commit_threshold` - the maximum number of messages that can be acknowledged without being committed. If not present, the `:commit_threshold` environment value is used. - * `:worker_name` - the name of the `KafkaEx.Server` process to use for communicating with the - Kafka brokers. If not present, the default worker is used. Any valid options for `GenServer.start_link/3` can also be specified. @@ -361,11 +357,11 @@ defmodule KafkaEx.GenConsumer do # GenServer callbacks def init({consumer_module, group_name, topic, partition, opts}) do - worker_name = Keyword.get(opts, :worker_name, Config.default_worker) commit_interval = Keyword.get(opts, :commit_interval, Application.get_env(:kafka_ex, :commit_interval, @commit_interval)) commit_threshold = Keyword.get(opts, :commit_threshold, Application.get_env(:kafka_ex, :commit_threshold, @commit_threshold)) {:ok, consumer_state} = consumer_module.init(topic, partition) + {:ok, worker_name} = KafkaEx.create_worker(:no_name, consumer_group: group_name) state = %State{ consumer_module: consumer_module, From 46a8054243e077efb6e4af202e294d408a03908b Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Tue, 25 Jul 2017 17:07:19 -0400 Subject: [PATCH 11/47] Unbreak merge issues --- lib/kafka_ex/server.ex | 2 +- lib/kafka_ex/server_0_p_9_p_0.ex | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/lib/kafka_ex/server.ex b/lib/kafka_ex/server.ex index f7df10a3..f06bf238 100644 --- a/lib/kafka_ex/server.ex +++ b/lib/kafka_ex/server.ex @@ -403,7 +403,7 @@ defmodule KafkaEx.Server do end) end - defp sync_timeout(timeout \\ nil) do + defp config_sync_timeout(timeout \\ nil) do timeout || Application.get_env(:kafka_ex, :sync_timeout, @sync_timeout) end end diff --git a/lib/kafka_ex/server_0_p_9_p_0.ex b/lib/kafka_ex/server_0_p_9_p_0.ex index 9d0e1baa..332dc266 100644 --- a/lib/kafka_ex/server_0_p_9_p_0.ex +++ b/lib/kafka_ex/server_0_p_9_p_0.ex @@ -76,7 +76,7 @@ defmodule KafkaEx.Server0P9P0 do {broker, state} = broker_for_consumer_group_with_update(state) request = JoinGroup.create_request(state.correlation_id, @client_id, join_group_request) response = broker - |> NetworkClient.send_sync_request(request, sync_timeout(network_timeout)) + |> NetworkClient.send_sync_request(request, config_sync_timeout(network_timeout)) |> JoinGroup.parse_response {:reply, response, %{state | correlation_id: state.correlation_id + 1}} end @@ -86,7 +86,7 @@ defmodule KafkaEx.Server0P9P0 do {broker, state} = broker_for_consumer_group_with_update(state) request = SyncGroup.create_request(state.correlation_id, @client_id, sync_group_request) response = broker - |> NetworkClient.send_sync_request(request, sync_timeout(network_timeout)) + |> NetworkClient.send_sync_request(request, config_sync_timeout(network_timeout)) |> SyncGroup.parse_response {:reply, response, %{state | correlation_id: state.correlation_id + 1}} end @@ -96,7 +96,7 @@ defmodule KafkaEx.Server0P9P0 do {broker, state} = broker_for_consumer_group_with_update(state) request = LeaveGroup.create_request(state.correlation_id, @client_id, request) response = broker - |> NetworkClient.send_sync_request(request, sync_timeout(network_timeout)) + |> NetworkClient.send_sync_request(request, config_sync_timeout(network_timeout)) |> LeaveGroup.parse_response {:reply, response, %{state | correlation_id: state.correlation_id + 1}} end @@ -106,7 +106,7 @@ defmodule KafkaEx.Server0P9P0 do {broker, state} = broker_for_consumer_group_with_update(state) request = Heartbeat.create_request(state.correlation_id, @client_id, request) response = broker - |> NetworkClient.send_sync_request(request, sync_timeout(network_timeout)) + |> NetworkClient.send_sync_request(request, config_sync_timeout(network_timeout)) |> Heartbeat.parse_response {:reply, response, %{state | correlation_id: state.correlation_id + 1}} end From d16c15c363a57dc999b66976a7160fa88988019a Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Wed, 26 Jul 2017 11:00:15 -0400 Subject: [PATCH 12/47] Thunk logger calls --- lib/kafka_ex/consumer_group.ex | 8 +++----- lib/kafka_ex/gen_consumer.ex | 4 ++-- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 629ce7e4..7879fa8e 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -261,7 +261,7 @@ defmodule KafkaEx.ConsumerGroup do join_response = %JoinGroupResponse{error_code: :no_error} = KafkaEx.join_group(join_request, worker_name: worker_name, timeout: session_timeout + @session_timeout_padding) - Logger.debug("Joined consumer group #{group_name}") + Logger.debug(fn -> "Joined consumer group #{group_name}" end) new_state = %State{state | member_id: join_response.member_id, generation_id: join_response.generation_id} @@ -330,11 +330,9 @@ defmodule KafkaEx.ConsumerGroup do case KafkaEx.heartbeat(heartbeat_request, worker_name: worker_name) do %HeartbeatResponse{error_code: :no_error} -> new_state = start_heartbeat_timer(state) - {:ok, new_state} - %HeartbeatResponse{error_code: :rebalance_in_progress} -> - Logger.debug("Rebalancing consumer group #{group_name}") + Logger.debug(fn -> "Rebalancing consumer group #{group_name}" end) rebalance(state) end end @@ -352,7 +350,7 @@ defmodule KafkaEx.ConsumerGroup do %LeaveGroupResponse{error_code: :no_error} = KafkaEx.leave_group(leave_request, worker_name: worker_name) - Logger.debug("Left consumer group #{group_name}") + Logger.debug(fn -> "Left consumer group #{group_name}" end) :ok end diff --git a/lib/kafka_ex/gen_consumer.ex b/lib/kafka_ex/gen_consumer.ex index 61d24ce1..a9cfaeda 100644 --- a/lib/kafka_ex/gen_consumer.ex +++ b/lib/kafka_ex/gen_consumer.ex @@ -24,7 +24,7 @@ defmodule KafkaEx.GenConsumer do require Logger def handle_message(%Message{value: message}, state) do - Logger.debug("message: " <> inspect(message)) + Logger.debug(fn -> "message: " <> inspect(message) end) {:ack, state} end end @@ -450,7 +450,7 @@ defmodule KafkaEx.GenConsumer do [%OffsetCommitResponse{topic: ^topic, partitions: [^partition]}] = KafkaEx.offset_commit(worker_name, request) - Logger.debug("Committed offset #{topic}/#{partition}@#{offset} for #{group}") + Logger.debug(fn -> "Committed offset #{topic}/#{partition}@#{offset} for #{group}" end) %State{state | committed_offset: offset, last_commit: :erlang.monotonic_time(:milli_seconds)} end From eb3e1e3fe409295fe22edbca3826034dcc6b2e56 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Wed, 26 Jul 2017 11:15:37 -0400 Subject: [PATCH 13/47] Move supervisors to subdirectories --- lib/kafka_ex/consumer_group.ex | 86 ----------------------- lib/kafka_ex/consumer_group/supervisor.ex | 85 ++++++++++++++++++++++ lib/kafka_ex/gen_consumer.ex | 60 ---------------- lib/kafka_ex/gen_consumer/supervisor.ex | 59 ++++++++++++++++ 4 files changed, 144 insertions(+), 146 deletions(-) create mode 100644 lib/kafka_ex/consumer_group/supervisor.ex create mode 100644 lib/kafka_ex/gen_consumer/supervisor.ex diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 7879fa8e..0112fa2e 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -66,92 +66,6 @@ defmodule KafkaEx.ConsumerGroup do require Logger - defmodule Supervisor do - @moduledoc """ - A supervisor for managing a consumer group. A `KafkaEx.ConsumerGroup.Supervisor` process will - manage an entire process tree for a single consumer group. Multiple supervisors can be used for - multiple consumer groups within the same application. - - ## Example - - This supervisor can be addeded to an application's supervision tree with a custom `GenConsumer` - implementation with the following child spec: - - ``` - supervisor(KafkaEx.ConsumerGroup.Supervisor, [MyApp.Consumer, "group_name", ["topic1", "topic2"]]) - ``` - """ - - use Elixir.Supervisor - - @typedoc """ - Option values used when starting a `ConsumerGroup.Supervisor`. - """ - @type option :: KafkaEx.GenConsumer.option - | {:name, Elixir.Supervisor.name} - | {:max_restarts, non_neg_integer} - | {:max_seconds, non_neg_integer} - - @typedoc """ - Options used when starting a `ConsumerGroup.Supervisor`. - """ - @type options :: [option] - - @doc """ - Starts a `ConsumerGroup.Supervisor` process linked to the current process. - - This can be used to start a `KafkaEx.ConsumerGroup` as part of a supervision tree. - - `module` is a module that implements the `KafkaEx.GenConsumer` behaviour. `group_name` is the - name of the consumer group. `topics` is a list of topics that the consumer group should consume - from. `opts` can be any options accepted by `KafkaEx.ConsumerGroup` or `Supervisor`. - - ### Return Values - - This function has the same return values as `Supervisor.start_link/3`. - - If the supervisor and consumer group are successfully created and initialized, this function - returns `{:ok, pid}`, where `pid` is the PID of the consumer group supervisor process. - """ - @spec start_link(module, binary, [binary], options) :: Elixir.Supervisor.on_start - def start_link(consumer_module, group_name, topics, opts \\ []) do - {supervisor_opts, module_opts} = Keyword.split(opts, [:name, :strategy, :max_restarts, :max_seconds]) - - Elixir.Supervisor.start_link(__MODULE__, {consumer_module, group_name, topics, module_opts}, supervisor_opts) - end - - @doc false # used by ConsumerGroup to set partition assignments - def start_consumer(pid, consumer_module, group_name, assignments, opts) do - child = supervisor(KafkaEx.GenConsumer.Supervisor, [consumer_module, group_name, assignments, opts], id: :consumer) - - case Elixir.Supervisor.start_child(pid, child) do - {:ok, _child} -> :ok - {:ok, _child, _info} -> :ok - end - end - - @doc false # used by ConsumerGroup to pause consumption during rebalance - def stop_consumer(pid) do - case Elixir.Supervisor.terminate_child(pid, :consumer) do - :ok -> - Elixir.Supervisor.delete_child(pid, :consumer) - - {:error, :not_found} -> - :ok - end - end - - def init({consumer_module, group_name, topics, opts}) do - opts = Keyword.put(opts, :supervisor_pid, self()) - - children = [ - worker(KafkaEx.ConsumerGroup, [consumer_module, group_name, topics, opts]), - ] - - supervise(children, strategy: :one_for_all) - end - end - defmodule State do @moduledoc false defstruct [ diff --git a/lib/kafka_ex/consumer_group/supervisor.ex b/lib/kafka_ex/consumer_group/supervisor.ex new file mode 100644 index 00000000..e1d070d7 --- /dev/null +++ b/lib/kafka_ex/consumer_group/supervisor.ex @@ -0,0 +1,85 @@ +defmodule KafkaEx.ConsumerGroup.Supervisor do + @moduledoc """ + A supervisor for managing a consumer group. A `KafkaEx.ConsumerGroup.Supervisor` process will + manage an entire process tree for a single consumer group. Multiple supervisors can be used for + multiple consumer groups within the same application. + + ## Example + + This supervisor can be addeded to an application's supervision tree with a custom `GenConsumer` + implementation with the following child spec: + + ``` + supervisor(KafkaEx.ConsumerGroup.Supervisor, [MyApp.Consumer, "group_name", ["topic1", "topic2"]]) + ``` + """ + + use Elixir.Supervisor + + @typedoc """ + Option values used when starting a `ConsumerGroup.Supervisor`. + """ + @type option :: KafkaEx.GenConsumer.option + | {:name, Elixir.Supervisor.name} + | {:max_restarts, non_neg_integer} + | {:max_seconds, non_neg_integer} + + @typedoc """ + Options used when starting a `ConsumerGroup.Supervisor`. + """ + @type options :: [option] + + @doc """ + Starts a `ConsumerGroup.Supervisor` process linked to the current process. + + This can be used to start a `KafkaEx.ConsumerGroup` as part of a supervision tree. + + `module` is a module that implements the `KafkaEx.GenConsumer` behaviour. `group_name` is the + name of the consumer group. `topics` is a list of topics that the consumer group should consume + from. `opts` can be any options accepted by `KafkaEx.ConsumerGroup` or `Supervisor`. + + ### Return Values + + This function has the same return values as `Supervisor.start_link/3`. + + If the supervisor and consumer group are successfully created and initialized, this function + returns `{:ok, pid}`, where `pid` is the PID of the consumer group supervisor process. + """ + @spec start_link(module, binary, [binary], options) :: Elixir.Supervisor.on_start + def start_link(consumer_module, group_name, topics, opts \\ []) do + {supervisor_opts, module_opts} = Keyword.split(opts, [:name, :strategy, :max_restarts, :max_seconds]) + + Elixir.Supervisor.start_link(__MODULE__, {consumer_module, group_name, topics, module_opts}, supervisor_opts) + end + + @doc false # used by ConsumerGroup to set partition assignments + def start_consumer(pid, consumer_module, group_name, assignments, opts) do + child = supervisor(KafkaEx.GenConsumer.Supervisor, [consumer_module, group_name, assignments, opts], id: :consumer) + + case Elixir.Supervisor.start_child(pid, child) do + {:ok, _child} -> :ok + {:ok, _child, _info} -> :ok + end + end + + @doc false # used by ConsumerGroup to pause consumption during rebalance + def stop_consumer(pid) do + case Elixir.Supervisor.terminate_child(pid, :consumer) do + :ok -> + Elixir.Supervisor.delete_child(pid, :consumer) + + {:error, :not_found} -> + :ok + end + end + + def init({consumer_module, group_name, topics, opts}) do + opts = Keyword.put(opts, :supervisor_pid, self()) + + children = [ + worker(KafkaEx.ConsumerGroup, [consumer_module, group_name, topics, opts]), + ] + + supervise(children, strategy: :one_for_all) + end +end diff --git a/lib/kafka_ex/gen_consumer.ex b/lib/kafka_ex/gen_consumer.ex index a9cfaeda..5821bd7d 100644 --- a/lib/kafka_ex/gen_consumer.ex +++ b/lib/kafka_ex/gen_consumer.ex @@ -239,66 +239,6 @@ defmodule KafkaEx.GenConsumer do end end - defmodule Supervisor do - @moduledoc """ - A supervisor for managing `GenConsumer` processes that are part of a consumer group. - - The supervisor will launch individual `GenConsumer` processes for each partition given by the - `partitions` argument to `start_link/4`. When terminated, each of the supervisor's child - processes will commit its latest offset before terminating. - - This module manages a static list of consumer processes. For dynamically distributing consumers - in a consumer group across a cluster of nodes, see `KafkaEx.ConsumerGroup`. - """ - - use Elixir.Supervisor - - @doc """ - Starts a `GenConsumer.Supervisor` process linked to the current process. - - `module` is a module that implements the `GenConsumer` behaviour. `group_name` is the name of a - consumer group, and `assignments` is a list of partitions for the `GenConsumer`s to consume. - `opts` accepts the same options as `KafkaEx.GenConsumer.start_link/5`. - - ### Return Values - - This function has the same return values as `Supervisor.start_link/3`. - - If the supervisor and its consumers are successfully created, this function returns `{:ok, - pid}`, where `pid` is the PID of the supervisor. - """ - @spec start_link(module, binary, [KafkaEx.GenConsumer.partition], KafkaEx.GenConsumer.options) :: Elixir.Supervisor.on_start - def start_link(consumer_module, group_name, assignments, opts \\ []) do - case Elixir.Supervisor.start_link(__MODULE__, {consumer_module, group_name, assignments, opts}) do - {:ok, pid} -> - :ok = start_workers(pid, assignments, opts) - {:ok, pid} - - error -> - error - end - end - - def init({consumer_module, group_name, _assignments, _opts}) do - children = [ - worker(KafkaEx.GenConsumer, [consumer_module, group_name]) - ] - - supervise(children, strategy: :simple_one_for_one) - end - - defp start_workers(pid, assignments, opts) do - Enum.each(assignments, fn ({topic, partition}) -> - case Elixir.Supervisor.start_child(pid, [topic, partition, opts]) do - {:ok, _child} -> nil - {:ok, _child, _info} -> nil - end - end) - - :ok - end - end - defmodule State do @moduledoc false defstruct [ diff --git a/lib/kafka_ex/gen_consumer/supervisor.ex b/lib/kafka_ex/gen_consumer/supervisor.ex new file mode 100644 index 00000000..e127ae47 --- /dev/null +++ b/lib/kafka_ex/gen_consumer/supervisor.ex @@ -0,0 +1,59 @@ +defmodule KafkaEx.GenConsumer.Supervisor do + @moduledoc """ + A supervisor for managing `GenConsumer` processes that are part of a consumer group. + + The supervisor will launch individual `GenConsumer` processes for each partition given by the + `partitions` argument to `start_link/4`. When terminated, each of the supervisor's child + processes will commit its latest offset before terminating. + + This module manages a static list of consumer processes. For dynamically distributing consumers + in a consumer group across a cluster of nodes, see `KafkaEx.ConsumerGroup`. + """ + + use Elixir.Supervisor + + @doc """ + Starts a `GenConsumer.Supervisor` process linked to the current process. + + `module` is a module that implements the `GenConsumer` behaviour. `group_name` is the name of a + consumer group, and `assignments` is a list of partitions for the `GenConsumer`s to consume. + `opts` accepts the same options as `KafkaEx.GenConsumer.start_link/5`. + + ### Return Values + + This function has the same return values as `Supervisor.start_link/3`. + + If the supervisor and its consumers are successfully created, this function returns `{:ok, + pid}`, where `pid` is the PID of the supervisor. + """ + @spec start_link(module, binary, [KafkaEx.GenConsumer.partition], KafkaEx.GenConsumer.options) :: Elixir.Supervisor.on_start + def start_link(consumer_module, group_name, assignments, opts \\ []) do + case Elixir.Supervisor.start_link(__MODULE__, {consumer_module, group_name, assignments, opts}) do + {:ok, pid} -> + :ok = start_workers(pid, assignments, opts) + {:ok, pid} + + error -> + error + end + end + + def init({consumer_module, group_name, _assignments, _opts}) do + children = [ + worker(KafkaEx.GenConsumer, [consumer_module, group_name]) + ] + + supervise(children, strategy: :simple_one_for_one) + end + + defp start_workers(pid, assignments, opts) do + Enum.each(assignments, fn ({topic, partition}) -> + case Elixir.Supervisor.start_child(pid, [topic, partition, opts]) do + {:ok, _child} -> nil + {:ok, _child, _info} -> nil + end + end) + + :ok + end +end From bbf26951b62ebf8f133f6ca09245b31fe096759b Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Thu, 27 Jul 2017 19:24:10 -0400 Subject: [PATCH 14/47] Rename :ack / :commit to :async_commit / :sync_commit --- lib/kafka_ex/consumer_group.ex | 2 +- lib/kafka_ex/gen_consumer.ex | 22 +++++++++++----------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 0112fa2e..4712c32d 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -31,7 +31,7 @@ defmodule KafkaEx.ConsumerGroup do def handle_message(%Message{value: message}, state) do IO.puts(to_string(node()) <> ": " <> inspect(message)) - {:ack, state} + {:async_commit, state} end end diff --git a/lib/kafka_ex/gen_consumer.ex b/lib/kafka_ex/gen_consumer.ex index 5821bd7d..6896099e 100644 --- a/lib/kafka_ex/gen_consumer.ex +++ b/lib/kafka_ex/gen_consumer.ex @@ -25,19 +25,19 @@ defmodule KafkaEx.GenConsumer do def handle_message(%Message{value: message}, state) do Logger.debug(fn -> "message: " <> inspect(message) end) - {:ack, state} + {:async_commit, state} end end ``` `c:handle_message/2` will be called for each message that's fetched from a Kafka broker. In this - example, since `c:handle_message/2` always returns `{:ack, new_state}`, the message offsets will + example, since `c:handle_message/2` always returns `{:async_commit, new_state}`, the message offsets will be auto-committed. ## Auto-Committing Offsets `GenConsumer` manages a consumer's offsets by committing the offsets of acknowledged messages. - Messages are acknowledged by returning `{:ack, new_state}` from `c:handle_message/2`. + Messages are acknowledged by returning `{:async_commit, new_state}` from `c:handle_message/2`. Acknowledged messages are not committed immediately. To avoid excessive network calls, acknowledged messages may be batched and committed periodically. Offsets are also committed when a `GenServer` is terminated. @@ -101,7 +101,7 @@ defmodule KafkaEx.GenConsumer do test "it acks a message", %{state: state} do message = %Message{offset: 0, value: "hello"} {response, _new_state} = ExampleGenConsumer.handle_message(message, state) - assert response == :ack + assert response == :async_commit end end ``` @@ -170,18 +170,18 @@ defmodule KafkaEx.GenConsumer do `message` is a message fetched from a Kafka broker and `state` is the current state of the `GenConsumer`. - Returning `{:ack, new_state}` acknowledges `message` and continues to consume from the Kafka queue + Returning `{:async_commit, new_state}` acknowledges `message` and continues to consume from the Kafka queue with new state `new_state`. Acknowledged messages will be auto-committed (possibly at a later time) based on the `:commit_interval` and `:commit_threshold` options. - Returning `{:commit, new_state}` commits `message` synchronously before continuing to consume from + Returning `{:sync_commit, new_state}` commits `message` synchronously before continuing to consume from the Kafka queue with new state `new_state`. Committing a message synchronously means that no more - messages will be consumed until the message's offset is committed. `:commit` should be used + messages will be consumed until the message's offset is committed. `:sync_commit` should be used sparingly, since committing every message synchronously would impact a consumer's performance and could result in excessive network traffic. """ - @callback handle_message(message :: Message.t, state :: term) :: {:ack, new_state :: term} - | {:commit, new_state :: term} + @callback handle_message(message :: Message.t, state :: term) :: {:async_commit, new_state :: term} + | {:sync_commit, new_state :: term} @doc """ Invoked to determine partition assignments for a coordinated consumer group. @@ -352,10 +352,10 @@ defmodule KafkaEx.GenConsumer do defp handle_message(%Message{offset: offset} = message, %State{consumer_module: consumer_module, consumer_state: consumer_state} = state) do case consumer_module.handle_message(message, consumer_state) do - {:ack, new_state} -> + {:async_commit, new_state} -> auto_commit %State{state | consumer_state: new_state, acked_offset: offset + 1, current_offset: offset + 1} - {:commit, new_state} -> + {:sync_commit, new_state} -> commit %State{state | consumer_state: new_state, acked_offset: offset + 1, current_offset: offset + 1} end end From 5b85bb4d385f315a6247b93148cfdfae97330b1d Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Fri, 28 Jul 2017 13:48:07 -0400 Subject: [PATCH 15/47] Have GenConsumer handle message sets i.e., instead of `handle_message/2` we have `handle_message_set/2` It's very easy for the user to iterate over the batch, and there is at least one use case for passing batches to the consumer: parallelized processing of messages. At Simpli.fi we have some applications where each message we receive from Kafka some amount of work that can be done asynchronously, and we've found a big performance increase by processing those in parallel. There are some delivery guarantee implications to that, but it should be up to the user to decide how to handle that. --- lib/kafka_ex/consumer_group.ex | 9 +- lib/kafka_ex/gen_consumer.ex | 152 ++++++++++++++++++++++----------- 2 files changed, 107 insertions(+), 54 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 4712c32d..3646bf1f 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -29,8 +29,13 @@ defmodule KafkaEx.ConsumerGroup do defmodule DistributedConsumer do use KafkaEx.GenConsumer - def handle_message(%Message{value: message}, state) do - IO.puts(to_string(node()) <> ": " <> inspect(message)) + alias KafkaEx.Protocol.Fetch.Message + + # note - messages are delivered in batches + def handle_message_set(message_set, state) do + for %Message{value: message} <- message_set do + IO.puts(to_string(node()) <> ": " <> inspect(message)) + end {:async_commit, state} end end diff --git a/lib/kafka_ex/gen_consumer.ex b/lib/kafka_ex/gen_consumer.ex index 6896099e..3b4ae930 100644 --- a/lib/kafka_ex/gen_consumer.ex +++ b/lib/kafka_ex/gen_consumer.ex @@ -13,7 +13,7 @@ defmodule KafkaEx.GenConsumer do The `GenConsumer` behaviour abstracts common Kafka consumer interactions. `GenConsumer` will take care of the details of determining a starting offset, fetching messages from a Kafka broker, and committing offsets for consumed messages. Developers are only required to implement - `c:handle_message/2` to process messages from the queue. + `c:handle_message_set/2` to process message sets. The following is a minimal example that logs each message as it's consumed: @@ -21,30 +21,60 @@ defmodule KafkaEx.GenConsumer do defmodule ExampleGenConsumer do use KafkaEx.GenConsumer + alias KafkaEx.Protocol.Fetch.Message + require Logger - def handle_message(%Message{value: message}, state) do - Logger.debug(fn -> "message: " <> inspect(message) end) + # note - messages are delivered in batches + def handle_message_set(message_set, state) do + for %Message{value: message} <- message_set do + Logger.debug(fn -> "message: " <> inspect(message) end) + end {:async_commit, state} end end ``` - `c:handle_message/2` will be called for each message that's fetched from a Kafka broker. In this - example, since `c:handle_message/2` always returns `{:async_commit, new_state}`, the message offsets will - be auto-committed. + `c:handle_message_set/2` will be called with the batch of messages fetched + from the broker. The number of messages in a batch is determined by the + number of messages available and the `max_bytes` and `min_bytes` parameters + of the fetch request. In this example, because `c:handle_message_set/2` + always returns `{:async_commit, new_state}`, the message offsets will be + automatically committed asynchronously. + + ## Committing Offsets + + `GenConsumer` manages a consumer's offsets by committing the the offsets + of consumed messages. KafkaEx supports two commit strategies: asynchronous + and synchronous. The return value of `c:handle_message_set/2` determines + which strategy is used: + + * `{:sync_commit, new_state}` causes synchronous offset commits. + * `{:async_commit, new_state}` causes asynchronous offset commits. + + ### Synchronous offset commits + + When `c:handle_message_set/2` returns `{:sync_commit, new_state}`, the offset + of the final message in the batch is committed immediately before consuming + any more messages. This strategy causes significantly more communication + with the broker and will correspondingly degrade performance, but it will + keep the offset commits tightly synchronized with the consumer state. - ## Auto-Committing Offsets + Choose the synchronous offset commit strategy if you want to favor + consistency of offset commits over performance, or if you have a low rate of + message arrival (example: tens of messages per second or less). - `GenConsumer` manages a consumer's offsets by committing the offsets of acknowledged messages. - Messages are acknowledged by returning `{:async_commit, new_state}` from `c:handle_message/2`. - Acknowledged messages are not committed immediately. To avoid excessive network calls, - acknowledged messages may be batched and committed periodically. Offsets are also committed when a - `GenServer` is terminated. + ### Asynchronous offset commits - How often a `GenConsumer` auto-commits offsets is controlled by the two configuration values - `:commit_interval` and `:commit_threshold`. These can be set globally in the `:kafka_ex` app's - environment or on a per-consumer basis by passing options to `start_link/5`: + When `c:handle_message_set/2` returns `{:async_commit, new_state}`, KafkaEx + will not commit offsets after every message batch consumed. To avoid + excessive network calls, the offsets are committed periodically (and when + the worker terminates). + + How often a `GenConsumer` auto-commits offsets is controlled by the two + configuration values `:commit_interval` and `:commit_threshold`. These can be + set globally in the `:kafka_ex` app's environment or on a per-consumer basis + by passing options to `start_link/5`: ``` # In config/config.exs @@ -58,31 +88,34 @@ defmodule KafkaEx.GenConsumer do commit_threshold: 100) ``` - * `:commit_interval` is the maximum time (in milliseconds) that a `GenConsumer` will delay - committing the offset for an acknowledged message. - * `:commit_threshold` is the maximum number of acknowledged messages that a `GenConsumer` will - allow to be uncommitted before triggering an auto-commit. + * `:commit_interval` is the maximum time (in milliseconds) that a + `GenConsumer` will delay committing the offset for an acknowledged message. + * `:commit_threshold` is the maximum number of acknowledged messages that a + `GenConsumer` will allow to be uncommitted before triggering an auto-commit. - For low-volume topics, `:commit_interval` is the dominant factor for how often a `GenConsumer` - auto-commits. For high-volume topics, `:commit_threshold` is the dominant factor. + For low-volume topics, `:commit_interval` is the dominant factor for how + often a `GenConsumer` auto-commits. For high-volume topics, + `:commit_threshold` is the dominant factor. ## Callbacks - There are three callbacks that are required to be implemented in a `GenConsumer`. By adding `use - KafkaEx.GenServer` to a module, two of the callbacks will be defined with default behavior, - leaving you to implement `c:handle_message/2`. + There are three callbacks that are required to be implemented in a + `GenConsumer`. By adding `use KafkaEx.GenServer` to a module, two of the + callbacks will be defined with default behavior, leaving you to implement + `c:handle_message_set/2`. ## Integration with OTP - A `GenConsumer` is a specialized `GenServer`. It can be supervised, registered, and debugged the - same as any other `GenServer`. However, its arguments for `c:GenServer.init/1` are unspecified, so - `start_link/5` should be used to start a `GenConsumer` process instead of `GenServer` primitives. + A `GenConsumer` is a specialized `GenServer`. It can be supervised, + registered, and debugged the same as any other `GenServer`. However, its + arguments for `c:GenServer.init/1` are unspecified, so `start_link/5` should + be used to start a `GenConsumer` process instead of `GenServer` primitives. ## Testing - A `GenConsumer` can be unit-tested without a running Kafka broker by sending messages directly to - its `c:handle_message/2` function. The following recipe can be used as a starting point when - testing a `GenConsumer`: + A `GenConsumer` can be unit-tested without a running Kafka broker by sending + messages directly to its `c:handle_message_set/2` function. The following + recipe can be used as a starting point when testing a `GenConsumer`: ``` defmodule ExampleGenConsumerTest do @@ -99,8 +132,9 @@ defmodule KafkaEx.GenConsumer do end test "it acks a message", %{state: state} do - message = %Message{offset: 0, value: "hello"} - {response, _new_state} = ExampleGenConsumer.handle_message(message, state) + message_set = [%Message{offset: 0, value: "hello"}] + {response, _new_state} = + ExampleGenConsumer.handle_message_set(message_set, state) assert response == :async_commit end end @@ -165,10 +199,10 @@ defmodule KafkaEx.GenConsumer do @callback init(topic :: topic, partition :: partition_id) :: {:ok, state :: term} @doc """ - Invoked for each message consumed from a Kafka queue. + Invoked for each message set consumed from a Kafka topic partition. - `message` is a message fetched from a Kafka broker and `state` is the current state of the - `GenConsumer`. + `message_set` is a message set fetched from a Kafka broker and `state` is the + current state of the `GenConsumer`. Returning `{:async_commit, new_state}` acknowledges `message` and continues to consume from the Kafka queue with new state `new_state`. Acknowledged messages will be auto-committed (possibly at a later @@ -180,8 +214,8 @@ defmodule KafkaEx.GenConsumer do sparingly, since committing every message synchronously would impact a consumer's performance and could result in excessive network traffic. """ - @callback handle_message(message :: Message.t, state :: term) :: {:async_commit, new_state :: term} - | {:sync_commit, new_state :: term} + @callback handle_message_set(message_set :: [Message.t], state :: term) :: + {:async_commit, new_state :: term} | {:sync_commit, new_state :: term} @doc """ Invoked to determine partition assignments for a coordinated consumer group. @@ -343,32 +377,46 @@ defmodule KafkaEx.GenConsumer do case response do %{last_offset: nil, message_set: []} -> - auto_commit(state) - - %{last_offset: _, message_set: messages} -> - Enum.reduce(messages, state, &handle_message/2) + handle_commit(:async_commit, state) + %{last_offset: _, message_set: message_set} -> + handle_message_set(message_set, state) end end - defp handle_message(%Message{offset: offset} = message, %State{consumer_module: consumer_module, consumer_state: consumer_state} = state) do - case consumer_module.handle_message(message, consumer_state) do - {:async_commit, new_state} -> - auto_commit %State{state | consumer_state: new_state, acked_offset: offset + 1, current_offset: offset + 1} + defp handle_message_set( + message_set, + %State{consumer_module: consumer_module, consumer_state: consumer_state} = state + ) do + {sync_status, new_consumer_state} = + consumer_module.handle_message_set(message_set, consumer_state) + + %Message{offset: last_offset} = List.last(message_set) + state_out = %State{ + state | + consumer_state: new_consumer_state, + acked_offset: last_offset + 1, + current_offset: last_offset + 1 + } - {:sync_commit, new_state} -> - commit %State{state | consumer_state: new_state, acked_offset: offset + 1, current_offset: offset + 1} - end + handle_commit(sync_status, state_out) end - defp auto_commit(%State{acked_offset: acked, committed_offset: committed, commit_threshold: threshold, - last_commit: last_commit, commit_interval: interval} = state) do + defp handle_commit(:sync_commit, %State{} = state), do: commit(state) + defp handle_commit( + :async_commit, + %State{ + acked_offset: acked, + committed_offset: committed, + commit_threshold: threshold, + last_commit: last_commit, + commit_interval: interval + } = state + ) do case acked - committed do 0 -> %State{state | last_commit: :erlang.monotonic_time(:milli_seconds)} - n when n >= threshold -> commit(state) - _ -> if :erlang.monotonic_time(:milli_seconds) - last_commit >= interval do commit(state) From 69eba79b3e75a5ea72bdca12880c6baa77fcd949 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Fri, 28 Jul 2017 16:31:07 -0400 Subject: [PATCH 16/47] Strict credo cleanup Mostly just line length. Added an intermediate variable in a couple places to handle long case statement input. --- lib/kafka_ex/consumer_group.ex | 286 +++++++++++++++------- lib/kafka_ex/consumer_group/supervisor.ex | 51 ++-- lib/kafka_ex/gen_consumer.ex | 268 ++++++++++++++------ lib/kafka_ex/gen_consumer/supervisor.ex | 39 ++- 4 files changed, 448 insertions(+), 196 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 3646bf1f..da7e824c 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -2,28 +2,33 @@ defmodule KafkaEx.ConsumerGroup do @moduledoc """ A process that manages membership in a Kafka consumer group. - Consumers in a consumer group coordinate with each other through a Kafka broker to distribute the - work of consuming one or several topics without any overlap. This is facilitated by the [Kafka - client-side assignment - protocol](https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Client-side+Assignment+Proposal). - - Any time group membership changes (a member joins or leaves the group), a Kafka broker initiates - group synchronization by asking one of the group members (the leader) to provide partition - assignments for the whole group. Partition assignment is handled by the - `c:KafkaEx.GenConsumer.assign_partitions/2` callback of the provided consumer module. + Consumers in a consumer group coordinate with each other through a Kafka + broker to distribute the work of consuming one or several topics without any + overlap. This is facilitated by the + [Kafka client-side assignment protocol](https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Client-side+Assignment+Proposal). + + Any time group membership changes (a member joins or leaves the group), a + Kafka broker initiates group synchronization by asking one of the group + members (the leader) to provide partition assignments for the whole group. + Partition assignment is handled by the + `c:KafkaEx.GenConsumer.assign_partitions/2` callback of the provided consumer + module. A `ConsumerGroup` process is responsible for: 1. Maintaining membership in a Kafka consumer group. 2. Determining partition assignments if elected as the group leader. - 3. Launching and terminating `GenConsumer` processes based on its assigned partitions. + 3. Launching and terminating `GenConsumer` processes based on its assigned + partitions. - To use a `ConsumerGroup`, a developer must define a module that implements the - `KafkaEx.GenConsumer` behaviour and start a `ConsumerGroup` with that module. + To use a `ConsumerGroup`, a developer must define a module that implements + the `KafkaEx.GenConsumer` behaviour and start a `ConsumerGroup` with that + module. ## Example - The following consumer prints each message with the name of the node that's consuming the message: + The following consumer prints each message with the name of the node that's + consuming the message: ``` defmodule DistributedConsumer do @@ -53,8 +58,8 @@ defmodule KafkaEx.ConsumerGroup do node1@host: "nodes" ``` - It is not necessary for the nodes to be connected, because `ConsumerGroup` uses Kafka's built-in - group coordination protocol. + It is not necessary for the nodes to be connected, because `ConsumerGroup` + uses Kafka's built-in group coordination protocol. """ use GenServer @@ -96,26 +101,45 @@ defmodule KafkaEx.ConsumerGroup do # Client API @doc """ - Starts a `ConsumerGroup` process linked to the current process. Client programs should use - `KafkaEx.ConsumerGroup.Supervisor.start_link/4` instead. + Starts a `ConsumerGroup` process linked to the current process. + + Client programs should use `KafkaEx.ConsumerGroup.Supervisor.start_link/4` + instead. """ - @spec start_link(module, binary, [binary], KafkaEx.GenConsumer.options) :: GenServer.on_start + @spec start_link(module, binary, [binary], KafkaEx.GenConsumer.options) :: + GenServer.on_start def start_link(consumer_module, group_name, topics, opts \\ []) do - {server_opts, consumer_opts} = Keyword.split(opts, [:debug, :name, :timeout, :spawn_opt]) - - GenServer.start_link(__MODULE__, {consumer_module, group_name, topics, consumer_opts}, server_opts) + {server_opts, consumer_opts} = + Keyword.split(opts, [:debug, :name, :timeout, :spawn_opt]) + + GenServer.start_link( + __MODULE__, + {consumer_module, group_name, topics, consumer_opts}, + server_opts + ) end # GenServer callbacks def init({consumer_module, group_name, topics, opts}) do - heartbeat_interval = Keyword.get(opts, :heartbeat_interval, Application.get_env(:kafka_ex, :heartbeat_interval, @heartbeat_interval)) - session_timeout = Keyword.get(opts, :session_timeout, Application.get_env(:kafka_ex, :session_timeout, @session_timeout)) + heartbeat_interval = Keyword.get( + opts, + :heartbeat_interval, + Application.get_env(:kafka_ex, :heartbeat_interval, @heartbeat_interval) + ) + session_timeout = Keyword.get( + opts, + :session_timeout, + Application.get_env(:kafka_ex, :session_timeout, @session_timeout) + ) supervisor_pid = Keyword.fetch!(opts, :supervisor_pid) - consumer_opts = Keyword.drop(opts, [:supervisor_pid, :heartbeat_interval, :session_timeout]) + consumer_opts = Keyword.drop( + opts, [:supervisor_pid, :heartbeat_interval, :session_timeout] + ) - {:ok, worker_name} = KafkaEx.create_worker(:no_name, consumer_group: group_name) + {:ok, worker_name} = + KafkaEx.create_worker(:no_name, consumer_group: group_name) state = %State{ supervisor_pid: supervisor_pid, @@ -134,23 +158,28 @@ defmodule KafkaEx.ConsumerGroup do {:ok, state, 0} end - # If `member_id` and `generation_id` aren't set, we haven't yet joined the group. `member_id` and - # `generation_id` are initialized by `JoinGroupResponse`. - def handle_info(:timeout, %State{generation_id: nil, member_id: ""} = state) do + # If `member_id` and `generation_id` aren't set, we haven't yet joined the + # group. `member_id` and `generation_id` are initialized by + # `JoinGroupResponse`. + def handle_info( + :timeout, %State{generation_id: nil, member_id: ""} = state + ) do {:ok, new_state} = join(state) {:noreply, new_state} end - # After joining the group, a member must periodically send heartbeats to the group coordinator. + # After joining the group, a member must periodically send heartbeats to the + # group coordinator. def handle_info(:heartbeat, %State{} = state) do {:ok, new_state} = heartbeat(state) {:noreply, new_state} end - # When terminating, inform the group coordinator that this member is leaving the group so that the - # group can rebalance without waiting for a session timeout. + # When terminating, inform the group coordinator that this member is leaving + # the group so that the group can rebalance without waiting for a session + # timeout. def terminate(_reason, %State{generation_id: nil, member_id: ""}), do: :ok def terminate(_reason, %State{} = state) do :ok = leave(state) @@ -158,18 +187,30 @@ defmodule KafkaEx.ConsumerGroup do ### Helpers - # `JoinGroupRequest` is used to set the active members of a group. The response blocks until the - # broker has decided that it has a full list of group members. This requires that all active - # members send a `JoinGroupRequest`. For active members, this is triggered by the broker - # responding to a heartbeat with a `:rebalance_in_progress` error code. If any group members fail - # to send a `JoinGroupRequest` before the session timeout expires, then those group members are - # removed from the group and synchronization continues without them. + # `JoinGroupRequest` is used to set the active members of a group. The + # response blocks until the broker has decided that it has a full list of + # group members. This requires that all active members send a + # `JoinGroupRequest`. For active members, this is triggered by the broker + # responding to a heartbeat with a `:rebalance_in_progress` error code. If + # any group members fail to send a `JoinGroupRequest` before the session + # timeout expires, then those group members are removed from the group and + # synchronization continues without them. # - # `JoinGroupResponse` tells each member its unique member ID as well as the group's current - # generation ID. The broker will pick one group member to be the leader, which is reponsible for - # assigning partitions to all of the group members. Once a `JoinGroupResponse` is received, all - # group members must send a `SyncGroupRequest` (see sync/2). - defp join(%State{worker_name: worker_name, session_timeout: session_timeout, group_name: group_name, topics: topics, member_id: member_id} = state) do + # `JoinGroupResponse` tells each member its unique member ID as well as the + # group's current generation ID. The broker will pick one group member to be + # the leader, which is reponsible for assigning partitions to all of the + # group members. Once a `JoinGroupResponse` is received, all group members + # must send a `SyncGroupRequest` (see sync/2). + + defp join( + %State{ + worker_name: worker_name, + session_timeout: session_timeout, + group_name: group_name, + topics: topics, + member_id: member_id + } = state + ) do join_request = %JoinGroupRequest{ group_name: group_name, member_id: member_id, @@ -178,11 +219,19 @@ defmodule KafkaEx.ConsumerGroup do } join_response = %JoinGroupResponse{error_code: :no_error} = - KafkaEx.join_group(join_request, worker_name: worker_name, timeout: session_timeout + @session_timeout_padding) + KafkaEx.join_group( + join_request, + worker_name: worker_name, + timeout: session_timeout + @session_timeout_padding + ) Logger.debug(fn -> "Joined consumer group #{group_name}" end) - new_state = %State{state | member_id: join_response.member_id, generation_id: join_response.generation_id} + new_state = %State{ + state | + member_id: join_response.member_id, + generation_id: join_response.generation_id + } assignments = if JoinGroupResponse.leader?(join_response) do @@ -197,17 +246,28 @@ defmodule KafkaEx.ConsumerGroup do sync(new_state, assignments) end - # `SyncGroupRequest` is used to distribute partition assignments to all group members. All group - # members must send this request after receiving a response to a `JoinGroupRequest`. The request - # blocks until assignments are provided by the leader. The leader sends partition assignments - # (given by the `assignments` parameter) as part of its `SyncGroupRequest`. For all other members, - # `assignments` must be empty. + # `SyncGroupRequest` is used to distribute partition assignments to all group + # members. All group members must send this request after receiving a + # response to a `JoinGroupRequest`. The request blocks until assignments are + # provided by the leader. The leader sends partition assignments (given by + # the `assignments` parameter) as part of its `SyncGroupRequest`. For all + # other members, `assignments` must be empty. # - # `SyncGroupResponse` contains the individual member's partition assignments. Upon receiving a - # successful `SyncGroupResponse`, a group member is free to start consuming from its assigned - # partitions, but must send periodic heartbeats to the coordinating broker. - defp sync(%State{group_name: group_name, member_id: member_id, generation_id: generation_id, - worker_name: worker_name, session_timeout: session_timeout} = state, assignments) do + # `SyncGroupResponse` contains the individual member's partition assignments. + # Upon receiving a successful `SyncGroupResponse`, a group member is free to + # start consuming from its assigned partitions, but must send periodic + # heartbeats to the coordinating broker. + + defp sync( + %State{ + group_name: group_name, + member_id: member_id, + generation_id: generation_id, + worker_name: worker_name, + session_timeout: session_timeout + } = state, + assignments + ) do sync_request = %SyncGroupRequest{ group_name: group_name, member_id: member_id, @@ -215,31 +275,43 @@ defmodule KafkaEx.ConsumerGroup do assignments: assignments, } - case KafkaEx.sync_group(sync_request, worker_name: worker_name, timeout: session_timeout + @session_timeout_padding) do + sync_group_response = KafkaEx.sync_group( + sync_request, + worker_name: worker_name, + timeout: session_timeout + @session_timeout_padding + ) + + case sync_group_response do %SyncGroupResponse{error_code: :no_error, assignments: assignments} -> new_state = state |> start_consumer(unpack_assignments(assignments)) |> start_heartbeat_timer() - {:ok, new_state} - %SyncGroupResponse{error_code: :rebalance_in_progress} -> rebalance(state) end end - # `HeartbeatRequest` is sent periodically by each active group member (after completing the - # join/sync phase) to inform the broker that the member is still alive and participating in the - # group. If a group member fails to send a heartbeat before the group's session timeout expires, - # the coordinator removes that member from the group and initiates a rebalance. + # `HeartbeatRequest` is sent periodically by each active group member (after + # completing the join/sync phase) to inform the broker that the member is + # still alive and participating in the group. If a group member fails to send + # a heartbeat before the group's session timeout expires, the coordinator + # removes that member from the group and initiates a rebalance. # - # `HeartbeatResponse` allows the coordinating broker to communicate the group's status to each - # member: + # `HeartbeatResponse` allows the coordinating broker to communicate the + # group's status to each member: # - # * `:no_error` indicates that the group is up to date and no action is needed. - # * `:rebalance_in_progress` instructs each member to rejoin the group by sending a - # `JoinGroupRequest` (see join/1). - defp heartbeat(%State{worker_name: worker_name, group_name: group_name, generation_id: generation_id, member_id: member_id} = state) do + # * `:no_error` indicates that the group is up to date and no action is + # needed. * `:rebalance_in_progress` instructs each member to rejoin the + # group by sending a `JoinGroupRequest` (see join/1). + defp heartbeat( + %State{ + worker_name: worker_name, + group_name: group_name, + generation_id: generation_id, + member_id: member_id + } = state + ) do heartbeat_request = %HeartbeatRequest{ group_name: group_name, member_id: member_id, @@ -256,10 +328,17 @@ defmodule KafkaEx.ConsumerGroup do end end - # `LeaveGroupRequest` is used to voluntarily leave a group. This tells the broker that the member - # is leaving the group without having to wait for the session timeout to expire. Leaving a group - # triggers a rebalance for the remaining group members. - defp leave(%State{worker_name: worker_name, group_name: group_name, member_id: member_id} = state) do + # `LeaveGroupRequest` is used to voluntarily leave a group. This tells the + # broker that the member is leaving the group without having to wait for the + # session timeout to expire. Leaving a group triggers a rebalance for the + # remaining group members. + defp leave( + %State{ + worker_name: worker_name, + group_name: group_name, + member_id: member_id + } = state + ) do stop_heartbeat_timer(state) leave_request = %LeaveGroupRequest{ @@ -267,16 +346,18 @@ defmodule KafkaEx.ConsumerGroup do member_id: member_id, } - %LeaveGroupResponse{error_code: :no_error} = KafkaEx.leave_group(leave_request, worker_name: worker_name) + %LeaveGroupResponse{error_code: :no_error} = + KafkaEx.leave_group(leave_request, worker_name: worker_name) Logger.debug(fn -> "Left consumer group #{group_name}" end) :ok end - # When instructed that a rebalance is in progress, a group member must rejoin the group with - # `JoinGroupRequest` (see join/1). To keep the state synchronized during the join/sync phase, each - # member pauses its consumers and commits its offsets before rejoining the group. + # When instructed that a rebalance is in progress, a group member must rejoin + # the group with `JoinGroupRequest` (see join/1). To keep the state + # synchronized during the join/sync phase, each member pauses its consumers + # and commits its offsets before rejoining the group. defp rebalance(%State{} = state) do state |> stop_heartbeat_timer() @@ -287,7 +368,9 @@ defmodule KafkaEx.ConsumerGroup do ### Timer Management # Starts a timer for the next heartbeat. - defp start_heartbeat_timer(%State{heartbeat_interval: heartbeat_interval} = state) do + defp start_heartbeat_timer( + %State{heartbeat_interval: heartbeat_interval} = state + ) do {:ok, timer} = :timer.send_after(heartbeat_interval, :heartbeat) %State{state | heartbeat_timer: timer} @@ -295,7 +378,9 @@ defmodule KafkaEx.ConsumerGroup do # Stops any active heartbeat timer. defp stop_heartbeat_timer(%State{heartbeat_timer: nil} = state), do: state - defp stop_heartbeat_timer(%State{heartbeat_timer: heartbeat_timer} = state) do + defp stop_heartbeat_timer( + %State{heartbeat_timer: heartbeat_timer} = state + ) do {:ok, :cancel} = :timer.cancel(heartbeat_timer) %State{state | heartbeat_timer: nil} @@ -304,9 +389,22 @@ defmodule KafkaEx.ConsumerGroup do ### Consumer Management # Starts consuming from the member's assigned partitions. - defp start_consumer(%State{consumer_module: consumer_module, consumer_opts: consumer_opts, - group_name: group_name, supervisor_pid: pid} = state, assignments) do - :ok = KafkaEx.ConsumerGroup.Supervisor.start_consumer(pid, consumer_module, group_name, assignments, consumer_opts) + defp start_consumer( + %State{ + consumer_module: consumer_module, + consumer_opts: consumer_opts, + group_name: group_name, + supervisor_pid: pid + } = state, + assignments + ) do + :ok = KafkaEx.ConsumerGroup.Supervisor.start_consumer( + pid, + consumer_module, + group_name, + assignments, + consumer_opts + ) state end @@ -320,10 +418,13 @@ defmodule KafkaEx.ConsumerGroup do ### Partition Assignment - # Queries the Kafka brokers for a list of partitions for the topics of interest to this consumer - # group. This function returns a list of topic/partition tuples that can be passed to a - # GenConsumer's `assign_partitions` method. - defp assignable_partitions(%State{worker_name: worker_name, topics: topics}) do + # Queries the Kafka brokers for a list of partitions for the topics of + # interest to this consumer group. This function returns a list of + # topic/partition tuples that can be passed to a GenConsumer's + # `assign_partitions` method. + defp assignable_partitions( + %State{worker_name: worker_name, topics: topics} + ) do metadata = KafkaEx.metadata(worker_name: worker_name) Enum.flat_map(topics, fn (topic) -> @@ -335,12 +436,17 @@ defmodule KafkaEx.ConsumerGroup do end) end - # This function is used by the group leader to determine partition assignments during the - # join/sync phase. `members` is provided to the leader by the coordinating broker in - # `JoinGroupResponse`. `partitions` is a list of topic/partition tuples, obtained from - # `assignable_partitions/1`. The return value is a complete list of member assignments in the - # format needed by `SyncGroupResponse`. - defp assign_partitions(%State{consumer_module: consumer_module}, members, partitions) do + # This function is used by the group leader to determine partition + # assignments during the join/sync phase. `members` is provided to the leader + # by the coordinating broker in `JoinGroupResponse`. `partitions` is a list + # of topic/partition tuples, obtained from `assignable_partitions/1`. The + # return value is a complete list of member assignments in the format needed + # by `SyncGroupResponse`. + defp assign_partitions( + %State{consumer_module: consumer_module}, + members, + partitions + ) do # Delegate partition assignment to GenConsumer module. assignments = consumer_module.assign_partitions(members, partitions) diff --git a/lib/kafka_ex/consumer_group/supervisor.ex b/lib/kafka_ex/consumer_group/supervisor.ex index e1d070d7..4551eeec 100644 --- a/lib/kafka_ex/consumer_group/supervisor.ex +++ b/lib/kafka_ex/consumer_group/supervisor.ex @@ -1,13 +1,14 @@ defmodule KafkaEx.ConsumerGroup.Supervisor do @moduledoc """ - A supervisor for managing a consumer group. A `KafkaEx.ConsumerGroup.Supervisor` process will - manage an entire process tree for a single consumer group. Multiple supervisors can be used for - multiple consumer groups within the same application. + A supervisor for managing a consumer group. A + `KafkaEx.ConsumerGroup.Supervisor` process will manage an entire process tree + for a single consumer group. Multiple supervisors can be used for multiple + consumer groups within the same application. ## Example - This supervisor can be addeded to an application's supervision tree with a custom `GenConsumer` - implementation with the following child spec: + This supervisor can be addeded to an application's supervision tree with a + custom `GenConsumer` implementation with the following child spec: ``` supervisor(KafkaEx.ConsumerGroup.Supervisor, [MyApp.Consumer, "group_name", ["topic1", "topic2"]]) @@ -32,29 +33,42 @@ defmodule KafkaEx.ConsumerGroup.Supervisor do @doc """ Starts a `ConsumerGroup.Supervisor` process linked to the current process. - This can be used to start a `KafkaEx.ConsumerGroup` as part of a supervision tree. + This can be used to start a `KafkaEx.ConsumerGroup` as part of a supervision + tree. - `module` is a module that implements the `KafkaEx.GenConsumer` behaviour. `group_name` is the - name of the consumer group. `topics` is a list of topics that the consumer group should consume - from. `opts` can be any options accepted by `KafkaEx.ConsumerGroup` or `Supervisor`. + `module` is a module that implements the `KafkaEx.GenConsumer` behaviour. + `group_name` is the name of the consumer group. `topics` is a list of topics + that the consumer group should consume from. `opts` can be any options + accepted by `KafkaEx.ConsumerGroup` or `Supervisor`. ### Return Values This function has the same return values as `Supervisor.start_link/3`. - If the supervisor and consumer group are successfully created and initialized, this function - returns `{:ok, pid}`, where `pid` is the PID of the consumer group supervisor process. + If the supervisor and consumer group are successfully created and + initialized, this function returns `{:ok, pid}`, where `pid` is the PID of + the consumer group supervisor process. """ - @spec start_link(module, binary, [binary], options) :: Elixir.Supervisor.on_start + @spec start_link(module, binary, [binary], options) :: + Elixir.Supervisor.on_start def start_link(consumer_module, group_name, topics, opts \\ []) do - {supervisor_opts, module_opts} = Keyword.split(opts, [:name, :strategy, :max_restarts, :max_seconds]) - - Elixir.Supervisor.start_link(__MODULE__, {consumer_module, group_name, topics, module_opts}, supervisor_opts) + {supervisor_opts, module_opts} = + Keyword.split(opts, [:name, :strategy, :max_restarts, :max_seconds]) + + Elixir.Supervisor.start_link( + __MODULE__, + {consumer_module, group_name, topics, module_opts}, + supervisor_opts + ) end @doc false # used by ConsumerGroup to set partition assignments def start_consumer(pid, consumer_module, group_name, assignments, opts) do - child = supervisor(KafkaEx.GenConsumer.Supervisor, [consumer_module, group_name, assignments, opts], id: :consumer) + child = supervisor( + KafkaEx.GenConsumer.Supervisor, + [consumer_module, group_name, assignments, opts], + id: :consumer + ) case Elixir.Supervisor.start_child(pid, child) do {:ok, _child} -> :ok @@ -77,7 +91,10 @@ defmodule KafkaEx.ConsumerGroup.Supervisor do opts = Keyword.put(opts, :supervisor_pid, self()) children = [ - worker(KafkaEx.ConsumerGroup, [consumer_module, group_name, topics, opts]), + worker( + KafkaEx.ConsumerGroup, + [consumer_module, group_name, topics, opts] + ), ] supervise(children, strategy: :one_for_all) diff --git a/lib/kafka_ex/gen_consumer.ex b/lib/kafka_ex/gen_consumer.ex index 3b4ae930..4168eee0 100644 --- a/lib/kafka_ex/gen_consumer.ex +++ b/lib/kafka_ex/gen_consumer.ex @@ -2,18 +2,20 @@ defmodule KafkaEx.GenConsumer do @moduledoc """ A behaviour module for implementing a Kafka consumer. - A `GenConsumer` is an Elixir process that consumes messages from Kafka. A single `GenConsumer` - process consumes from a single partition of a Kafka topic. Several `GenConsumer` processes can be - used to consume from multiple partitions or even multiple topics. Partition assignments for a - group of `GenConsumer`s can be defined manually using `KafkaEx.GenConsumer.Supervisor` or - coordinated across a cluster of nodes using `KafkaEx.ConsumerGroup`. + A `GenConsumer` is an Elixir process that consumes messages from Kafka. A + single `GenConsumer` process consumes from a single partition of a Kafka + topic. Several `GenConsumer` processes can be used to consume from multiple + partitions or even multiple topics. Partition assignments for a group of + `GenConsumer`s can be defined manually using `KafkaEx.GenConsumer.Supervisor` + or coordinated across a cluster of nodes using `KafkaEx.ConsumerGroup`. ## Example - The `GenConsumer` behaviour abstracts common Kafka consumer interactions. `GenConsumer` will take - care of the details of determining a starting offset, fetching messages from a Kafka broker, and - committing offsets for consumed messages. Developers are only required to implement - `c:handle_message_set/2` to process message sets. + The `GenConsumer` behaviour abstracts common Kafka consumer interactions. + `GenConsumer` will take care of the details of determining a starting offset, + fetching messages from a Kafka broker, and committing offsets for consumed + messages. Developers are only required to implement `c:handle_message_set/2` + to process message sets. The following is a minimal example that logs each message as it's consumed: @@ -185,18 +187,21 @@ defmodule KafkaEx.GenConsumer do @type options :: [option | GenServer.option] @doc """ - Invoked when the server is started. `start_link/5` will block until it returns. + Invoked when the server is started. `start_link/5` will block until it + returns. - `topic` and `partition` are the arguments passed to `start_link/5`. They identify the Kafka - partition that the `GenConsumer` will consume from. + `topic` and `partition` are the arguments passed to `start_link/5`. They + identify the Kafka partition that the `GenConsumer` will consume from. - Returning `{:ok, state}` will cause `start_link/5` to return `{:ok, pid}` and the process to start - consuming from its assigned partition. `state` becomes the consumer's state. + Returning `{:ok, state}` will cause `start_link/5` to return `{:ok, pid}` and + the process to start consuming from its assigned partition. `state` becomes + the consumer's state. - Any other return value will cause the `start_link/5` to return `{:error, error}` and the process - to exit. + Any other return value will cause the `start_link/5` to return `{:error, + error}` and the process to exit. """ - @callback init(topic :: topic, partition :: partition_id) :: {:ok, state :: term} + @callback init(topic :: topic, partition :: partition_id) :: + {:ok, state :: term} @doc """ Invoked for each message set consumed from a Kafka topic partition. @@ -204,15 +209,17 @@ defmodule KafkaEx.GenConsumer do `message_set` is a message set fetched from a Kafka broker and `state` is the current state of the `GenConsumer`. - Returning `{:async_commit, new_state}` acknowledges `message` and continues to consume from the Kafka queue - with new state `new_state`. Acknowledged messages will be auto-committed (possibly at a later - time) based on the `:commit_interval` and `:commit_threshold` options. - - Returning `{:sync_commit, new_state}` commits `message` synchronously before continuing to consume from - the Kafka queue with new state `new_state`. Committing a message synchronously means that no more - messages will be consumed until the message's offset is committed. `:sync_commit` should be used - sparingly, since committing every message synchronously would impact a consumer's performance and - could result in excessive network traffic. + Returning `{:async_commit, new_state}` acknowledges `message` and continues + to consume from the Kafka queue with new state `new_state`. Acknowledged + messages will be auto-committed (possibly at a later time) based on the + `:commit_interval` and `:commit_threshold` options. + + Returning `{:sync_commit, new_state}` commits `message` synchronously before + continuing to consume from the Kafka queue with new state `new_state`. + Committing a message synchronously means that no more messages will be + consumed until the message's offset is committed. `:sync_commit` should be + used sparingly, since committing every message synchronously would impact a + consumer's performance and could result in excessive network traffic. """ @callback handle_message_set(message_set :: [Message.t], state :: term) :: {:async_commit, new_state :: term} | {:sync_commit, new_state :: term} @@ -220,17 +227,18 @@ defmodule KafkaEx.GenConsumer do @doc """ Invoked to determine partition assignments for a coordinated consumer group. - `members` is a list of member IDs and `partitions` is a list of partitions that need to be - assigned to a group member. + `members` is a list of member IDs and `partitions` is a list of partitions + that need to be assigned to a group member. - The return value must be a map with member IDs as keys and a list of partition assignments as - values. For each member ID in the returned map, the assigned partitions will become the - `assignments` argument to `KafkaEx.GenConsumer.Supervisor.start_link/4` in the corresponding - member process. Any member that's omitted from the return value will not be assigned any - partitions. + The return value must be a map with member IDs as keys and a list of + partition assignments as values. For each member ID in the returned map, the + assigned partitions will become the `assignments` argument to + `KafkaEx.GenConsumer.Supervisor.start_link/4` in the corresponding member + process. Any member that's omitted from the return value will not be assigned + any partitions. - If this callback is not implemented, the default implementation by `use KafkaEx.GenConsumer` - implements a simple round-robin assignment. + If this callback is not implemented, the default implementation by `use + KafkaEx.GenConsumer` implements a simple round-robin assignment. ### Example @@ -248,11 +256,15 @@ defmodule KafkaEx.GenConsumer do #=> %{"member1" => [{"topic", 0}, {"topic", 2}], "member2" => [{"topic", 1}]} ``` - In this case, the consumer group process for `"member1"` will launch two `GenConsumer` processes - (one for each of its assigned partitions), `"member2"` will launch one `GenConsumer` process, and - `"member3"` will launch no processes. + In this case, the consumer group process for `"member1"` will launch two + `GenConsumer` processes (one for each of its assigned partitions), + `"member2"` will launch one `GenConsumer` process, and `"member3"` will + launch no processes. """ - @callback assign_partitions(members :: [member_id], partitions :: [partition]) :: %{member_id => [partition]} + @callback assign_partitions( + members :: [member_id], + partitions :: [partition] + ) :: %{member_id => [partition]} defmacro __using__(_opts) do quote do @@ -264,7 +276,8 @@ defmodule KafkaEx.GenConsumer do end def assign_partitions(members, partitions) do - Stream.cycle(members) + members + |> Stream.cycle |> Enum.zip(partitions) |> Enum.group_by(&(elem(&1, 0)), &(elem(&1, 1))) end @@ -301,16 +314,20 @@ defmodule KafkaEx.GenConsumer do This can be used to start the `GenConsumer` as part of a supervision tree. - Once the consumer has been started, the `c:init/2` function of the given `consumer_module` is - called with the given `topic` and `partition`. `group_name` is the consumer group name that will - be used for managing consumer offsets. + Once the consumer has been started, the `c:init/2` function of the given + `consumer_module` is called with the given `topic` and `partition`. + `group_name` is the consumer group name that will be used for managing + consumer offsets. ### Options - * `:commit_interval` - the interval in milliseconds that the consumer will wait to commit - acknowledged messages. If not present, the `:commit_interval` environment value is used. - * `:commit_threshold` - the maximum number of messages that can be acknowledged without being - committed. If not present, the `:commit_threshold` environment value is used. + * `:commit_interval` - the interval in milliseconds that the consumer will + wait to commit acknowledged messages. If not present, the `:commit_interval` + environment value is used. + + * `:commit_threshold` - the maximum number of messages that can be + acknowledged without being committed. If not present, the `:commit_threshold` + environment value is used. Any valid options for `GenServer.start_link/3` can also be specified. @@ -318,24 +335,44 @@ defmodule KafkaEx.GenConsumer do This function has the same return values as `GenServer.start_link/3`. - If the consumer is successfully created and initialized, this function returns `{:ok, pid}`, where - `pid` is the PID of the consumer process. + If the consumer is successfully created and initialized, this function + returns `{:ok, pid}`, where `pid` is the PID of the consumer process. """ - @spec start_link(module, binary, topic, partition_id, options) :: GenServer.on_start + @spec start_link( + module, + binary, + topic, + partition_id, + options + ) :: GenServer.on_start def start_link(consumer_module, group_name, topic, partition, opts \\ []) do - {server_opts, consumer_opts} = Keyword.split(opts, [:debug, :name, :timeout, :spawn_opt]) - - GenServer.start_link(__MODULE__, {consumer_module, group_name, topic, partition, consumer_opts}, server_opts) + {server_opts, consumer_opts} = + Keyword.split(opts, [:debug, :name, :timeout, :spawn_opt]) + + GenServer.start_link( + __MODULE__, + {consumer_module, group_name, topic, partition, consumer_opts}, + server_opts + ) end # GenServer callbacks def init({consumer_module, group_name, topic, partition, opts}) do - commit_interval = Keyword.get(opts, :commit_interval, Application.get_env(:kafka_ex, :commit_interval, @commit_interval)) - commit_threshold = Keyword.get(opts, :commit_threshold, Application.get_env(:kafka_ex, :commit_threshold, @commit_threshold)) + commit_interval = Keyword.get( + opts, + :commit_interval, + Application.get_env(:kafka_ex, :commit_interval, @commit_interval) + ) + commit_threshold = Keyword.get( + opts, + :commit_threshold, + Application.get_env(:kafka_ex, :commit_threshold, @commit_threshold) + ) {:ok, consumer_state} = consumer_module.init(topic, partition) - {:ok, worker_name} = KafkaEx.create_worker(:no_name, consumer_group: group_name) + {:ok, worker_name} = + KafkaEx.create_worker(:no_name, consumer_group: group_name) state = %State{ consumer_module: consumer_module, @@ -353,8 +390,14 @@ defmodule KafkaEx.GenConsumer do {:ok, state, 0} end - def handle_info(:timeout, %State{current_offset: nil, last_commit: nil} = state) do - new_state = %State{load_offsets(state) | last_commit: :erlang.monotonic_time(:milli_seconds)} + def handle_info( + :timeout, + %State{current_offset: nil, last_commit: nil} = state + ) do + new_state = %State{ + load_offsets(state) | + last_commit: :erlang.monotonic_time(:milli_seconds) + } {:noreply, new_state, 0} end @@ -371,9 +414,28 @@ defmodule KafkaEx.GenConsumer do # Helpers - defp consume(%State{worker_name: worker_name, topic: topic, partition: partition, current_offset: offset} = state) do - [%FetchResponse{topic: ^topic, partitions: [response = %{error_code: :no_error, partition: ^partition}]}] = - KafkaEx.fetch(topic, partition, offset: offset, auto_commit: false, worker_name: worker_name) + defp consume( + %State{ + worker_name: worker_name, + topic: topic, + partition: partition, + current_offset: offset + } = state + ) do + [ + %FetchResponse{ + topic: ^topic, + partitions: [ + response = %{error_code: :no_error, partition: ^partition} + ] + } + ] = KafkaEx.fetch( + topic, + partition, + offset: offset, + auto_commit: false, + worker_name: worker_name + ) case response do %{last_offset: nil, message_set: []} -> @@ -385,7 +447,10 @@ defmodule KafkaEx.GenConsumer do defp handle_message_set( message_set, - %State{consumer_module: consumer_module, consumer_state: consumer_state} = state + %State{ + consumer_module: consumer_module, + consumer_state: consumer_state + } = state ) do {sync_status, new_consumer_state} = consumer_module.handle_message_set(message_set, consumer_state) @@ -426,8 +491,21 @@ defmodule KafkaEx.GenConsumer do end end - defp commit(%State{acked_offset: offset, committed_offset: offset} = state), do: state - defp commit(%State{worker_name: worker_name, group: group, topic: topic, partition: partition, acked_offset: offset} = state) do + defp commit( + %State{acked_offset: offset, committed_offset: offset} = state + ) do + state + end + + defp commit( + %State{ + worker_name: worker_name, + group: group, + topic: topic, + partition: partition, + acked_offset: offset + } = state + ) do request = %OffsetCommitRequest{ consumer_group: group, topic: topic, @@ -438,26 +516,64 @@ defmodule KafkaEx.GenConsumer do [%OffsetCommitResponse{topic: ^topic, partitions: [^partition]}] = KafkaEx.offset_commit(worker_name, request) - Logger.debug(fn -> "Committed offset #{topic}/#{partition}@#{offset} for #{group}" end) + Logger.debug(fn -> + "Committed offset #{topic}/#{partition}@#{offset} for #{group}" + end) - %State{state | committed_offset: offset, last_commit: :erlang.monotonic_time(:milli_seconds)} + %State{ + state | + committed_offset: offset, + last_commit: :erlang.monotonic_time(:milli_seconds) + } end - defp load_offsets(%State{worker_name: worker_name, group: group, topic: topic, partition: partition} = state) do - request = %OffsetFetchRequest{consumer_group: group, topic: topic, partition: partition} + defp load_offsets( + %State{ + worker_name: worker_name, + group: group, + topic: topic, + partition: partition + } = state + ) do + request = %OffsetFetchRequest{ + consumer_group: group, + topic: topic, + partition: partition + } - [%OffsetFetchResponse{topic: ^topic, partitions: [%{partition: ^partition, error_code: error_code, offset: offset}]}] = - KafkaEx.offset_fetch(worker_name, request) + [ + %OffsetFetchResponse{ + topic: ^topic, + partitions: [ + %{partition: ^partition, error_code: error_code, offset: offset} + ] + } + ] = KafkaEx.offset_fetch(worker_name, request) case error_code do :no_error -> - %State{state | current_offset: offset, committed_offset: offset, acked_offset: offset} - + %State{ + state | + current_offset: offset, + committed_offset: offset, + acked_offset: offset + } :unknown_topic_or_partition -> - [%OffsetResponse{topic: ^topic, partition_offsets: [%{partition: ^partition, error_code: :no_error, offset: [offset]}]}] = - KafkaEx.earliest_offset(topic, partition, worker_name) - - %State{state | current_offset: offset, committed_offset: offset, acked_offset: offset} + [ + %OffsetResponse{ + topic: ^topic, + partition_offsets: [ + %{partition: ^partition, error_code: :no_error, offset: [offset]} + ] + } + ] = KafkaEx.earliest_offset(topic, partition, worker_name) + + %State{ + state | + current_offset: offset, + committed_offset: offset, + acked_offset: offset + } end end end diff --git a/lib/kafka_ex/gen_consumer/supervisor.ex b/lib/kafka_ex/gen_consumer/supervisor.ex index e127ae47..250a9c5a 100644 --- a/lib/kafka_ex/gen_consumer/supervisor.ex +++ b/lib/kafka_ex/gen_consumer/supervisor.ex @@ -1,13 +1,16 @@ defmodule KafkaEx.GenConsumer.Supervisor do @moduledoc """ - A supervisor for managing `GenConsumer` processes that are part of a consumer group. + A supervisor for managing `GenConsumer` processes that are part of a consumer + group. - The supervisor will launch individual `GenConsumer` processes for each partition given by the - `partitions` argument to `start_link/4`. When terminated, each of the supervisor's child - processes will commit its latest offset before terminating. + The supervisor will launch individual `GenConsumer` processes for each + partition given by the `partitions` argument to `start_link/4`. When + terminated, each of the supervisor's child processes will commit its latest + offset before terminating. - This module manages a static list of consumer processes. For dynamically distributing consumers - in a consumer group across a cluster of nodes, see `KafkaEx.ConsumerGroup`. + This module manages a static list of consumer processes. For dynamically + distributing consumers in a consumer group across a cluster of nodes, see + `KafkaEx.ConsumerGroup`. """ use Elixir.Supervisor @@ -15,20 +18,30 @@ defmodule KafkaEx.GenConsumer.Supervisor do @doc """ Starts a `GenConsumer.Supervisor` process linked to the current process. - `module` is a module that implements the `GenConsumer` behaviour. `group_name` is the name of a - consumer group, and `assignments` is a list of partitions for the `GenConsumer`s to consume. - `opts` accepts the same options as `KafkaEx.GenConsumer.start_link/5`. + `module` is a module that implements the `GenConsumer` behaviour. + `group_name` is the name of a consumer group, and `assignments` is a list of + partitions for the `GenConsumer`s to consume. `opts` accepts the same + options as `KafkaEx.GenConsumer.start_link/5`. ### Return Values This function has the same return values as `Supervisor.start_link/3`. - If the supervisor and its consumers are successfully created, this function returns `{:ok, - pid}`, where `pid` is the PID of the supervisor. + If the supervisor and its consumers are successfully created, this function + returns `{:ok, pid}`, where `pid` is the PID of the supervisor. """ - @spec start_link(module, binary, [KafkaEx.GenConsumer.partition], KafkaEx.GenConsumer.options) :: Elixir.Supervisor.on_start + @spec start_link( + module, + binary, + [KafkaEx.GenConsumer.partition], + KafkaEx.GenConsumer.options + ) :: Elixir.Supervisor.on_start def start_link(consumer_module, group_name, assignments, opts \\ []) do - case Elixir.Supervisor.start_link(__MODULE__, {consumer_module, group_name, assignments, opts}) do + start_link_result = Elixir.Supervisor.start_link( + __MODULE__, + {consumer_module, group_name, assignments, opts} + ) + case start_link_result do {:ok, pid} -> :ok = start_workers(pid, assignments, opts) {:ok, pid} From 3f971934ec7d40ea7afa4e5b7a34d35acc80e5d9 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 31 Jul 2017 12:04:54 -0400 Subject: [PATCH 17/47] Edit documentation --- README.md | 43 +++++-- config/config.exs | 5 + lib/kafka_ex/consumer_group.ex | 86 +++++++------ lib/kafka_ex/consumer_group/supervisor.ex | 23 ++-- lib/kafka_ex/gen_consumer.ex | 145 ++++++++++++---------- 5 files changed, 166 insertions(+), 136 deletions(-) diff --git a/README.md b/README.md index 6850ee86..29b913b2 100644 --- a/README.md +++ b/README.md @@ -23,24 +23,12 @@ KakfaEx supports the following Kafka features: * Fetch Messages * Message Compression with Snappy and gzip * Offset Management (fetch / commit / autocommit) +* Consumer Groups See [Kafka Protocol Documentation](http://kafka.apache.org/protocol.html) and [A Guide to the Kafka Protocol](https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol) for details of these features. -KafkaEx **does support** consumer groups for message consumption. This feature -was added in Kafka 0.8.2. This translates to providing a consumer group -name when committing offsets. It is up to the client to assign partitions to -workers in this mode of operation. - -KafkaEx currently provides **limited support** for the [Kafka ConsumerGroup -API](https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-GroupMembershipAPI) -that was added in Kafka 0.9.0. Most of the protocol requests are implemented -in KafkaEx, but we do not yet support automatic joining and management of -consumer group memebership (e.g., automatically assigning partitions to -clients). We are actively working on an implementation for automatic consumer -group management. - ## Using KafkaEx in an Elixir project The standard approach for adding dependencies to an Elixir application applies: @@ -89,6 +77,35 @@ You can also override options when creating a worker, see below. ## Usage Examples +### Consumer Groups + +To use a consumer group, first implement a handler module using +`KafkaEx.GenConsumer`. + +``` +defmodule ExampleGenConsumer do + use KafkaEx.GenConsumer + + alias KafkaEx.Protocol.Fetch.Message + + require Logger + + # note - messages are delivered in batches + def handle_message_set(message_set, state) do + for %Message{value: message} <- message_set do + Logger.debug(fn -> "message: " <> inspect(message) end) + end + {:async_commit, state} + end +end +``` + +Then add a `KafkaEx.ConsumerGroup.Supervisor` to your application's supervision +tree and configure it to use the implementation module. + +See the `KafkaEx.GenConsumer` and `KafkaEx.ConsumerGroup` documentation for +details. + ### Create a KafkaEx Worker KafkaEx worker processes manage the state of the connection to the Kafka broker. diff --git a/config/config.exs b/config/config.exs index 1b906d86..816b98cf 100644 --- a/config/config.exs +++ b/config/config.exs @@ -26,6 +26,11 @@ config :kafka_ex, max_restarts: 10, # Supervision max_seconds - the time frame in which :max_restarts applies max_seconds: 60, + # Interval in milliseconds that GenConsumer waits to commit offsets. + commit_interval: 5_000, + # Threshold number of messages consumed for GenConsumer to commit offsets + # to the broker. + commit_threshold: 100, # This is the flag that enables use of ssl use_ssl: true, # see SSL OPTION DESCRIPTIONS - CLIENT SIDE at http://erlang.org/doc/man/ssl.html diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index da7e824c..6120c583 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -9,57 +9,67 @@ defmodule KafkaEx.ConsumerGroup do Any time group membership changes (a member joins or leaves the group), a Kafka broker initiates group synchronization by asking one of the group - members (the leader) to provide partition assignments for the whole group. - Partition assignment is handled by the + members (the leader elected by the broker) to provide partition assignments + for the whole group. Partition assignment is handled by the `c:KafkaEx.GenConsumer.assign_partitions/2` callback of the provided consumer module. - A `ConsumerGroup` process is responsible for: + A `KafkaEx.ConsumerGroup` process is responsible for: 1. Maintaining membership in a Kafka consumer group. 2. Determining partition assignments if elected as the group leader. - 3. Launching and terminating `GenConsumer` processes based on its assigned - partitions. + 3. Launching and terminating `KafkaEx.GenConsumer` processes based on its + assigned partitions. - To use a `ConsumerGroup`, a developer must define a module that implements - the `KafkaEx.GenConsumer` behaviour and start a `ConsumerGroup` with that - module. + To use a `KafkaEx.ConsumerGroup`, a developer must define a module that + implements the `KafkaEx.GenConsumer` behaviour and start a + `KafkaEx.ConsumerGroup` configured to use that module. ## Example - The following consumer prints each message with the name of the node that's - consuming the message: + Suppose we want to consume from a topic called `"example_topic"` with a + consumer group named `"example_group"` and we have a `KafkaEx.GenConsumer` + implementation called `ExampleGenConsumer` (see the `KafkaEx.GenConsumer` + documentation). We could start a consumer group in our application's + supervision tree as follows: ``` - defmodule DistributedConsumer do - use KafkaEx.GenConsumer - - alias KafkaEx.Protocol.Fetch.Message - - # note - messages are delivered in batches - def handle_message_set(message_set, state) do - for %Message{value: message} <- message_set do - IO.puts(to_string(node()) <> ": " <> inspect(message)) - end - {:async_commit, state} + defmodule MyApp do + use Application + + def start(_type, _args) do + import Supervisor.Spec + + consumer_group_opts = [ + # setting for the ConsumerGroup + heartbeat_interval: 1_000, + # this setting will be forwarded to the GenConsumer + commit_interval: 1_000 + ] + + gen_consumer_impl = ExampleGenConsumer + consumer_group_name = "example_group" + topic_names = ["example_topic"] + + children = [ + # ... other children + supervisor( + KafkaEx.ConsumerGroup.Supervisor, + [gen_consumer_impl, consumer_group_name, topic_names, consumer_group_opts] + ) + ] + + Supervisor.start_link(children, strategy: :one_for_one) end end - - # use DistributedConsumer in a consumer group - {:ok, pid} = KafkaEx.ConsumerGroup.Supervisor.start_link(DistributedConsumer, "test_group", ["test_topic"]) ``` - Running this on multiple nodes might display the following: - - ```txt - node1@host: "messages" - node2@host: "on" - node2@host: "multiple" - node1@host: "nodes" - ``` + **Note** It is not necessary for the Elixir nodes in a consumer group to be + connected (i.e., using distributed Erlang methods). The coordination of + group consumers is mediated by the broker. - It is not necessary for the nodes to be connected, because `ConsumerGroup` - uses Kafka's built-in group coordination protocol. + See `KafkaEx.ConsumerGroup.Supervisor.start_link/4` for configuration + details. """ use GenServer @@ -100,12 +110,8 @@ defmodule KafkaEx.ConsumerGroup do # Client API - @doc """ - Starts a `ConsumerGroup` process linked to the current process. - - Client programs should use `KafkaEx.ConsumerGroup.Supervisor.start_link/4` - instead. - """ + @doc false + # use `KafkaEx.ConsumerGroup.Supervisor.start_link/4` instead @spec start_link(module, binary, [binary], KafkaEx.GenConsumer.options) :: GenServer.on_start def start_link(consumer_module, group_name, topics, opts \\ []) do diff --git a/lib/kafka_ex/consumer_group/supervisor.ex b/lib/kafka_ex/consumer_group/supervisor.ex index 4551eeec..f384451d 100644 --- a/lib/kafka_ex/consumer_group/supervisor.ex +++ b/lib/kafka_ex/consumer_group/supervisor.ex @@ -1,18 +1,12 @@ defmodule KafkaEx.ConsumerGroup.Supervisor do @moduledoc """ - A supervisor for managing a consumer group. A - `KafkaEx.ConsumerGroup.Supervisor` process will manage an entire process tree - for a single consumer group. Multiple supervisors can be used for multiple - consumer groups within the same application. + A supervisor for managing a consumer group. + + A `KafkaEx.ConsumerGroup.Supervisor` process manages the entire process + tree for a single consumer group. Multiple supervisors can be used for + multiple consumer groups within the same application. - ## Example - - This supervisor can be addeded to an application's supervision tree with a - custom `GenConsumer` implementation with the following child spec: - - ``` - supervisor(KafkaEx.ConsumerGroup.Supervisor, [MyApp.Consumer, "group_name", ["topic1", "topic2"]]) - ``` + See `KafkaEx.ConsumerGroup` for an example. """ use Elixir.Supervisor @@ -44,10 +38,6 @@ defmodule KafkaEx.ConsumerGroup.Supervisor do ### Return Values This function has the same return values as `Supervisor.start_link/3`. - - If the supervisor and consumer group are successfully created and - initialized, this function returns `{:ok, pid}`, where `pid` is the PID of - the consumer group supervisor process. """ @spec start_link(module, binary, [binary], options) :: Elixir.Supervisor.on_start @@ -87,6 +77,7 @@ defmodule KafkaEx.ConsumerGroup.Supervisor do end end + @doc false def init({consumer_module, group_name, topics, opts}) do opts = Keyword.put(opts, :supervisor_pid, self()) diff --git a/lib/kafka_ex/gen_consumer.ex b/lib/kafka_ex/gen_consumer.ex index 4168eee0..6d3ef9f1 100644 --- a/lib/kafka_ex/gen_consumer.ex +++ b/lib/kafka_ex/gen_consumer.ex @@ -2,20 +2,26 @@ defmodule KafkaEx.GenConsumer do @moduledoc """ A behaviour module for implementing a Kafka consumer. - A `GenConsumer` is an Elixir process that consumes messages from Kafka. A - single `GenConsumer` process consumes from a single partition of a Kafka - topic. Several `GenConsumer` processes can be used to consume from multiple - partitions or even multiple topics. Partition assignments for a group of - `GenConsumer`s can be defined manually using `KafkaEx.GenConsumer.Supervisor` - or coordinated across a cluster of nodes using `KafkaEx.ConsumerGroup`. + A `KafkaEx.GenConsumer` is an Elixir process that consumes messages from + Kafka. A single `KafkaEx.GenConsumer` process consumes from a single + partition of a Kafka topic. Several `KafkaEx.GenConsumer` processes can be + used to consume from multiple partitions or even multiple topics. Partition + assignments for a group of `KafkaEx.GenConsumer`s can be defined manually + using `KafkaEx.GenConsumer.Supervisor` or coordinated across a cluster of + nodes using `KafkaEx.ConsumerGroup`. + + A `KafkaEx.GenConsumer` must implement three callbacks. Two of these will be + defined with default behavior if you add `use KafkaEx.GenConsumer` to your + module, leaving just `c:handle_message_set/2` to be implemented. This is the + recommended usage. ## Example - The `GenConsumer` behaviour abstracts common Kafka consumer interactions. - `GenConsumer` will take care of the details of determining a starting offset, - fetching messages from a Kafka broker, and committing offsets for consumed - messages. Developers are only required to implement `c:handle_message_set/2` - to process message sets. + The `KafkaEx.GenConsumer` behaviour abstracts common Kafka consumer + interactions. `KafkaEx.GenConsumer` will take care of the details of + determining a starting offset, fetching messages from a Kafka broker, and + committing offsets for consumed messages. Developers are only required to + implement `c:handle_message_set/2` to process messages. The following is a minimal example that logs each message as it's consumed: @@ -40,13 +46,13 @@ defmodule KafkaEx.GenConsumer do `c:handle_message_set/2` will be called with the batch of messages fetched from the broker. The number of messages in a batch is determined by the number of messages available and the `max_bytes` and `min_bytes` parameters - of the fetch request. In this example, because `c:handle_message_set/2` - always returns `{:async_commit, new_state}`, the message offsets will be - automatically committed asynchronously. + of the fetch request (which can be configured in KafkaEx). In this example, + because `c:handle_message_set/2` always returns `{:async_commit, new_state}`, + the message offsets will be automatically committed asynchronously. ## Committing Offsets - `GenConsumer` manages a consumer's offsets by committing the the offsets + `KafkaEx.GenConsumer` manages a consumer's offsets by committing the the offsets of consumed messages. KafkaEx supports two commit strategies: asynchronous and synchronous. The return value of `c:handle_message_set/2` determines which strategy is used: @@ -54,29 +60,47 @@ defmodule KafkaEx.GenConsumer do * `{:sync_commit, new_state}` causes synchronous offset commits. * `{:async_commit, new_state}` causes asynchronous offset commits. + Note that with both of the offset commit strategies, only of the final offset + in the message set is committed and this is done after the messages are + consumed. If you want to commit the offset of every message consumed, use + the synchronous offset commit strategy and implement calls to + `KafkaEx.offset_commit/2` within your consumer as appropriate. + ### Synchronous offset commits When `c:handle_message_set/2` returns `{:sync_commit, new_state}`, the offset - of the final message in the batch is committed immediately before consuming - any more messages. This strategy causes significantly more communication - with the broker and will correspondingly degrade performance, but it will - keep the offset commits tightly synchronized with the consumer state. + of the final message in the message set is committed immediately before + fetching any more messages. This strategy requires a significant amount of + communication with the broker and could correspondingly degrade consumer + performance, but it will keep the offset commits tightly synchronized with + the consumer state. Choose the synchronous offset commit strategy if you want to favor consistency of offset commits over performance, or if you have a low rate of - message arrival (example: tens of messages per second or less). + message arrival. The definition of a "low rate" depends on the situation, + but tens of messages per second could be considered a "low rate" in most + situations. ### Asynchronous offset commits When `c:handle_message_set/2` returns `{:async_commit, new_state}`, KafkaEx - will not commit offsets after every message batch consumed. To avoid + will not commit offsets after every message set consumed. To avoid excessive network calls, the offsets are committed periodically (and when the worker terminates). - How often a `GenConsumer` auto-commits offsets is controlled by the two - configuration values `:commit_interval` and `:commit_threshold`. These can be - set globally in the `:kafka_ex` app's environment or on a per-consumer basis - by passing options to `start_link/5`: + How often a `KafkaEx.GenConsumer` auto-commits offsets is controlled by the two + configuration values `:commit_interval` and `:commit_threshold`. + + * `:commit_interval` is the maximum time (in milliseconds) that a + `KafkaEx.GenConsumer` will delay committing the offset for an acknowledged + message. + + * `:commit_threshold` is the maximum number of acknowledged messages that a + `KafkaEx.GenConsumer` will allow to be uncommitted before triggering a + commit. + + These can be set globally in the `:kafka_ex` app's environment or on a + per-consumer basis by passing options to `start_link/5`: ``` # In config/config.exs @@ -90,34 +114,22 @@ defmodule KafkaEx.GenConsumer do commit_threshold: 100) ``` - * `:commit_interval` is the maximum time (in milliseconds) that a - `GenConsumer` will delay committing the offset for an acknowledged message. - * `:commit_threshold` is the maximum number of acknowledged messages that a - `GenConsumer` will allow to be uncommitted before triggering an auto-commit. - For low-volume topics, `:commit_interval` is the dominant factor for how - often a `GenConsumer` auto-commits. For high-volume topics, + often a `KafkaEx.GenConsumer` auto-commits. For high-volume topics, `:commit_threshold` is the dominant factor. - ## Callbacks - - There are three callbacks that are required to be implemented in a - `GenConsumer`. By adding `use KafkaEx.GenServer` to a module, two of the - callbacks will be defined with default behavior, leaving you to implement - `c:handle_message_set/2`. - ## Integration with OTP - A `GenConsumer` is a specialized `GenServer`. It can be supervised, - registered, and debugged the same as any other `GenServer`. However, its - arguments for `c:GenServer.init/1` are unspecified, so `start_link/5` should - be used to start a `GenConsumer` process instead of `GenServer` primitives. + A `KafkaEx.GenConsumer` is a specialized `GenServer`. It can be supervised, + registered, and debugged the same as any other `GenServer`. Use + `start_link/5` to start a `KafkaEx.GenConsumer` properly; do not use + `GenServer.start_link/3` directly to start a `KafkaEx.GenConsumer`. ## Testing - A `GenConsumer` can be unit-tested without a running Kafka broker by sending + A `KafkaEx.GenConsumer` can be unit-tested without a running Kafka broker by sending messages directly to its `c:handle_message_set/2` function. The following - recipe can be used as a starting point when testing a `GenConsumer`: + recipe can be used as a starting point when testing a `KafkaEx.GenConsumer`: ``` defmodule ExampleGenConsumerTest do @@ -156,33 +168,33 @@ defmodule KafkaEx.GenConsumer do require Logger @typedoc """ - The ID of a member of a consumer group, assigned by a Kafka broker. + The ID (string) of a member of a consumer group, assigned by a Kafka broker. """ @type member_id :: binary @typedoc """ - The name of a Kafka topic. + The string name of a Kafka topic. """ @type topic :: binary @typedoc """ - The ID of a partition of a Kafka topic. + The integer ID of a partition of a Kafka topic. """ @type partition_id :: integer @typedoc """ - A partition of a particular Kafka topic. + A partition of a single topic (embeds the name of the topic). """ @type partition :: {topic, partition_id} @typedoc """ - Option values used when starting a `GenConsumer`. + Option values used when starting a `KafkaEx.GenConsumer`. """ @type option :: {:commit_interval, non_neg_integer} | {:commit_threshold, non_neg_integer} @typedoc """ - Options used when starting a `GenConsumer`. + Options used when starting a `KafkaEx.GenConsumer`. """ @type options :: [option | GenServer.option] @@ -191,7 +203,7 @@ defmodule KafkaEx.GenConsumer do returns. `topic` and `partition` are the arguments passed to `start_link/5`. They - identify the Kafka partition that the `GenConsumer` will consume from. + identify the Kafka partition that the `KafkaEx.GenConsumer` will consume from. Returning `{:ok, state}` will cause `start_link/5` to return `{:ok, pid}` and the process to start consuming from its assigned partition. `state` becomes @@ -207,7 +219,7 @@ defmodule KafkaEx.GenConsumer do Invoked for each message set consumed from a Kafka topic partition. `message_set` is a message set fetched from a Kafka broker and `state` is the - current state of the `GenConsumer`. + current state of the `KafkaEx.GenConsumer`. Returning `{:async_commit, new_state}` acknowledges `message` and continues to consume from the Kafka queue with new state `new_state`. Acknowledged @@ -257,8 +269,8 @@ defmodule KafkaEx.GenConsumer do ``` In this case, the consumer group process for `"member1"` will launch two - `GenConsumer` processes (one for each of its assigned partitions), - `"member2"` will launch one `GenConsumer` process, and `"member3"` will + `KafkaEx.GenConsumer` processes (one for each of its assigned partitions), + `"member2"` will launch one `KafkaEx.GenConsumer` process, and `"member3"` will launch no processes. """ @callback assign_partitions( @@ -310,33 +322,32 @@ defmodule KafkaEx.GenConsumer do # Client API @doc """ - Starts a `GenConsumer` process linked to the current process. + Starts a `KafkaEx.GenConsumer` process linked to the current process. - This can be used to start the `GenConsumer` as part of a supervision tree. + This can be used to start the `KafkaEx.GenConsumer` as part of a supervision tree. - Once the consumer has been started, the `c:init/2` function of the given - `consumer_module` is called with the given `topic` and `partition`. + Once the consumer has been started, the `c:init/2` function of + `consumer_module` is called with `topic` and `partition` as its arguments. `group_name` is the consumer group name that will be used for managing consumer offsets. ### Options - * `:commit_interval` - the interval in milliseconds that the consumer will - wait to commit acknowledged messages. If not present, the `:commit_interval` - environment value is used. + * `:commit_interval` - The interval in milliseconds that the consumer will + wait to commit offsets of handled messages. Default 5_000. + + * `:commit_threshold` - Threshold number of messages consumed to commit + offsets to the broker. Default 100. - * `:commit_threshold` - the maximum number of messages that can be - acknowledged without being committed. If not present, the `:commit_threshold` - environment value is used. + Both `:commit_interval` and `:commit_threshold` default to the application + config (e.g., `Application.get_env/2`) if that value is present, or the + stated default if the application config is not present. Any valid options for `GenServer.start_link/3` can also be specified. ### Return Values This function has the same return values as `GenServer.start_link/3`. - - If the consumer is successfully created and initialized, this function - returns `{:ok, pid}`, where `pid` is the PID of the consumer process. """ @spec start_link( module, From 90d93b8a7c6b7aa1978b1798147ca3aaf793a098 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 31 Jul 2017 13:16:02 -0400 Subject: [PATCH 18/47] Refactor st ConsumerGroup is the supervisor This makes more intuitive sense - one wouldn't really start the manager process without the supervisor. --- README.md | 2 +- lib/kafka_ex/consumer_group.ex | 458 +++------------------- lib/kafka_ex/consumer_group/manager.ex | 424 ++++++++++++++++++++ lib/kafka_ex/consumer_group/supervisor.ex | 93 ----- 4 files changed, 487 insertions(+), 490 deletions(-) create mode 100644 lib/kafka_ex/consumer_group/manager.ex delete mode 100644 lib/kafka_ex/consumer_group/supervisor.ex diff --git a/README.md b/README.md index 29b913b2..1c50034a 100644 --- a/README.md +++ b/README.md @@ -100,7 +100,7 @@ defmodule ExampleGenConsumer do end ``` -Then add a `KafkaEx.ConsumerGroup.Supervisor` to your application's supervision +Then add a `KafkaEx.ConsumerGroup` to your application's supervision tree and configure it to use the implementation module. See the `KafkaEx.GenConsumer` and `KafkaEx.ConsumerGroup` documentation for diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 6120c583..d41d195b 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -54,7 +54,7 @@ defmodule KafkaEx.ConsumerGroup do children = [ # ... other children supervisor( - KafkaEx.ConsumerGroup.Supervisor, + KafkaEx.ConsumerGroup, [gen_consumer_impl, consumer_group_name, topic_names, consumer_group_opts] ) ] @@ -72,422 +72,88 @@ defmodule KafkaEx.ConsumerGroup do details. """ - use GenServer - - alias KafkaEx.Protocol.JoinGroup.Request, as: JoinGroupRequest - alias KafkaEx.Protocol.JoinGroup.Response, as: JoinGroupResponse - alias KafkaEx.Protocol.Heartbeat.Request, as: HeartbeatRequest - alias KafkaEx.Protocol.Heartbeat.Response, as: HeartbeatResponse - alias KafkaEx.Protocol.LeaveGroup.Request, as: LeaveGroupRequest - alias KafkaEx.Protocol.LeaveGroup.Response, as: LeaveGroupResponse - alias KafkaEx.Protocol.Metadata.Response, as: MetadataResponse - alias KafkaEx.Protocol.SyncGroup.Request, as: SyncGroupRequest - alias KafkaEx.Protocol.SyncGroup.Response, as: SyncGroupResponse - - require Logger - - defmodule State do - @moduledoc false - defstruct [ - :supervisor_pid, - :worker_name, - :heartbeat_interval, - :session_timeout, - :consumer_module, - :consumer_opts, - :group_name, - :topics, - :member_id, - :generation_id, - :assignments, - :heartbeat_timer, - ] - end - - @heartbeat_interval 5_000 - @session_timeout 30_000 - @session_timeout_padding 5_000 - - # Client API - - @doc false - # use `KafkaEx.ConsumerGroup.Supervisor.start_link/4` instead - @spec start_link(module, binary, [binary], KafkaEx.GenConsumer.options) :: - GenServer.on_start - def start_link(consumer_module, group_name, topics, opts \\ []) do - {server_opts, consumer_opts} = - Keyword.split(opts, [:debug, :name, :timeout, :spawn_opt]) - - GenServer.start_link( - __MODULE__, - {consumer_module, group_name, topics, consumer_opts}, - server_opts - ) - end - - # GenServer callbacks - - def init({consumer_module, group_name, topics, opts}) do - heartbeat_interval = Keyword.get( - opts, - :heartbeat_interval, - Application.get_env(:kafka_ex, :heartbeat_interval, @heartbeat_interval) - ) - session_timeout = Keyword.get( - opts, - :session_timeout, - Application.get_env(:kafka_ex, :session_timeout, @session_timeout) - ) - - supervisor_pid = Keyword.fetch!(opts, :supervisor_pid) - consumer_opts = Keyword.drop( - opts, [:supervisor_pid, :heartbeat_interval, :session_timeout] - ) + use Supervisor - {:ok, worker_name} = - KafkaEx.create_worker(:no_name, consumer_group: group_name) + @typedoc """ + Option values used when starting a consumer group - state = %State{ - supervisor_pid: supervisor_pid, - worker_name: worker_name, - heartbeat_interval: heartbeat_interval, - session_timeout: session_timeout, - consumer_module: consumer_module, - consumer_opts: consumer_opts, - group_name: group_name, - topics: topics, - member_id: "", - } - - Process.flag(:trap_exit, true) - - {:ok, state, 0} - end + * Any of `KafkaEx.GenConsumer.option`, which will be passed on to consumers + * `:name` - Name for the consumer group supervisor + * `:max_restarts`, `:max_seconds` - Supervisor restart policy parameters + """ + @type option :: KafkaEx.GenConsumer.option + | {:name, Elixir.Supervisor.name} + | {:max_restarts, non_neg_integer} + | {:max_seconds, non_neg_integer} - # If `member_id` and `generation_id` aren't set, we haven't yet joined the - # group. `member_id` and `generation_id` are initialized by - # `JoinGroupResponse`. - def handle_info( - :timeout, %State{generation_id: nil, member_id: ""} = state - ) do - {:ok, new_state} = join(state) + @type options :: [option] - {:noreply, new_state} - end + @doc """ + Starts a consumer group process tree process linked to the current process. - # After joining the group, a member must periodically send heartbeats to the - # group coordinator. - def handle_info(:heartbeat, %State{} = state) do - {:ok, new_state} = heartbeat(state) + This can be used to start a `KafkaEx.ConsumerGroup` as part of a supervision + tree. - {:noreply, new_state} - end + `module` is a module that implements the `KafkaEx.GenConsumer` behaviour. + `group_name` is the name of the consumer group. `topics` is a list of topics + that the consumer group should consume from. `opts` can be composed of + options for the supervisor as well as for the `KafkEx.GenConsumer` processes + that will be spawned by the supervisor. See `t:option/0` for details. - # When terminating, inform the group coordinator that this member is leaving - # the group so that the group can rebalance without waiting for a session - # timeout. - def terminate(_reason, %State{generation_id: nil, member_id: ""}), do: :ok - def terminate(_reason, %State{} = state) do - :ok = leave(state) - end + ### Return Values - ### Helpers - - # `JoinGroupRequest` is used to set the active members of a group. The - # response blocks until the broker has decided that it has a full list of - # group members. This requires that all active members send a - # `JoinGroupRequest`. For active members, this is triggered by the broker - # responding to a heartbeat with a `:rebalance_in_progress` error code. If - # any group members fail to send a `JoinGroupRequest` before the session - # timeout expires, then those group members are removed from the group and - # synchronization continues without them. - # - # `JoinGroupResponse` tells each member its unique member ID as well as the - # group's current generation ID. The broker will pick one group member to be - # the leader, which is reponsible for assigning partitions to all of the - # group members. Once a `JoinGroupResponse` is received, all group members - # must send a `SyncGroupRequest` (see sync/2). - - defp join( - %State{ - worker_name: worker_name, - session_timeout: session_timeout, - group_name: group_name, - topics: topics, - member_id: member_id - } = state - ) do - join_request = %JoinGroupRequest{ - group_name: group_name, - member_id: member_id, - topics: topics, - session_timeout: session_timeout, - } - - join_response = %JoinGroupResponse{error_code: :no_error} = - KafkaEx.join_group( - join_request, - worker_name: worker_name, - timeout: session_timeout + @session_timeout_padding - ) - - Logger.debug(fn -> "Joined consumer group #{group_name}" end) - - new_state = %State{ - state | - member_id: join_response.member_id, - generation_id: join_response.generation_id - } - - assignments = - if JoinGroupResponse.leader?(join_response) do - # Leader is responsible for assigning partitions to all group members. - partitions = assignable_partitions(new_state) - assign_partitions(new_state, join_response.members, partitions) - else - # Follower does not assign partitions; must be empty. - [] - end - - sync(new_state, assignments) - end + This function has the same return values as `Supervisor.start_link/3`. + """ + @spec start_link(module, binary, [binary], options) :: + Elixir.Supervisor.on_start + def start_link(consumer_module, group_name, topics, opts \\ []) do + {supervisor_opts, module_opts} = + Keyword.split(opts, [:name, :strategy, :max_restarts, :max_seconds]) - # `SyncGroupRequest` is used to distribute partition assignments to all group - # members. All group members must send this request after receiving a - # response to a `JoinGroupRequest`. The request blocks until assignments are - # provided by the leader. The leader sends partition assignments (given by - # the `assignments` parameter) as part of its `SyncGroupRequest`. For all - # other members, `assignments` must be empty. - # - # `SyncGroupResponse` contains the individual member's partition assignments. - # Upon receiving a successful `SyncGroupResponse`, a group member is free to - # start consuming from its assigned partitions, but must send periodic - # heartbeats to the coordinating broker. - - defp sync( - %State{ - group_name: group_name, - member_id: member_id, - generation_id: generation_id, - worker_name: worker_name, - session_timeout: session_timeout - } = state, - assignments - ) do - sync_request = %SyncGroupRequest{ - group_name: group_name, - member_id: member_id, - generation_id: generation_id, - assignments: assignments, - } - - sync_group_response = KafkaEx.sync_group( - sync_request, - worker_name: worker_name, - timeout: session_timeout + @session_timeout_padding + Elixir.Supervisor.start_link( + __MODULE__, + {consumer_module, group_name, topics, module_opts}, + supervisor_opts ) - - case sync_group_response do - %SyncGroupResponse{error_code: :no_error, assignments: assignments} -> - new_state = state - |> start_consumer(unpack_assignments(assignments)) - |> start_heartbeat_timer() - {:ok, new_state} - %SyncGroupResponse{error_code: :rebalance_in_progress} -> - rebalance(state) - end - end - - # `HeartbeatRequest` is sent periodically by each active group member (after - # completing the join/sync phase) to inform the broker that the member is - # still alive and participating in the group. If a group member fails to send - # a heartbeat before the group's session timeout expires, the coordinator - # removes that member from the group and initiates a rebalance. - # - # `HeartbeatResponse` allows the coordinating broker to communicate the - # group's status to each member: - # - # * `:no_error` indicates that the group is up to date and no action is - # needed. * `:rebalance_in_progress` instructs each member to rejoin the - # group by sending a `JoinGroupRequest` (see join/1). - defp heartbeat( - %State{ - worker_name: worker_name, - group_name: group_name, - generation_id: generation_id, - member_id: member_id - } = state - ) do - heartbeat_request = %HeartbeatRequest{ - group_name: group_name, - member_id: member_id, - generation_id: generation_id, - } - - case KafkaEx.heartbeat(heartbeat_request, worker_name: worker_name) do - %HeartbeatResponse{error_code: :no_error} -> - new_state = start_heartbeat_timer(state) - {:ok, new_state} - %HeartbeatResponse{error_code: :rebalance_in_progress} -> - Logger.debug(fn -> "Rebalancing consumer group #{group_name}" end) - rebalance(state) - end - end - - # `LeaveGroupRequest` is used to voluntarily leave a group. This tells the - # broker that the member is leaving the group without having to wait for the - # session timeout to expire. Leaving a group triggers a rebalance for the - # remaining group members. - defp leave( - %State{ - worker_name: worker_name, - group_name: group_name, - member_id: member_id - } = state - ) do - stop_heartbeat_timer(state) - - leave_request = %LeaveGroupRequest{ - group_name: group_name, - member_id: member_id, - } - - %LeaveGroupResponse{error_code: :no_error} = - KafkaEx.leave_group(leave_request, worker_name: worker_name) - - Logger.debug(fn -> "Left consumer group #{group_name}" end) - - :ok - end - - # When instructed that a rebalance is in progress, a group member must rejoin - # the group with `JoinGroupRequest` (see join/1). To keep the state - # synchronized during the join/sync phase, each member pauses its consumers - # and commits its offsets before rejoining the group. - defp rebalance(%State{} = state) do - state - |> stop_heartbeat_timer() - |> stop_consumer() - |> join() - end - - ### Timer Management - - # Starts a timer for the next heartbeat. - defp start_heartbeat_timer( - %State{heartbeat_interval: heartbeat_interval} = state - ) do - {:ok, timer} = :timer.send_after(heartbeat_interval, :heartbeat) - - %State{state | heartbeat_timer: timer} end - # Stops any active heartbeat timer. - defp stop_heartbeat_timer(%State{heartbeat_timer: nil} = state), do: state - defp stop_heartbeat_timer( - %State{heartbeat_timer: heartbeat_timer} = state - ) do - {:ok, :cancel} = :timer.cancel(heartbeat_timer) + @doc false # used by ConsumerGroup to set partition assignments + def start_consumer(pid, consumer_module, group_name, assignments, opts) do - %State{state | heartbeat_timer: nil} - end - - ### Consumer Management - - # Starts consuming from the member's assigned partitions. - defp start_consumer( - %State{ - consumer_module: consumer_module, - consumer_opts: consumer_opts, - group_name: group_name, - supervisor_pid: pid - } = state, - assignments - ) do - :ok = KafkaEx.ConsumerGroup.Supervisor.start_consumer( - pid, - consumer_module, - group_name, - assignments, - consumer_opts + child = supervisor( + KafkaEx.GenConsumer.Supervisor, + [consumer_module, group_name, assignments, opts], + id: :consumer ) - state - end - - # Stops consuming from the member's assigned partitions and commits offsets. - defp stop_consumer(%State{supervisor_pid: pid} = state) do - :ok = KafkaEx.ConsumerGroup.Supervisor.stop_consumer(pid) - - state + case Elixir.Supervisor.start_child(pid, child) do + {:ok, _child} -> :ok + {:ok, _child, _info} -> :ok + end end - ### Partition Assignment - - # Queries the Kafka brokers for a list of partitions for the topics of - # interest to this consumer group. This function returns a list of - # topic/partition tuples that can be passed to a GenConsumer's - # `assign_partitions` method. - defp assignable_partitions( - %State{worker_name: worker_name, topics: topics} - ) do - metadata = KafkaEx.metadata(worker_name: worker_name) + @doc false # used by ConsumerGroup to pause consumption during rebalance + def stop_consumer(pid) do + case Elixir.Supervisor.terminate_child(pid, :consumer) do + :ok -> + Elixir.Supervisor.delete_child(pid, :consumer) - Enum.flat_map(topics, fn (topic) -> - partitions = MetadataResponse.partitions_for_topic(metadata, topic) - - Enum.map(partitions, fn (partition) -> - {topic, partition} - end) - end) + {:error, :not_found} -> + :ok + end end - # This function is used by the group leader to determine partition - # assignments during the join/sync phase. `members` is provided to the leader - # by the coordinating broker in `JoinGroupResponse`. `partitions` is a list - # of topic/partition tuples, obtained from `assignable_partitions/1`. The - # return value is a complete list of member assignments in the format needed - # by `SyncGroupResponse`. - defp assign_partitions( - %State{consumer_module: consumer_module}, - members, - partitions - ) do - # Delegate partition assignment to GenConsumer module. - assignments = consumer_module.assign_partitions(members, partitions) - - # Convert assignments to format expected by Kafka protocol. - packed_assignments = - Enum.map(assignments, fn ({member, topic_partitions}) -> - {member, pack_assignments(topic_partitions)} - end) - assignments_map = Map.new(packed_assignments) - - # Fill in empty assignments for missing member IDs. - Enum.map(members, fn (member) -> - {member, Map.get(assignments_map, member, [])} - end) - end + @doc false + def init({consumer_module, group_name, topics, opts}) do + opts = Keyword.put(opts, :supervisor_pid, self()) - # Converts assignments from Kafka's protocol format to topic/partition tuples. - # - # Example: - # - # unpack_assignments([{"foo", [0, 1]}]) #=> [{"foo", 0}, {"foo", 1}] - defp unpack_assignments(assignments) do - Enum.flat_map(assignments, fn ({topic, partition_ids}) -> - Enum.map(partition_ids, &({topic, &1})) - end) - end + children = [ + worker( + KafkaEx.ConsumerGroup.Manager, + [consumer_module, group_name, topics, opts] + ), + ] - # Converts assignments from topic/partition tuples to Kafka's protocol format. - # - # Example: - # - # pack_assignments([{"foo", 0}, {"foo", 1}]) #=> [{"foo", [0, 1]}] - defp pack_assignments(assignments) do - assignments - |> Enum.group_by(&(elem(&1, 0)), &(elem(&1, 1))) - |> Enum.into([]) + supervise(children, strategy: :one_for_all) end end diff --git a/lib/kafka_ex/consumer_group/manager.ex b/lib/kafka_ex/consumer_group/manager.ex new file mode 100644 index 00000000..3943b685 --- /dev/null +++ b/lib/kafka_ex/consumer_group/manager.ex @@ -0,0 +1,424 @@ +defmodule KafkaEx.ConsumerGroup.Manager do + @moduledoc false + + # actual consumer group management implementation + + use GenServer + + alias KafkaEx.Protocol.JoinGroup.Request, as: JoinGroupRequest + alias KafkaEx.Protocol.JoinGroup.Response, as: JoinGroupResponse + alias KafkaEx.Protocol.Heartbeat.Request, as: HeartbeatRequest + alias KafkaEx.Protocol.Heartbeat.Response, as: HeartbeatResponse + alias KafkaEx.Protocol.LeaveGroup.Request, as: LeaveGroupRequest + alias KafkaEx.Protocol.LeaveGroup.Response, as: LeaveGroupResponse + alias KafkaEx.Protocol.Metadata.Response, as: MetadataResponse + alias KafkaEx.Protocol.SyncGroup.Request, as: SyncGroupRequest + alias KafkaEx.Protocol.SyncGroup.Response, as: SyncGroupResponse + + require Logger + + defmodule State do + @moduledoc false + defstruct [ + :supervisor_pid, + :worker_name, + :heartbeat_interval, + :session_timeout, + :consumer_module, + :consumer_opts, + :group_name, + :topics, + :member_id, + :generation_id, + :assignments, + :heartbeat_timer, + ] + end + + @heartbeat_interval 5_000 + @session_timeout 30_000 + @session_timeout_padding 5_000 + + # Client API + + @doc false + # use `KafkaEx.ConsumerGroup.Supervisor.start_link/4` instead + @spec start_link(module, binary, [binary], KafkaEx.GenConsumer.options) :: + GenServer.on_start + def start_link(consumer_module, group_name, topics, opts \\ []) do + {server_opts, consumer_opts} = + Keyword.split(opts, [:debug, :name, :timeout, :spawn_opt]) + + GenServer.start_link( + __MODULE__, + {consumer_module, group_name, topics, consumer_opts}, + server_opts + ) + end + + # GenServer callbacks + + def init({consumer_module, group_name, topics, opts}) do + heartbeat_interval = Keyword.get( + opts, + :heartbeat_interval, + Application.get_env(:kafka_ex, :heartbeat_interval, @heartbeat_interval) + ) + session_timeout = Keyword.get( + opts, + :session_timeout, + Application.get_env(:kafka_ex, :session_timeout, @session_timeout) + ) + + supervisor_pid = Keyword.fetch!(opts, :supervisor_pid) + consumer_opts = Keyword.drop( + opts, [:supervisor_pid, :heartbeat_interval, :session_timeout] + ) + + {:ok, worker_name} = + KafkaEx.create_worker(:no_name, consumer_group: group_name) + + state = %State{ + supervisor_pid: supervisor_pid, + worker_name: worker_name, + heartbeat_interval: heartbeat_interval, + session_timeout: session_timeout, + consumer_module: consumer_module, + consumer_opts: consumer_opts, + group_name: group_name, + topics: topics, + member_id: "", + } + + Process.flag(:trap_exit, true) + + {:ok, state, 0} + end + + # If `member_id` and `generation_id` aren't set, we haven't yet joined the + # group. `member_id` and `generation_id` are initialized by + # `JoinGroupResponse`. + def handle_info( + :timeout, %State{generation_id: nil, member_id: ""} = state + ) do + {:ok, new_state} = join(state) + + {:noreply, new_state} + end + + # After joining the group, a member must periodically send heartbeats to the + # group coordinator. + def handle_info(:heartbeat, %State{} = state) do + {:ok, new_state} = heartbeat(state) + + {:noreply, new_state} + end + + # When terminating, inform the group coordinator that this member is leaving + # the group so that the group can rebalance without waiting for a session + # timeout. + def terminate(_reason, %State{generation_id: nil, member_id: ""}), do: :ok + def terminate(_reason, %State{} = state) do + :ok = leave(state) + end + + ### Helpers + + # `JoinGroupRequest` is used to set the active members of a group. The + # response blocks until the broker has decided that it has a full list of + # group members. This requires that all active members send a + # `JoinGroupRequest`. For active members, this is triggered by the broker + # responding to a heartbeat with a `:rebalance_in_progress` error code. If + # any group members fail to send a `JoinGroupRequest` before the session + # timeout expires, then those group members are removed from the group and + # synchronization continues without them. + # + # `JoinGroupResponse` tells each member its unique member ID as well as the + # group's current generation ID. The broker will pick one group member to be + # the leader, which is reponsible for assigning partitions to all of the + # group members. Once a `JoinGroupResponse` is received, all group members + # must send a `SyncGroupRequest` (see sync/2). + + defp join( + %State{ + worker_name: worker_name, + session_timeout: session_timeout, + group_name: group_name, + topics: topics, + member_id: member_id + } = state + ) do + join_request = %JoinGroupRequest{ + group_name: group_name, + member_id: member_id, + topics: topics, + session_timeout: session_timeout, + } + + join_response = %JoinGroupResponse{error_code: :no_error} = + KafkaEx.join_group( + join_request, + worker_name: worker_name, + timeout: session_timeout + @session_timeout_padding + ) + + Logger.debug(fn -> "Joined consumer group #{group_name}" end) + + new_state = %State{ + state | + member_id: join_response.member_id, + generation_id: join_response.generation_id + } + + assignments = + if JoinGroupResponse.leader?(join_response) do + # Leader is responsible for assigning partitions to all group members. + partitions = assignable_partitions(new_state) + assign_partitions(new_state, join_response.members, partitions) + else + # Follower does not assign partitions; must be empty. + [] + end + + sync(new_state, assignments) + end + + # `SyncGroupRequest` is used to distribute partition assignments to all group + # members. All group members must send this request after receiving a + # response to a `JoinGroupRequest`. The request blocks until assignments are + # provided by the leader. The leader sends partition assignments (given by + # the `assignments` parameter) as part of its `SyncGroupRequest`. For all + # other members, `assignments` must be empty. + # + # `SyncGroupResponse` contains the individual member's partition assignments. + # Upon receiving a successful `SyncGroupResponse`, a group member is free to + # start consuming from its assigned partitions, but must send periodic + # heartbeats to the coordinating broker. + + defp sync( + %State{ + group_name: group_name, + member_id: member_id, + generation_id: generation_id, + worker_name: worker_name, + session_timeout: session_timeout + } = state, + assignments + ) do + sync_request = %SyncGroupRequest{ + group_name: group_name, + member_id: member_id, + generation_id: generation_id, + assignments: assignments, + } + + sync_group_response = KafkaEx.sync_group( + sync_request, + worker_name: worker_name, + timeout: session_timeout + @session_timeout_padding + ) + + case sync_group_response do + %SyncGroupResponse{error_code: :no_error, assignments: assignments} -> + new_state = state + |> start_consumer(unpack_assignments(assignments)) + |> start_heartbeat_timer() + {:ok, new_state} + %SyncGroupResponse{error_code: :rebalance_in_progress} -> + rebalance(state) + end + end + + # `HeartbeatRequest` is sent periodically by each active group member (after + # completing the join/sync phase) to inform the broker that the member is + # still alive and participating in the group. If a group member fails to send + # a heartbeat before the group's session timeout expires, the coordinator + # removes that member from the group and initiates a rebalance. + # + # `HeartbeatResponse` allows the coordinating broker to communicate the + # group's status to each member: + # + # * `:no_error` indicates that the group is up to date and no action is + # needed. * `:rebalance_in_progress` instructs each member to rejoin the + # group by sending a `JoinGroupRequest` (see join/1). + defp heartbeat( + %State{ + worker_name: worker_name, + group_name: group_name, + generation_id: generation_id, + member_id: member_id + } = state + ) do + heartbeat_request = %HeartbeatRequest{ + group_name: group_name, + member_id: member_id, + generation_id: generation_id, + } + + case KafkaEx.heartbeat(heartbeat_request, worker_name: worker_name) do + %HeartbeatResponse{error_code: :no_error} -> + new_state = start_heartbeat_timer(state) + {:ok, new_state} + %HeartbeatResponse{error_code: :rebalance_in_progress} -> + Logger.debug(fn -> "Rebalancing consumer group #{group_name}" end) + rebalance(state) + end + end + + # `LeaveGroupRequest` is used to voluntarily leave a group. This tells the + # broker that the member is leaving the group without having to wait for the + # session timeout to expire. Leaving a group triggers a rebalance for the + # remaining group members. + defp leave( + %State{ + worker_name: worker_name, + group_name: group_name, + member_id: member_id + } = state + ) do + stop_heartbeat_timer(state) + + leave_request = %LeaveGroupRequest{ + group_name: group_name, + member_id: member_id, + } + + %LeaveGroupResponse{error_code: :no_error} = + KafkaEx.leave_group(leave_request, worker_name: worker_name) + + Logger.debug(fn -> "Left consumer group #{group_name}" end) + + :ok + end + + # When instructed that a rebalance is in progress, a group member must rejoin + # the group with `JoinGroupRequest` (see join/1). To keep the state + # synchronized during the join/sync phase, each member pauses its consumers + # and commits its offsets before rejoining the group. + defp rebalance(%State{} = state) do + state + |> stop_heartbeat_timer() + |> stop_consumer() + |> join() + end + + ### Timer Management + + # Starts a timer for the next heartbeat. + defp start_heartbeat_timer( + %State{heartbeat_interval: heartbeat_interval} = state + ) do + {:ok, timer} = :timer.send_after(heartbeat_interval, :heartbeat) + + %State{state | heartbeat_timer: timer} + end + + # Stops any active heartbeat timer. + defp stop_heartbeat_timer(%State{heartbeat_timer: nil} = state), do: state + defp stop_heartbeat_timer( + %State{heartbeat_timer: heartbeat_timer} = state + ) do + {:ok, :cancel} = :timer.cancel(heartbeat_timer) + + %State{state | heartbeat_timer: nil} + end + + ### Consumer Management + + # Starts consuming from the member's assigned partitions. + defp start_consumer( + %State{ + consumer_module: consumer_module, + consumer_opts: consumer_opts, + group_name: group_name, + supervisor_pid: pid + } = state, + assignments + ) do + :ok = KafkaEx.ConsumerGroup.start_consumer( + pid, + consumer_module, + group_name, + assignments, + consumer_opts + ) + + state + end + + # Stops consuming from the member's assigned partitions and commits offsets. + defp stop_consumer(%State{supervisor_pid: pid} = state) do + :ok = KafkaEx.ConsumerGroup.stop_consumer(pid) + + state + end + + ### Partition Assignment + + # Queries the Kafka brokers for a list of partitions for the topics of + # interest to this consumer group. This function returns a list of + # topic/partition tuples that can be passed to a GenConsumer's + # `assign_partitions` method. + defp assignable_partitions( + %State{worker_name: worker_name, topics: topics} + ) do + metadata = KafkaEx.metadata(worker_name: worker_name) + + Enum.flat_map(topics, fn (topic) -> + partitions = MetadataResponse.partitions_for_topic(metadata, topic) + + Enum.map(partitions, fn (partition) -> + {topic, partition} + end) + end) + end + + # This function is used by the group leader to determine partition + # assignments during the join/sync phase. `members` is provided to the leader + # by the coordinating broker in `JoinGroupResponse`. `partitions` is a list + # of topic/partition tuples, obtained from `assignable_partitions/1`. The + # return value is a complete list of member assignments in the format needed + # by `SyncGroupResponse`. + defp assign_partitions( + %State{consumer_module: consumer_module}, + members, + partitions + ) do + # Delegate partition assignment to GenConsumer module. + assignments = consumer_module.assign_partitions(members, partitions) + + # Convert assignments to format expected by Kafka protocol. + packed_assignments = + Enum.map(assignments, fn ({member, topic_partitions}) -> + {member, pack_assignments(topic_partitions)} + end) + assignments_map = Map.new(packed_assignments) + + # Fill in empty assignments for missing member IDs. + Enum.map(members, fn (member) -> + {member, Map.get(assignments_map, member, [])} + end) + end + + # Converts assignments from Kafka's protocol format to topic/partition tuples. + # + # Example: + # + # unpack_assignments([{"foo", [0, 1]}]) #=> [{"foo", 0}, {"foo", 1}] + defp unpack_assignments(assignments) do + Enum.flat_map(assignments, fn ({topic, partition_ids}) -> + Enum.map(partition_ids, &({topic, &1})) + end) + end + + # Converts assignments from topic/partition tuples to Kafka's protocol format. + # + # Example: + # + # pack_assignments([{"foo", 0}, {"foo", 1}]) #=> [{"foo", [0, 1]}] + defp pack_assignments(assignments) do + assignments + |> Enum.group_by(&(elem(&1, 0)), &(elem(&1, 1))) + |> Enum.into([]) + end +end diff --git a/lib/kafka_ex/consumer_group/supervisor.ex b/lib/kafka_ex/consumer_group/supervisor.ex deleted file mode 100644 index f384451d..00000000 --- a/lib/kafka_ex/consumer_group/supervisor.ex +++ /dev/null @@ -1,93 +0,0 @@ -defmodule KafkaEx.ConsumerGroup.Supervisor do - @moduledoc """ - A supervisor for managing a consumer group. - - A `KafkaEx.ConsumerGroup.Supervisor` process manages the entire process - tree for a single consumer group. Multiple supervisors can be used for - multiple consumer groups within the same application. - - See `KafkaEx.ConsumerGroup` for an example. - """ - - use Elixir.Supervisor - - @typedoc """ - Option values used when starting a `ConsumerGroup.Supervisor`. - """ - @type option :: KafkaEx.GenConsumer.option - | {:name, Elixir.Supervisor.name} - | {:max_restarts, non_neg_integer} - | {:max_seconds, non_neg_integer} - - @typedoc """ - Options used when starting a `ConsumerGroup.Supervisor`. - """ - @type options :: [option] - - @doc """ - Starts a `ConsumerGroup.Supervisor` process linked to the current process. - - This can be used to start a `KafkaEx.ConsumerGroup` as part of a supervision - tree. - - `module` is a module that implements the `KafkaEx.GenConsumer` behaviour. - `group_name` is the name of the consumer group. `topics` is a list of topics - that the consumer group should consume from. `opts` can be any options - accepted by `KafkaEx.ConsumerGroup` or `Supervisor`. - - ### Return Values - - This function has the same return values as `Supervisor.start_link/3`. - """ - @spec start_link(module, binary, [binary], options) :: - Elixir.Supervisor.on_start - def start_link(consumer_module, group_name, topics, opts \\ []) do - {supervisor_opts, module_opts} = - Keyword.split(opts, [:name, :strategy, :max_restarts, :max_seconds]) - - Elixir.Supervisor.start_link( - __MODULE__, - {consumer_module, group_name, topics, module_opts}, - supervisor_opts - ) - end - - @doc false # used by ConsumerGroup to set partition assignments - def start_consumer(pid, consumer_module, group_name, assignments, opts) do - child = supervisor( - KafkaEx.GenConsumer.Supervisor, - [consumer_module, group_name, assignments, opts], - id: :consumer - ) - - case Elixir.Supervisor.start_child(pid, child) do - {:ok, _child} -> :ok - {:ok, _child, _info} -> :ok - end - end - - @doc false # used by ConsumerGroup to pause consumption during rebalance - def stop_consumer(pid) do - case Elixir.Supervisor.terminate_child(pid, :consumer) do - :ok -> - Elixir.Supervisor.delete_child(pid, :consumer) - - {:error, :not_found} -> - :ok - end - end - - @doc false - def init({consumer_module, group_name, topics, opts}) do - opts = Keyword.put(opts, :supervisor_pid, self()) - - children = [ - worker( - KafkaEx.ConsumerGroup, - [consumer_module, group_name, topics, opts] - ), - ] - - supervise(children, strategy: :one_for_all) - end -end From eff912f9227525d30a0daadd4ae8643a69fce198 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 31 Jul 2017 13:18:59 -0400 Subject: [PATCH 19/47] Fix documentation oversight --- lib/kafka_ex/consumer_group.ex | 3 +-- lib/kafka_ex/consumer_group/manager.ex | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index d41d195b..23d75563 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -68,8 +68,7 @@ defmodule KafkaEx.ConsumerGroup do connected (i.e., using distributed Erlang methods). The coordination of group consumers is mediated by the broker. - See `KafkaEx.ConsumerGroup.Supervisor.start_link/4` for configuration - details. + See `start_link/4` for configuration details. """ use Supervisor diff --git a/lib/kafka_ex/consumer_group/manager.ex b/lib/kafka_ex/consumer_group/manager.ex index 3943b685..15667875 100644 --- a/lib/kafka_ex/consumer_group/manager.ex +++ b/lib/kafka_ex/consumer_group/manager.ex @@ -42,7 +42,7 @@ defmodule KafkaEx.ConsumerGroup.Manager do # Client API @doc false - # use `KafkaEx.ConsumerGroup.Supervisor.start_link/4` instead + # use `KafkaEx.ConsumerGroup.start_link/4` instead @spec start_link(module, binary, [binary], KafkaEx.GenConsumer.options) :: GenServer.on_start def start_link(consumer_module, group_name, topics, opts \\ []) do From 9c326961be09126996cbdda8f7b13e17c4522c21 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 31 Jul 2017 15:58:52 -0400 Subject: [PATCH 20/47] Fix default worker name for test helper --- test/test_helper.exs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/test_helper.exs b/test/test_helper.exs index 0efe9908..5629714b 100644 --- a/test/test_helper.exs +++ b/test/test_helper.exs @@ -51,7 +51,7 @@ defmodule TestHelper do {x, {a,b,c + 60}} end - def latest_offset_number(topic, partition_id, worker \\ KafkaEx.Server) do + def latest_offset_number(topic, partition_id, worker \\ :kafka_ex) do offset = KafkaEx.latest_offset(topic, partition_id, worker) |> first_partition_offset From 3a7719c73798d1479482b9711dc67f7c291ed1aa Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 31 Jul 2017 15:59:12 -0400 Subject: [PATCH 21/47] Create topic on docker up --- scripts/docker_up.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/scripts/docker_up.sh b/scripts/docker_up.sh index ef22a7d5..da117f27 100755 --- a/scripts/docker_up.sh +++ b/scripts/docker_up.sh @@ -46,3 +46,6 @@ do done docker-compose up -d + +# create topics needed for testing +docker-compose run --rm --no-deps kafka1 /bin/bash -c '${KAFKA_HOME}/bin/kafka-topics.sh --create --topic consumer_group_implementation_test --replication-factor 2 --partitions 4 --zookeeper zookeeper:2181' From 6187e9f58f907c04dce879976e53542bc07de041 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 31 Jul 2017 15:59:29 -0400 Subject: [PATCH 22/47] Add a basic integration test for the consumer group --- .../consumer_group_implementation_test.exs | 122 ++++++++++++++++++ 1 file changed, 122 insertions(+) create mode 100644 test/integration/consumer_group_implementation_test.exs diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs new file mode 100644 index 00000000..9ed9368d --- /dev/null +++ b/test/integration/consumer_group_implementation_test.exs @@ -0,0 +1,122 @@ +defmodule KafkaEx.ConsumerGroupImplementationTest do + use ExUnit.Case + + alias KafkaEx.ConsumerGroup + import TestHelper + + require Logger + + @moduletag :consumer_group + + # note this topic is created by docker_up.sh + @topic_name "consumer_group_implementation_test" + @partition_count 4 + @consumer_group_name "consumer_group_implementation" + + defmodule TestObserver do + def start_link do + Agent.start_link(fn -> %{} end, name: __MODULE__) + end + + def on_handled_message_set(message_set, topic, partition) do + Agent.update( + __MODULE__, + fn(state) -> + key = {topic, partition} + Logger.debug("FUCK #{inspect message_set}") + Map.update(state, key, [message_set], &(&1 ++ [message_set])) + end + ) + end + + def last_handled_message_set(topic, partition) do + Agent.get( + __MODULE__, + fn(state) -> + key = {topic, partition} + message_sets_handled = Map.get(state, key, []) + Logger.debug("XXX #{inspect message_sets_handled}") + List.last(message_sets_handled) + end + ) + end + + def on_assign_partitions(topic, members, partitions) do + Agent.update( + __MODULE__, + fn(state) -> + key = {:assigns, topic} + value = [members, partitions] + Map.update(state, key, value, &(&1 ++ value)) + end + ) + end + + def get do + Agent.get(__MODULE__, &(&1)) + end + end + + defmodule TestConsumer do + use KafkaEx.GenConsumer + + alias KafkaEx.ConsumerGroupImplementationTest.TestObserver + + def init(topic, partition) do + {:ok, %{topic: topic, partition: partition}} + end + + def handle_message_set(message_set, state) do + TestObserver.on_handled_message_set(message_set, state.topic, state.partition) + {:async_commit, state} + end + + def assign_partitions(members, partitions) do + # TODO this function should get the state as part of its call and be + # allowed to mutate the state + topic_name = KafkaEx.ConsumerGroupImplementationTest.topic_name + TestObserver.on_assign_partitions(topic_name, members, partitions) + super(members, partitions) + end + end + + def produce(message, partition) do + KafkaEx.produce(@topic_name, partition, message) + end + + def topic_name do + @topic_name + end + + setup do + {:ok, _} = TestObserver.start_link + {:ok, _} = ConsumerGroup.start_link(TestConsumer, @consumer_group_name, [@topic_name]) + + :ok + end + + test "basic startup and consume test" do + starting_offset = latest_offset_number(@topic_name, 0) + + produce("OHAI", 0) + + wait_for(fn -> + state = TestObserver.get() + length(Map.get(state, {:assigns, @topic_name}, [])) > 0 + end) + + # the assign_partitions callback should have been called with all 4 + # partitions + assigns = Map.get(TestObserver.get(), {:assigns, @topic_name}, []) + [[_consumer_id], partitions] = assigns + assert @partition_count == length(partitions) + for ix <- 0..(@partition_count - 1) do + assert {@topic_name, ix} in partitions + end + + wait_for(fn -> + message_set = TestObserver.last_handled_message_set(@topic_name, 0) + message_set && Map.get(List.last(message_set), :offset) >= starting_offset + end) + end +end From 574ab4fbc2a9e59181089772ff2db95c55c6d241 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 31 Jul 2017 16:09:31 -0400 Subject: [PATCH 23/47] Fix topic creation --- scripts/docker_up.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/docker_up.sh b/scripts/docker_up.sh index da117f27..953fb0e2 100755 --- a/scripts/docker_up.sh +++ b/scripts/docker_up.sh @@ -48,4 +48,4 @@ done docker-compose up -d # create topics needed for testing -docker-compose run --rm --no-deps kafka1 /bin/bash -c '${KAFKA_HOME}/bin/kafka-topics.sh --create --topic consumer_group_implementation_test --replication-factor 2 --partitions 4 --zookeeper zookeeper:2181' +docker-compose exec kafka3 /bin/bash -c "KAFKA_ZOOKEEPER_CONNECT=zookeeper:2181 KAFKA_PORT=9094 KAFKA_CREATE_TOPICS=consumer_group_implementation_test:4:2 create-topics.sh" From ee89433e4bbb0d4df2fb874563be303cecba3ed2 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 31 Jul 2017 17:33:15 -0400 Subject: [PATCH 24/47] Try to fix test on older elixir --- test/integration/consumer_group_implementation_test.exs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index 9ed9368d..db05c931 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -46,8 +46,8 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do __MODULE__, fn(state) -> key = {:assigns, topic} - value = [members, partitions] - Map.update(state, key, value, &(&1 ++ value)) + value = {members, partitions} + Map.update(state, key, [value], &(&1 ++ [value])) end ) end @@ -108,7 +108,7 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do # the assign_partitions callback should have been called with all 4 # partitions assigns = Map.get(TestObserver.get(), {:assigns, @topic_name}, []) - [[_consumer_id], partitions] = assigns + [{[_consumer_id], partitions}] = assigns assert @partition_count == length(partitions) for ix <- 0..(@partition_count - 1) do assert {@topic_name, ix} in partitions From 874c14cc08086d92bbe898cb1b477eb82931c405 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 31 Jul 2017 18:26:20 -0400 Subject: [PATCH 25/47] Add PartitionAssignment, fix group_by compat issue --- lib/kafka_ex/consumer_group/manager.ex | 6 ++-- .../consumer_group/partition_assignment.ex | 29 +++++++++++++++++++ lib/kafka_ex/gen_consumer.ex | 6 ++-- .../partition_assignment_test.exs | 18 ++++++++++++ 4 files changed, 53 insertions(+), 6 deletions(-) create mode 100644 lib/kafka_ex/consumer_group/partition_assignment.ex create mode 100644 test/kafka_ex/consumer_group/partition_assignment_test.exs diff --git a/lib/kafka_ex/consumer_group/manager.ex b/lib/kafka_ex/consumer_group/manager.ex index 15667875..03cd6b62 100644 --- a/lib/kafka_ex/consumer_group/manager.ex +++ b/lib/kafka_ex/consumer_group/manager.ex @@ -418,7 +418,9 @@ defmodule KafkaEx.ConsumerGroup.Manager do # pack_assignments([{"foo", 0}, {"foo", 1}]) #=> [{"foo", [0, 1]}] defp pack_assignments(assignments) do assignments - |> Enum.group_by(&(elem(&1, 0)), &(elem(&1, 1))) - |> Enum.into([]) + |> Enum.reduce(%{}, fn({topic, partition}, assignments) -> + Map.update(assignments, topic, [partition], &(&1 ++ [partition])) + end) + |> Map.to_list end end diff --git a/lib/kafka_ex/consumer_group/partition_assignment.ex b/lib/kafka_ex/consumer_group/partition_assignment.ex new file mode 100644 index 00000000..fa787822 --- /dev/null +++ b/lib/kafka_ex/consumer_group/partition_assignment.ex @@ -0,0 +1,29 @@ +defmodule KafkaEx.ConsumerGroup.PartitionAssignment do + @moduledoc """ + Contains useful partition assignment algorithms for consumer groups + """ + + alias KafkaEx.GenConsumer + + @doc """ + Round robin assignment + + Iterates over the partitions and members, giving the first member the first + partition, the second member the second partition, etc, looping back to the + beginning of the list of members when finished. + + Example: + iex> KafkaEx.ConsumerGroup.PartitionAssignment(["m1", "m2"], [{"t1", 0}, {"t2, 1"}, {"t3", 2}]) + %{"m1" => [{"t1", 0}, {"t3", 2}], "m2" => [{"t2", 1}]} + """ + @spec round_robin([binary], [GenConsumer.partition]) :: + %{binary => [GenConsumer.partition]} + def round_robin(members, partitions) do + members + |> Stream.cycle + |> Enum.zip(partitions) + |> Enum.reduce(%{}, fn({member, partition}, assignments) -> + Map.update(assignments, member, [partition], &(&1 ++ [partition])) + end) + end +end diff --git a/lib/kafka_ex/gen_consumer.ex b/lib/kafka_ex/gen_consumer.ex index 6d3ef9f1..79dc10a5 100644 --- a/lib/kafka_ex/gen_consumer.ex +++ b/lib/kafka_ex/gen_consumer.ex @@ -281,6 +281,7 @@ defmodule KafkaEx.GenConsumer do defmacro __using__(_opts) do quote do @behaviour KafkaEx.GenConsumer + alias KafkaEx.ConsumerGroup.PartitionAssignment alias KafkaEx.Protocol.Fetch.Message def init(_topic, _partition) do @@ -288,10 +289,7 @@ defmodule KafkaEx.GenConsumer do end def assign_partitions(members, partitions) do - members - |> Stream.cycle - |> Enum.zip(partitions) - |> Enum.group_by(&(elem(&1, 0)), &(elem(&1, 1))) + PartitionAssignment.round_robin(members, partitions) end defoverridable [init: 2, assign_partitions: 2] diff --git a/test/kafka_ex/consumer_group/partition_assignment_test.exs b/test/kafka_ex/consumer_group/partition_assignment_test.exs new file mode 100644 index 00000000..b22dbfa9 --- /dev/null +++ b/test/kafka_ex/consumer_group/partition_assignment_test.exs @@ -0,0 +1,18 @@ +defmodule KafkaEx.ConsumerGroup.PartitionAssignmentTest do + use ExUnit.Case + + alias KafkaEx.ConsumerGroup.PartitionAssignment + + test "round robin partition assignment works" do + topic = "topic" + members = ["member1", "member2"] + partitions = [{topic, 0}, {topic, 1}, {topic, 2}] + + assignments = PartitionAssignment.round_robin(members, partitions) + expected = %{ + "member1" => [{topic, 0}, {topic, 2}], + "member2" => [{topic, 1}] + } + assert expected == assignments + end +end From 3e68eeb29c92bb653cf2d061d38555639d8001e8 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 31 Jul 2017 18:33:49 -0400 Subject: [PATCH 26/47] Fix Map.new compat --- lib/kafka_ex/consumer_group/manager.ex | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/kafka_ex/consumer_group/manager.ex b/lib/kafka_ex/consumer_group/manager.ex index 03cd6b62..ea6dca00 100644 --- a/lib/kafka_ex/consumer_group/manager.ex +++ b/lib/kafka_ex/consumer_group/manager.ex @@ -392,7 +392,7 @@ defmodule KafkaEx.ConsumerGroup.Manager do Enum.map(assignments, fn ({member, topic_partitions}) -> {member, pack_assignments(topic_partitions)} end) - assignments_map = Map.new(packed_assignments) + assignments_map = Enum.into(packed_assignments, %{}) # Fill in empty assignments for missing member IDs. Enum.map(members, fn (member) -> From af0230270d3ed5ffd77a4d99adb1bf070a976ac9 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 31 Jul 2017 18:48:09 -0400 Subject: [PATCH 27/47] Remove R17 from build matrix We need :erlang.monotonic_time for consumer groups. We could work around this, but 17 is now 3 versions behind, so I'd rather we just drop support. --- .travis.yml | 2 -- 1 file changed, 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 5543a4f8..3088bef9 100644 --- a/.travis.yml +++ b/.travis.yml @@ -22,8 +22,6 @@ matrix: otp_release: 19.2 env: COVERALLS=false include: - - elixir: 1.1.1 - otp_release: 17.5 - elixir: 1.4.0 otp_release: 19.2 env: COVERALLS=true From 6e984b67408eb51862677b7e99503d97f2142bb1 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 31 Jul 2017 19:13:05 -0400 Subject: [PATCH 28/47] Don't re-run tests if they pass the first time --- scripts/ci_tests.sh | 23 ++++++++++++++++------- 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/scripts/ci_tests.sh b/scripts/ci_tests.sh index 04ba1528..ca190817 100755 --- a/scripts/ci_tests.sh +++ b/scripts/ci_tests.sh @@ -6,15 +6,24 @@ # # This script could be used for local testing as long as COVERALLS is not set. -set -ev - -# first test run - tends to work the kinks out of the kafka brokers -# (we should strive to remove this but it is necessary for now) -mix test --include integration --include consumer_group --include server_0_p_9_p_0 || true +export MIX_ENV=test if [ "$COVERALLS" = true ] then - MIX_ENV=test mix coveralls.travis --include integration --include consumer_group --include server_0_p_9_p_0 + echo "Coveralls will be reported" + TEST_COMMAND=coveralls +else + TEST_COMMAND=test +fi + +mix "$TEST_COMMAND" --include integration --include consumer_group --include server_0_p_9_p_0 + +# sometimes the first test run fails due to broker issues and we need to run it again +# (we should strive to remove this but it is necessary for now) +if [ $? -eq 0 ] +then + echo "First tests passed, skipping repeat" else - mix test --cover --include integration --include consumer_group --include server_0_p_9_p_0 + echo "Repeating tests" + mix "$TEST_COMMAND" --include integration --include consumer_group --include server_0_p_9_p_0 fi From fa0232e0af09b1916980f2b0b48d8476fe96fcfb Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Tue, 1 Aug 2017 13:56:44 -0400 Subject: [PATCH 29/47] Forgot to save the merged file :( --- .travis.yml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/.travis.yml b/.travis.yml index ff638e1f..3f0247db 100644 --- a/.travis.yml +++ b/.travis.yml @@ -22,17 +22,12 @@ matrix: otp_release: 19.3 env: COVERALLS=false include: -<<<<<<< HEAD - - elixir: 1.4.0 - otp_release: 19.2 -======= - elixir: 1.5.0 otp_release: 19.3 - elixir: 1.5.0 otp_release: 20.0 - elixir: 1.4.5 otp_release: 19.3 ->>>>>>> master env: COVERALLS=true dist: trusty sudo: required From f46f6be16f91d9a10c884c8532e4538da2f098de Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Tue, 1 Aug 2017 15:33:02 -0400 Subject: [PATCH 30/47] Fix default worker name in test helper --- test/test_helper.exs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/test_helper.exs b/test/test_helper.exs index fdf7d29e..179c499d 100644 --- a/test/test_helper.exs +++ b/test/test_helper.exs @@ -54,7 +54,7 @@ defmodule TestHelper do def latest_consumer_offset_number(topic, partition, consumer_group, - worker \\ KafkaEx.Server) do + worker \\ :kafka_ex) do request = %KafkaEx.Protocol.OffsetFetch.Request{topic: topic, partition: partition, consumer_group: consumer_group} From 2dafc01619d1206c89ecea6966532acea50d2cfe Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Tue, 1 Aug 2017 15:33:20 -0400 Subject: [PATCH 31/47] Test that exiting consumer also commits offsets --- .../consumer_group_implementation_test.exs | 39 ++++++++++++++++--- 1 file changed, 34 insertions(+), 5 deletions(-) diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index db05c931..3dc15aa4 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -23,7 +23,6 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do __MODULE__, fn(state) -> key = {topic, partition} - Logger.debug("FUCK #{inspect message_set}") Map.update(state, key, [message_set], &(&1 ++ [message_set])) end ) @@ -35,7 +34,6 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do fn(state) -> key = {topic, partition} message_sets_handled = Map.get(state, key, []) - Logger.debug("XXX #{inspect message_sets_handled}") List.last(message_sets_handled) end ) @@ -88,14 +86,31 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do @topic_name end + def sync_stop(pid) when is_pid(pid) do + wait_for(fn -> + if Process.alive?(pid) do + Process.exit(pid, :normal) + end + !Process.alive?(pid) + end) + end + setup do {:ok, _} = TestObserver.start_link - {:ok, _} = ConsumerGroup.start_link(TestConsumer, @consumer_group_name, [@topic_name]) + {:ok, consumer_group_pid} = ConsumerGroup.start_link( + TestConsumer, + @consumer_group_name, + [@topic_name] + ) + + on_exit fn -> + sync_stop(consumer_group_pid) + end - :ok + {:ok, consumer_group_pid: consumer_group_pid} end - test "basic startup and consume test" do + test "basic startup, consume, and shutdown test", context do starting_offset = latest_offset_number(@topic_name, 0) produce("OHAI", 0) @@ -114,9 +129,23 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do assert {@topic_name, ix} in partitions end + # we actually consume the messages wait_for(fn -> message_set = TestObserver.last_handled_message_set(@topic_name, 0) message_set && Map.get(List.last(message_set), :offset) >= starting_offset end) + + # stop the supervisor + Process.unlink(context[:consumer_group_pid]) + sync_stop(context[:consumer_group_pid]) + + # offsets should be committed on exit + wait_for(fn -> + ending_offset = + latest_consumer_offset_number(@topic_name, 0, @consumer_group_name) + message_set = TestObserver.last_handled_message_set(@topic_name, 0) + last_message = List.last(message_set) + ending_offset == last_message.offset + 1 + end) end end From 52dcba9d1d8c1b79af7941d4440ec42ff922623b Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Tue, 1 Aug 2017 16:35:11 -0400 Subject: [PATCH 32/47] Test across all partitions on the topic --- .../consumer_group_implementation_test.exs | 51 ++++++++++++++----- 1 file changed, 38 insertions(+), 13 deletions(-) diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index 3dc15aa4..6690b78f 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -80,6 +80,18 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do def produce(message, partition) do KafkaEx.produce(@topic_name, partition, message) + message + end + + def right_last_message?(nil, _, _), do: false + def right_last_message?([], _, _), do: false + def right_last_message?(message_set, expected_message, expected_offset) do + Logger.debug(fn -> + "Got message set: #{inspect message_set} " <> + "expecting '#{expected_message}' @ offset #{expected_offset}" + end) + message = List.last(message_set) + message.value == expected_message && message.offset == expected_offset end def topic_name do @@ -111,9 +123,18 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end test "basic startup, consume, and shutdown test", context do - starting_offset = latest_offset_number(@topic_name, 0) + partition_range = 0..(@partition_count - 1) - produce("OHAI", 0) + starting_offsets = partition_range + |> Enum.map(fn(px) -> {px, latest_offset_number(@topic_name, px)} end) + |> Enum.into(%{}) + + messages = partition_range + |> Enum.map(fn(px) -> + offset = Map.get(starting_offsets, px) + {px, produce("M #{px} #{offset}", px)} + end) + |> Enum.into(%{}) wait_for(fn -> state = TestObserver.get() @@ -130,22 +151,26 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end # we actually consume the messages - wait_for(fn -> - message_set = TestObserver.last_handled_message_set(@topic_name, 0) - message_set && Map.get(List.last(message_set), :offset) >= starting_offset - end) + for px <- partition_range do + wait_for(fn -> + message_set = TestObserver.last_handled_message_set(@topic_name, px) + right_last_message?(message_set, messages[px], starting_offsets[px]) + end) + end # stop the supervisor Process.unlink(context[:consumer_group_pid]) sync_stop(context[:consumer_group_pid]) # offsets should be committed on exit - wait_for(fn -> - ending_offset = - latest_consumer_offset_number(@topic_name, 0, @consumer_group_name) - message_set = TestObserver.last_handled_message_set(@topic_name, 0) - last_message = List.last(message_set) - ending_offset == last_message.offset + 1 - end) + for px <- partition_range do + wait_for(fn -> + ending_offset = + latest_consumer_offset_number(@topic_name, px, @consumer_group_name) + message_set = TestObserver.last_handled_message_set(@topic_name, px) + last_message = List.last(message_set) + ending_offset == last_message.offset + 1 + end) + end end end From 74036e25708055bbc87a44c6344bc43d39e1a5bb Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Wed, 2 Aug 2017 11:19:40 -0400 Subject: [PATCH 33/47] Add a second consumer to the test group --- .../consumer_group_implementation_test.exs | 69 ++++++++++++++----- 1 file changed, 53 insertions(+), 16 deletions(-) diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index 6690b78f..1b4b23f7 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -50,6 +50,10 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do ) end + def get_assigns(topic_name) do + Map.get(get(), {:assigns, topic_name}) + end + def get do Agent.get(__MODULE__, &(&1)) end @@ -61,15 +65,25 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do alias KafkaEx.ConsumerGroupImplementationTest.TestObserver def init(topic, partition) do + Logger.debug(fn -> + "Initialized consumer #{inspect self()} for #{topic}:#{partition}" + end) {:ok, %{topic: topic, partition: partition}} end def handle_message_set(message_set, state) do + Logger.debug(fn -> + "Consumer #{inspect self()} handled message set #{inspect message_set}" + end) TestObserver.on_handled_message_set(message_set, state.topic, state.partition) {:async_commit, state} end def assign_partitions(members, partitions) do + Logger.debug(fn -> + "Consumer #{inspect self()} got " <> + "partition assignment: #{inspect members} #{inspect partitions}" + end) # TODO this function should get the state as part of its call and be # allowed to mutate the state topic_name = KafkaEx.ConsumerGroupImplementationTest.topic_name @@ -109,22 +123,52 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do setup do {:ok, _} = TestObserver.start_link - {:ok, consumer_group_pid} = ConsumerGroup.start_link( + {:ok, consumer_group_pid1} = ConsumerGroup.start_link( TestConsumer, @consumer_group_name, - [@topic_name] + [@topic_name], + heartbeat_interval: 100 + ) + {:ok, consumer_group_pid2} = ConsumerGroup.start_link( + TestConsumer, + @consumer_group_name, + [@topic_name], + heartbeat_interval: 100 ) on_exit fn -> - sync_stop(consumer_group_pid) + sync_stop(consumer_group_pid1) + sync_stop(consumer_group_pid2) end - {:ok, consumer_group_pid: consumer_group_pid} + { + :ok, + consumer_group_pid1: consumer_group_pid1, + consumer_group_pid2: consumer_group_pid2 + } end test "basic startup, consume, and shutdown test", context do partition_range = 0..(@partition_count - 1) + # wait for both consumer groups to join + wait_for(fn -> + assigns = TestObserver.get_assigns(@topic_name) || [] + length(assigns) > 0 && length(elem(List.last(assigns), 0)) == 2 + end) + + # the assign_partitions callback should have been called with all 4 + # partitions + assigns = TestObserver.get_assigns(@topic_name) + assert length(assigns) == 2 + last_assigns = List.last(assigns) + # we should have two consumers in the most recent batch + {[_consumer1_id, _consumer2_id], partitions} = last_assigns + assert @partition_count == length(partitions) + for ix <- 0..(@partition_count - 1) do + assert {@topic_name, ix} in partitions + end + starting_offsets = partition_range |> Enum.map(fn(px) -> {px, latest_offset_number(@topic_name, px)} end) |> Enum.into(%{}) @@ -141,15 +185,6 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do length(Map.get(state, {:assigns, @topic_name}, [])) > 0 end) - # the assign_partitions callback should have been called with all 4 - # partitions - assigns = Map.get(TestObserver.get(), {:assigns, @topic_name}, []) - [{[_consumer_id], partitions}] = assigns - assert @partition_count == length(partitions) - for ix <- 0..(@partition_count - 1) do - assert {@topic_name, ix} in partitions - end - # we actually consume the messages for px <- partition_range do wait_for(fn -> @@ -158,9 +193,11 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end) end - # stop the supervisor - Process.unlink(context[:consumer_group_pid]) - sync_stop(context[:consumer_group_pid]) + # stop the supervisors + Process.unlink(context[:consumer_group_pid1]) + sync_stop(context[:consumer_group_pid1]) + Process.unlink(context[:consumer_group_pid2]) + sync_stop(context[:consumer_group_pid2]) # offsets should be committed on exit for px <- partition_range do From 15bf18f1cb544b001a3ba6080a9f50d82e52863e Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Wed, 2 Aug 2017 12:42:21 -0400 Subject: [PATCH 34/47] Refactor TestObserver --- .../consumer_group_implementation_test.exs | 89 ++++++++++++------- 1 file changed, 55 insertions(+), 34 deletions(-) diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index 1b4b23f7..846d5519 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -14,48 +14,74 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do @consumer_group_name "consumer_group_implementation" defmodule TestObserver do + defmodule Event do + defstruct type: nil, from: nil, key: nil, payload: nil + + def type?(%Event{type: type}, type), do: true + def type?(%Event{}, _type), do: false + + def key?(%Event{key: key}, key), do: true + def key?(%Event{}, _key), do: false + end + def start_link do - Agent.start_link(fn -> %{} end, name: __MODULE__) + Agent.start_link(fn -> [] end, name: __MODULE__) end - def on_handled_message_set(message_set, topic, partition) do - Agent.update( - __MODULE__, - fn(state) -> - key = {topic, partition} - Map.update(state, key, [message_set], &(&1 ++ [message_set])) - end - ) + def event(event = %Event{}) do + event = %{event | from: self()} + Agent.update(__MODULE__, fn(events) -> events ++ [event] end) end - def last_handled_message_set(topic, partition) do - Agent.get( - __MODULE__, - fn(state) -> - key = {topic, partition} - message_sets_handled = Map.get(state, key, []) - List.last(message_sets_handled) - end - ) + def all_events() do + Agent.get(__MODULE__, &(&1)) + end + + def by_type(events, type) do + Enum.filter(events, &Event.type?(&1, type)) + end + + def by_key(events, key) do + Enum.filter(events, &Event.key?(&1, key)) + end + + def payloads(events) do + Enum.map(events, &(&1.payload)) end def on_assign_partitions(topic, members, partitions) do - Agent.update( - __MODULE__, - fn(state) -> - key = {:assigns, topic} - value = {members, partitions} - Map.update(state, key, [value], &(&1 ++ [value])) - end + event( + %Event{ + type: :assign_partitions, + key: topic, + payload: {members, partitions} + } ) end - def get_assigns(topic_name) do - Map.get(get(), {:assigns, topic_name}) + def on_handled_message_set(message_set, topic, partition) do + event( + %Event{ + type: :handled_message_set, + key: {topic, partition}, + payload: message_set + } + ) end - def get do - Agent.get(__MODULE__, &(&1)) + def last_handled_message_set(topic, partition) do + all_events() + |> by_type(:handled_message_set) + |> by_key({topic, partition}) + |> payloads() + |> List.last + end + + def get_assigns(topic) do + all_events() + |> by_type(:assign_partitions) + |> by_key(topic) + |> payloads() end end @@ -180,11 +206,6 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end) |> Enum.into(%{}) - wait_for(fn -> - state = TestObserver.get() - length(Map.get(state, {:assigns, @topic_name}, [])) > 0 - end) - # we actually consume the messages for px <- partition_range do wait_for(fn -> From 8a0cd1428d2be5ad9cad1b065469a974bbdff40e Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Wed, 2 Aug 2017 14:20:51 -0400 Subject: [PATCH 35/47] Better assignment tests --- .../consumer_group_implementation_test.exs | 36 ++++++++++++++----- 1 file changed, 28 insertions(+), 8 deletions(-) diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index 846d5519..46899326 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -49,12 +49,16 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do Enum.map(events, &(&1.payload)) end - def on_assign_partitions(topic, members, partitions) do + def on_assign_partitions(topic, members, partitions, assignments) do event( %Event{ type: :assign_partitions, key: topic, - payload: {members, partitions} + payload: %{ + members: members, + partitions: partitions, + assignments: assignments + } } ) end @@ -83,6 +87,11 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do |> by_key(topic) |> payloads() end + + def current_assignments(topic) do + last_assigns = List.last(get_assigns(topic)) + last_assigns.assignments + end end defmodule TestConsumer do @@ -113,8 +122,14 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do # TODO this function should get the state as part of its call and be # allowed to mutate the state topic_name = KafkaEx.ConsumerGroupImplementationTest.topic_name - TestObserver.on_assign_partitions(topic_name, members, partitions) - super(members, partitions) + assignments = super(members, partitions) + TestObserver.on_assign_partitions( + topic_name, + members, + partitions, + assignments + ) + assignments end end @@ -180,7 +195,7 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do # wait for both consumer groups to join wait_for(fn -> assigns = TestObserver.get_assigns(@topic_name) || [] - length(assigns) > 0 && length(elem(List.last(assigns), 0)) == 2 + length(assigns) > 0 && length(Map.get(List.last(assigns), :members)) == 2 end) # the assign_partitions callback should have been called with all 4 @@ -189,11 +204,16 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do assert length(assigns) == 2 last_assigns = List.last(assigns) # we should have two consumers in the most recent batch - {[_consumer1_id, _consumer2_id], partitions} = last_assigns - assert @partition_count == length(partitions) + assert 2 == length(last_assigns.members) + assert @partition_count == length(last_assigns.partitions) for ix <- 0..(@partition_count - 1) do - assert {@topic_name, ix} in partitions + assert {@topic_name, ix} in last_assigns.partitions end + assignments = TestObserver.current_assignments(@topic_name) + # there should be two cgs with assignments + assert 2 == length(Map.keys(assignments)) + # each worker should have two partitions + assert Enum.all?(Map.values(assignments), fn(p) -> length(p) == 2 end) starting_offsets = partition_range |> Enum.map(fn(px) -> {px, latest_offset_number(@topic_name, px)} end) From 8818e6ae0effda2c298f909fef8b9a60642931c1 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Wed, 2 Aug 2017 15:14:05 -0400 Subject: [PATCH 36/47] Fix test command for travis --- scripts/ci_tests.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/ci_tests.sh b/scripts/ci_tests.sh index ca190817..a853c67b 100755 --- a/scripts/ci_tests.sh +++ b/scripts/ci_tests.sh @@ -11,7 +11,7 @@ export MIX_ENV=test if [ "$COVERALLS" = true ] then echo "Coveralls will be reported" - TEST_COMMAND=coveralls + TEST_COMMAND=coveralls.travis else TEST_COMMAND=test fi From 1f29f41698058dc9c4e62dfee32283e0a01f0ca5 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Wed, 2 Aug 2017 15:24:08 -0400 Subject: [PATCH 37/47] Assert all 4 handlers --- .../consumer_group_implementation_test.exs | 23 +++++++++++++++++-- 1 file changed, 21 insertions(+), 2 deletions(-) diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index 46899326..0750bbc4 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -15,7 +15,7 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do defmodule TestObserver do defmodule Event do - defstruct type: nil, from: nil, key: nil, payload: nil + defstruct type: nil, source: nil, key: nil, payload: nil def type?(%Event{type: type}, type), do: true def type?(%Event{}, _type), do: false @@ -29,7 +29,7 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end def event(event = %Event{}) do - event = %{event | from: self()} + event = %{event | source: self()} Agent.update(__MODULE__, fn(events) -> events ++ [event] end) end @@ -49,6 +49,10 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do Enum.map(events, &(&1.payload)) end + def sources(events) do + Enum.map(events, &(&1.source)) + end + def on_assign_partitions(topic, members, partitions, assignments) do event( %Event{ @@ -88,6 +92,14 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do |> payloads() end + def last_handler(topic, partition) do + all_events() + |> by_type(:handled_message_set) + |> by_key({topic, partition}) + |> sources() + |> List.last + end + def current_assignments(topic) do last_assigns = List.last(get_assigns(topic)) last_assigns.assignments @@ -234,6 +246,13 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end) end + # each partition should be getting handled by a different consumer + handlers = Enum.map( + partition_range, + &(TestObserver.last_handler(@topic_name, &1)) + ) + assert handlers == Enum.uniq(handlers) + # stop the supervisors Process.unlink(context[:consumer_group_pid1]) sync_stop(context[:consumer_group_pid1]) From b3e73c0991c8b183decea4f4326d6006717a82e8 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Wed, 2 Aug 2017 15:36:55 -0400 Subject: [PATCH 38/47] Test for rebalancing --- .../consumer_group_implementation_test.exs | 57 +++++++++++++++++-- 1 file changed, 51 insertions(+), 6 deletions(-) diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index 0750bbc4..958a2734 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -189,6 +189,12 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do heartbeat_interval: 100 ) + # wait for both consumer groups to join + wait_for(fn -> + assigns = TestObserver.get_assigns(@topic_name) || [] + length(assigns) > 0 && length(Map.get(List.last(assigns), :members)) == 2 + end) + on_exit fn -> sync_stop(consumer_group_pid1) sync_stop(consumer_group_pid2) @@ -204,12 +210,6 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do test "basic startup, consume, and shutdown test", context do partition_range = 0..(@partition_count - 1) - # wait for both consumer groups to join - wait_for(fn -> - assigns = TestObserver.get_assigns(@topic_name) || [] - length(assigns) > 0 && length(Map.get(List.last(assigns), :members)) == 2 - end) - # the assign_partitions callback should have been called with all 4 # partitions assigns = TestObserver.get_assigns(@topic_name) @@ -270,4 +270,49 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end) end end + + test "starting/stopping consumers rebalances assignments", context do + last_assigns = List.last(TestObserver.get_assigns(@topic_name)) + assert 2 == length(last_assigns.members) + + Process.unlink(context[:consumer_group_pid1]) + sync_stop(context[:consumer_group_pid1]) + + wait_for(fn -> + last_assigns = List.last(TestObserver.get_assigns(@topic_name)) + 1 == length(last_assigns.members) + end) + + last_assigns = List.last(TestObserver.get_assigns(@topic_name)) + assert 1 == length(last_assigns.members) + + {:ok, consumer_group_pid3} = ConsumerGroup.start_link( + TestConsumer, + @consumer_group_name, + [@topic_name], + heartbeat_interval: 100 + ) + + wait_for(fn -> + last_assigns = List.last(TestObserver.get_assigns(@topic_name)) + 2 == length(last_assigns.members) + end) + + last_assigns = List.last(TestObserver.get_assigns(@topic_name)) + assert 2 == length(last_assigns.members) + + Process.unlink(context[:consumer_group_pid2]) + sync_stop(context[:consumer_group_pid2]) + + wait_for(fn -> + last_assigns = List.last(TestObserver.get_assigns(@topic_name)) + 1 == length(last_assigns.members) + end) + + last_assigns = List.last(TestObserver.get_assigns(@topic_name)) + assert 1 == length(last_assigns.members) + + Process.unlink(consumer_group_pid3) + sync_stop(consumer_group_pid3) + end end From 86bdc0b48b2cb94a5c994614c278146c678bdcd7 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Thu, 3 Aug 2017 14:49:01 -0400 Subject: [PATCH 39/47] Move partition assignment out of GenConsumer GenConsumer doesn't necessarily need to know anything about the way partitions are assigned - it makes more sense for the ConsumerGroup code to deal with this. I made it an optional callback that defaults to the round robin algorithm. I tried to update the docs where applicable. --- lib/kafka_ex/consumer_group.ex | 22 ++++-- lib/kafka_ex/consumer_group/manager.ex | 25 ++++-- .../consumer_group/partition_assignment.ex | 69 +++++++++++++++- lib/kafka_ex/gen_consumer.ex | 79 ++----------------- lib/kafka_ex/gen_consumer/supervisor.ex | 8 +- .../consumer_group_implementation_test.exs | 49 +++++++----- 6 files changed, 139 insertions(+), 113 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 23d75563..3e383767 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -10,9 +10,11 @@ defmodule KafkaEx.ConsumerGroup do Any time group membership changes (a member joins or leaves the group), a Kafka broker initiates group synchronization by asking one of the group members (the leader elected by the broker) to provide partition assignments - for the whole group. Partition assignment is handled by the - `c:KafkaEx.GenConsumer.assign_partitions/2` callback of the provided consumer - module. + for the whole group. KafkaEx uses a round robin partition assignment + algorithm by deafult. This can be overridden by passing a callback function + in the `:partition_assignment_callback` option. See + `KafkaEx.ConsumerGroup.PartitionAssignment` for details on partition + assignment functions. A `KafkaEx.ConsumerGroup` process is responsible for: @@ -73,17 +75,23 @@ defmodule KafkaEx.ConsumerGroup do use Supervisor + alias KafkaEx.ConsumerGroup.PartitionAssignment + @typedoc """ Option values used when starting a consumer group - * Any of `KafkaEx.GenConsumer.option`, which will be passed on to consumers + * Any of `t:KafkaEx.GenConsumer.option/0`, + which will be passed on to consumers * `:name` - Name for the consumer group supervisor * `:max_restarts`, `:max_seconds` - Supervisor restart policy parameters + * `:partition_assignment_callback` - See + `t:KafkaEx.ConsumerGroup.PartitionAssignment.callback/0` """ @type option :: KafkaEx.GenConsumer.option - | {:name, Elixir.Supervisor.name} - | {:max_restarts, non_neg_integer} - | {:max_seconds, non_neg_integer} + | {:partition_assignment_callback, PartitionAssignment.callback} + | {:name, Elixir.Supervisor.name} + | {:max_restarts, non_neg_integer} + | {:max_seconds, non_neg_integer} @type options :: [option] diff --git a/lib/kafka_ex/consumer_group/manager.ex b/lib/kafka_ex/consumer_group/manager.ex index ea6dca00..d2468f78 100644 --- a/lib/kafka_ex/consumer_group/manager.ex +++ b/lib/kafka_ex/consumer_group/manager.ex @@ -5,6 +5,8 @@ defmodule KafkaEx.ConsumerGroup.Manager do use GenServer + alias KafkaEx.ConsumerGroup + alias KafkaEx.ConsumerGroup.PartitionAssignment alias KafkaEx.Protocol.JoinGroup.Request, as: JoinGroupRequest alias KafkaEx.Protocol.JoinGroup.Response, as: JoinGroupResponse alias KafkaEx.Protocol.Heartbeat.Request, as: HeartbeatRequest @@ -26,6 +28,7 @@ defmodule KafkaEx.ConsumerGroup.Manager do :session_timeout, :consumer_module, :consumer_opts, + :partition_assignment_callback, :group_name, :topics, :member_id, @@ -69,10 +72,21 @@ defmodule KafkaEx.ConsumerGroup.Manager do :session_timeout, Application.get_env(:kafka_ex, :session_timeout, @session_timeout) ) + partition_assignment_callback = Keyword.get( + opts, + :partition_assignment_callback, + &PartitionAssignment.round_robin/2 + ) supervisor_pid = Keyword.fetch!(opts, :supervisor_pid) consumer_opts = Keyword.drop( - opts, [:supervisor_pid, :heartbeat_interval, :session_timeout] + opts, + [ + :supervisor_pid, + :heartbeat_interval, + :session_timeout, + :partition_assignment_callback + ] ) {:ok, worker_name} = @@ -84,6 +98,7 @@ defmodule KafkaEx.ConsumerGroup.Manager do heartbeat_interval: heartbeat_interval, session_timeout: session_timeout, consumer_module: consumer_module, + partition_assignment_callback: partition_assignment_callback, consumer_opts: consumer_opts, group_name: group_name, topics: topics, @@ -335,7 +350,7 @@ defmodule KafkaEx.ConsumerGroup.Manager do } = state, assignments ) do - :ok = KafkaEx.ConsumerGroup.start_consumer( + :ok = ConsumerGroup.start_consumer( pid, consumer_module, group_name, @@ -348,7 +363,7 @@ defmodule KafkaEx.ConsumerGroup.Manager do # Stops consuming from the member's assigned partitions and commits offsets. defp stop_consumer(%State{supervisor_pid: pid} = state) do - :ok = KafkaEx.ConsumerGroup.stop_consumer(pid) + :ok = ConsumerGroup.stop_consumer(pid) state end @@ -380,12 +395,12 @@ defmodule KafkaEx.ConsumerGroup.Manager do # return value is a complete list of member assignments in the format needed # by `SyncGroupResponse`. defp assign_partitions( - %State{consumer_module: consumer_module}, + %State{partition_assignment_callback: partition_assignment_callback}, members, partitions ) do # Delegate partition assignment to GenConsumer module. - assignments = consumer_module.assign_partitions(members, partitions) + assignments = partition_assignment_callback.(members, partitions) # Convert assignments to format expected by Kafka protocol. packed_assignments = diff --git a/lib/kafka_ex/consumer_group/partition_assignment.ex b/lib/kafka_ex/consumer_group/partition_assignment.ex index fa787822..d2c53ecc 100644 --- a/lib/kafka_ex/consumer_group/partition_assignment.ex +++ b/lib/kafka_ex/consumer_group/partition_assignment.ex @@ -1,9 +1,71 @@ defmodule KafkaEx.ConsumerGroup.PartitionAssignment do @moduledoc """ - Contains useful partition assignment algorithms for consumer groups + Contains typespecs and reference algorithms for assigning partitions + + `round_robin/2` is used by `KafkaEx.ConsumerGroup` by default and should + suffice in most cases. + + For custom assignments, any function matching the + `t:callback/0` type spec can be used. + """ + + @typedoc """ + The ID (string) of a member of a consumer group, assigned by a Kafka broker. + """ + @type member_id :: binary + + @typedoc """ + The string name of a Kafka topic. + """ + @type topic :: binary + + @typedoc """ + The integer ID of a partition of a Kafka topic. + """ + @type partition_id :: integer + + @typedoc """ + A partition of a single topic (embeds the name of the topic). """ + @type partition :: {topic, partition_id} - alias KafkaEx.GenConsumer + @typedoc """ + A function that can assign partitions. + + `members` is a list of member IDs and `partitions` is a list of partitions + that need to be assigned to a group member. + + The return value must be a map with member IDs as keys and a list of + partition assignments as values. For each member ID in the returned map, the + assigned partitions will become the `assignments` argument to + `KafkaEx.GenConsumer.Supervisor.start_link/4` in the corresponding member + process. Any member that's omitted from the return value will not be assigned + any partitions. + + ### Example + + Given the following `members` and `partitions` to be assigned: + + ``` + members = ["member1", "member2", "member3"] + partitions = [{"topic", 0}, {"topic", 1}, {"topic", 2}] + ``` + + One possible assignment is as follows: + + ``` + ExampleGenConsumer.assign_partitions(members, partitions) + #=> %{"member1" => [{"topic", 0}, {"topic", 2}], "member2" => [{"topic", 1}]} + ``` + + In this case, the consumer group process for `"member1"` will launch two + `KafkaEx.GenConsumer` processes (one for each of its assigned partitions), + `"member2"` will launch one `KafkaEx.GenConsumer` process, and `"member3"` will + launch no processes. + """ + @type callback :: + ((members :: [member_id], partitions :: [partition]) -> + %{member_id => [partition]}) @doc """ Round robin assignment @@ -16,8 +78,7 @@ defmodule KafkaEx.ConsumerGroup.PartitionAssignment do iex> KafkaEx.ConsumerGroup.PartitionAssignment(["m1", "m2"], [{"t1", 0}, {"t2, 1"}, {"t3", 2}]) %{"m1" => [{"t1", 0}, {"t3", 2}], "m2" => [{"t2", 1}]} """ - @spec round_robin([binary], [GenConsumer.partition]) :: - %{binary => [GenConsumer.partition]} + @spec round_robin([binary], [partition]) :: %{binary => [partition]} def round_robin(members, partitions) do members |> Stream.cycle diff --git a/lib/kafka_ex/gen_consumer.ex b/lib/kafka_ex/gen_consumer.ex index 79dc10a5..3258cc44 100644 --- a/lib/kafka_ex/gen_consumer.ex +++ b/lib/kafka_ex/gen_consumer.ex @@ -167,26 +167,6 @@ defmodule KafkaEx.GenConsumer do require Logger - @typedoc """ - The ID (string) of a member of a consumer group, assigned by a Kafka broker. - """ - @type member_id :: binary - - @typedoc """ - The string name of a Kafka topic. - """ - @type topic :: binary - - @typedoc """ - The integer ID of a partition of a Kafka topic. - """ - @type partition_id :: integer - - @typedoc """ - A partition of a single topic (embeds the name of the topic). - """ - @type partition :: {topic, partition_id} - @typedoc """ Option values used when starting a `KafkaEx.GenConsumer`. """ @@ -212,7 +192,7 @@ defmodule KafkaEx.GenConsumer do Any other return value will cause the `start_link/5` to return `{:error, error}` and the process to exit. """ - @callback init(topic :: topic, partition :: partition_id) :: + @callback init(topic :: binary, partition :: non_neg_integer) :: {:ok, state :: term} @doc """ @@ -236,63 +216,16 @@ defmodule KafkaEx.GenConsumer do @callback handle_message_set(message_set :: [Message.t], state :: term) :: {:async_commit, new_state :: term} | {:sync_commit, new_state :: term} - @doc """ - Invoked to determine partition assignments for a coordinated consumer group. - - `members` is a list of member IDs and `partitions` is a list of partitions - that need to be assigned to a group member. - - The return value must be a map with member IDs as keys and a list of - partition assignments as values. For each member ID in the returned map, the - assigned partitions will become the `assignments` argument to - `KafkaEx.GenConsumer.Supervisor.start_link/4` in the corresponding member - process. Any member that's omitted from the return value will not be assigned - any partitions. - - If this callback is not implemented, the default implementation by `use - KafkaEx.GenConsumer` implements a simple round-robin assignment. - - ### Example - - Given the following `members` and `partitions` to be assigned: - - ``` - members = ["member1", "member2", "member3"] - partitions = [{"topic", 0}, {"topic", 1}, {"topic", 2}] - ``` - - One possible assignment is as follows: - - ``` - ExampleGenConsumer.assign_partitions(members, partitions) - #=> %{"member1" => [{"topic", 0}, {"topic", 2}], "member2" => [{"topic", 1}]} - ``` - - In this case, the consumer group process for `"member1"` will launch two - `KafkaEx.GenConsumer` processes (one for each of its assigned partitions), - `"member2"` will launch one `KafkaEx.GenConsumer` process, and `"member3"` will - launch no processes. - """ - @callback assign_partitions( - members :: [member_id], - partitions :: [partition] - ) :: %{member_id => [partition]} - defmacro __using__(_opts) do quote do @behaviour KafkaEx.GenConsumer - alias KafkaEx.ConsumerGroup.PartitionAssignment alias KafkaEx.Protocol.Fetch.Message def init(_topic, _partition) do {:ok, nil} end - def assign_partitions(members, partitions) do - PartitionAssignment.round_robin(members, partitions) - end - - defoverridable [init: 2, assign_partitions: 2] + defoverridable [init: 2] end end @@ -348,10 +281,10 @@ defmodule KafkaEx.GenConsumer do This function has the same return values as `GenServer.start_link/3`. """ @spec start_link( - module, - binary, - topic, - partition_id, + callback_module :: module, + consumer_group_name :: binary, + topic_name :: binary, + partition_id :: non_neg_integer, options ) :: GenServer.on_start def start_link(consumer_module, group_name, topic, partition, opts \\ []) do diff --git a/lib/kafka_ex/gen_consumer/supervisor.ex b/lib/kafka_ex/gen_consumer/supervisor.ex index 250a9c5a..78a920ed 100644 --- a/lib/kafka_ex/gen_consumer/supervisor.ex +++ b/lib/kafka_ex/gen_consumer/supervisor.ex @@ -31,9 +31,11 @@ defmodule KafkaEx.GenConsumer.Supervisor do returns `{:ok, pid}`, where `pid` is the PID of the supervisor. """ @spec start_link( - module, - binary, - [KafkaEx.GenConsumer.partition], + callback_module :: module, + consumer_group_name :: binary, + assigned_partitions :: [ + {topic_name :: binary, partition_id :: non_neg_integer} + ], KafkaEx.GenConsumer.options ) :: Elixir.Supervisor.on_start def start_link(consumer_module, group_name, assignments, opts \\ []) do diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index 958a2734..7b88f698 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -106,6 +106,28 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end end + defmodule TestPartitioner do + alias KafkaEx.ConsumerGroup.PartitionAssignment + + def assign_partitions(members, partitions) do + Logger.debug(fn -> + "Consumer #{inspect self()} got " <> + "partition assignment: #{inspect members} #{inspect partitions}" + end) + # TODO this function should get the state as part of its call and be + # allowed to mutate the state + topic_name = KafkaEx.ConsumerGroupImplementationTest.topic_name + assignments = PartitionAssignment.round_robin(members, partitions) + TestObserver.on_assign_partitions( + topic_name, + members, + partitions, + assignments + ) + assignments + end + end + defmodule TestConsumer do use KafkaEx.GenConsumer @@ -125,24 +147,6 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do TestObserver.on_handled_message_set(message_set, state.topic, state.partition) {:async_commit, state} end - - def assign_partitions(members, partitions) do - Logger.debug(fn -> - "Consumer #{inspect self()} got " <> - "partition assignment: #{inspect members} #{inspect partitions}" - end) - # TODO this function should get the state as part of its call and be - # allowed to mutate the state - topic_name = KafkaEx.ConsumerGroupImplementationTest.topic_name - assignments = super(members, partitions) - TestObserver.on_assign_partitions( - topic_name, - members, - partitions, - assignments - ) - assignments - end end def produce(message, partition) do @@ -180,13 +184,15 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do TestConsumer, @consumer_group_name, [@topic_name], - heartbeat_interval: 100 + heartbeat_interval: 100, + partition_assignment_callback: &TestPartitioner.assign_partitions/2 ) {:ok, consumer_group_pid2} = ConsumerGroup.start_link( TestConsumer, @consumer_group_name, [@topic_name], - heartbeat_interval: 100 + heartbeat_interval: 100, + partition_assignment_callback: &TestPartitioner.assign_partitions/2 ) # wait for both consumer groups to join @@ -290,7 +296,8 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do TestConsumer, @consumer_group_name, [@topic_name], - heartbeat_interval: 100 + heartbeat_interval: 100, + partition_assignment_callback: &TestPartitioner.assign_partitions/2 ) wait_for(fn -> From d15e82b8136a6b68c02ad793eed51686afc15c47 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Thu, 3 Aug 2017 17:50:54 -0400 Subject: [PATCH 40/47] Factoring/documentation for manager opts * Group the gen_server_opts * Specify heartbeat and session timeout opts --- lib/kafka_ex/consumer_group.ex | 17 +++++++++++++++++ lib/kafka_ex/consumer_group/manager.ex | 6 +++--- 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 3e383767..e45c4e0e 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -80,15 +80,32 @@ defmodule KafkaEx.ConsumerGroup do @typedoc """ Option values used when starting a consumer group + * `:heartbeat_interval` - How frequently, in milliseconds, to send heartbeats + to the broker. This impacts how quickly we will process partition + changes as consumers start/stop. Default: 5000 (5 seconds). + * `:session_timeout` - Consumer group session timeout in milliseconds. + Default: 30000 (30 seconds). See below. * Any of `t:KafkaEx.GenConsumer.option/0`, which will be passed on to consumers + * `:gen_server_opts` - `t:GenServer.options/0` passed on to the manager + GenServer * `:name` - Name for the consumer group supervisor * `:max_restarts`, `:max_seconds` - Supervisor restart policy parameters * `:partition_assignment_callback` - See `t:KafkaEx.ConsumerGroup.PartitionAssignment.callback/0` + + Note `:session_timeout` is registered with the broker and determines how long + before the broker will de-register a consumer from which it has not heard a + heartbeat. This value must between the broker cluster's configured values + for `group.min.session.timeout.ms` and `group.max.session.timeout.ms` (6000 + and 30000 by default). See + [https://kafka.apache.org/documentation/#configuration](https://kafka.apache.org/documentation/#configuration). """ @type option :: KafkaEx.GenConsumer.option + | {:heartbeat_interval, pos_integer} + | {:session_timeout, pos_integer} | {:partition_assignment_callback, PartitionAssignment.callback} + | {:gen_server_opts, GenServer.options} | {:name, Elixir.Supervisor.name} | {:max_restarts, non_neg_integer} | {:max_seconds, non_neg_integer} diff --git a/lib/kafka_ex/consumer_group/manager.ex b/lib/kafka_ex/consumer_group/manager.ex index d2468f78..feb06e01 100644 --- a/lib/kafka_ex/consumer_group/manager.ex +++ b/lib/kafka_ex/consumer_group/manager.ex @@ -49,13 +49,13 @@ defmodule KafkaEx.ConsumerGroup.Manager do @spec start_link(module, binary, [binary], KafkaEx.GenConsumer.options) :: GenServer.on_start def start_link(consumer_module, group_name, topics, opts \\ []) do - {server_opts, consumer_opts} = - Keyword.split(opts, [:debug, :name, :timeout, :spawn_opt]) + gen_server_opts = Keyword.get(opts, :gen_server_opts, []) + consumer_opts = Keyword.drop(opts, [:gen_server_opts]) GenServer.start_link( __MODULE__, {consumer_module, group_name, topics, consumer_opts}, - server_opts + gen_server_opts ) end From dde8f7dff2233f56cdcba862456f5c7a00f85cf7 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Thu, 3 Aug 2017 19:09:59 -0400 Subject: [PATCH 41/47] Clean up test, Supervisor usage --- lib/kafka_ex/consumer_group.ex | 14 ++++++-------- .../consumer_group_implementation_test.exs | 7 ++++--- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index e45c4e0e..fdcc9eff 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -106,7 +106,7 @@ defmodule KafkaEx.ConsumerGroup do | {:session_timeout, pos_integer} | {:partition_assignment_callback, PartitionAssignment.callback} | {:gen_server_opts, GenServer.options} - | {:name, Elixir.Supervisor.name} + | {:name, Supervisor.name} | {:max_restarts, non_neg_integer} | {:max_seconds, non_neg_integer} @@ -128,13 +128,12 @@ defmodule KafkaEx.ConsumerGroup do This function has the same return values as `Supervisor.start_link/3`. """ - @spec start_link(module, binary, [binary], options) :: - Elixir.Supervisor.on_start + @spec start_link(module, binary, [binary], options) :: Supervisor.on_start def start_link(consumer_module, group_name, topics, opts \\ []) do {supervisor_opts, module_opts} = Keyword.split(opts, [:name, :strategy, :max_restarts, :max_seconds]) - Elixir.Supervisor.start_link( + Supervisor.start_link( __MODULE__, {consumer_module, group_name, topics, module_opts}, supervisor_opts @@ -143,14 +142,13 @@ defmodule KafkaEx.ConsumerGroup do @doc false # used by ConsumerGroup to set partition assignments def start_consumer(pid, consumer_module, group_name, assignments, opts) do - child = supervisor( KafkaEx.GenConsumer.Supervisor, [consumer_module, group_name, assignments, opts], id: :consumer ) - case Elixir.Supervisor.start_child(pid, child) do + case Supervisor.start_child(pid, child) do {:ok, _child} -> :ok {:ok, _child, _info} -> :ok end @@ -158,9 +156,9 @@ defmodule KafkaEx.ConsumerGroup do @doc false # used by ConsumerGroup to pause consumption during rebalance def stop_consumer(pid) do - case Elixir.Supervisor.terminate_child(pid, :consumer) do + case Supervisor.terminate_child(pid, :consumer) do :ok -> - Elixir.Supervisor.delete_child(pid, :consumer) + Supervisor.delete_child(pid, :consumer) {:error, :not_found} -> :ok diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index 7b88f698..4f658147 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -114,9 +114,10 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do "Consumer #{inspect self()} got " <> "partition assignment: #{inspect members} #{inspect partitions}" end) - # TODO this function should get the state as part of its call and be - # allowed to mutate the state - topic_name = KafkaEx.ConsumerGroupImplementationTest.topic_name + + # this assumes we are only consuming from one topic + [{topic_name, _} | _] = partitions + assignments = PartitionAssignment.round_robin(members, partitions) TestObserver.on_assign_partitions( topic_name, From 48cf82f60fa3f4db365d5f708e158f1148e44ef8 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Fri, 4 Aug 2017 16:31:06 -0400 Subject: [PATCH 42/47] Add note about multiple topics I tried to add a test for this but had significant trouble getting the test to work. I BELIEVE the issues with the test are because of the number of connects/disconnects happening in rapid succession, but I'm not sure and I'd like to revisit that later. --- lib/kafka_ex/consumer_group.ex | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index fdcc9eff..0cc2d31c 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -124,6 +124,11 @@ defmodule KafkaEx.ConsumerGroup do options for the supervisor as well as for the `KafkEx.GenConsumer` processes that will be spawned by the supervisor. See `t:option/0` for details. + *Note* When starting a consumer group with multiple topics, you should + propagate this configuration change to your consumers. If you add a topic to + an existing consumer group from a single consumer, it may take a long time + to propagate depending on the leader election process. + ### Return Values This function has the same return values as `Supervisor.start_link/3`. From e122f4cbadae1f7c3387f5a80e2b6189b758edf8 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 7 Aug 2017 10:29:09 -0400 Subject: [PATCH 43/47] Add several ops helper functions --- lib/kafka_ex/consumer_group.ex | 90 ++++++++++++++++++- lib/kafka_ex/consumer_group/manager.ex | 47 ++++++++-- lib/kafka_ex/gen_consumer.ex | 13 +++ lib/kafka_ex/gen_consumer/supervisor.ex | 12 +++ .../consumer_group_implementation_test.exs | 48 ++++++++++ 5 files changed, 201 insertions(+), 9 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 0cc2d31c..f8e5ab6b 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -145,7 +145,74 @@ defmodule KafkaEx.ConsumerGroup do ) end - @doc false # used by ConsumerGroup to set partition assignments + @doc """ + Returns the generation id of the consumer group. + + The generation id is provided by the broker on sync. Returns `nil` if + queried before the initial sync has completed. + """ + @spec generation_id(Supervisor.supervisor) :: integer | nil + def generation_id(supervisor_pid) do + call_manager(supervisor_pid, :generation_id) + end + + @doc """ + Returns the consumer group member id + + The id is assigned by the broker. Returns `nil` if queried before the + initial sync has completed. + """ + @spec member_id(Supervisor.supervisor) :: binary | nil + def member_id(supervisor_pid) do + call_manager(supervisor_pid, :member_id) + end + + @doc """ + Returns the member id of the consumer group's leader + + This is provided by the broker on sync. Returns `nil` if queried before the + initial sync has completed + """ + @spec leader_id(Supervisor.supervisor) :: binary | nil + def leader_id(supervisor_pid) do + call_manager(supervisor_pid, :leader_id) + end + + @doc """ + Returns true if this consumer is the leader of the consumer group + + Leaders are elected by the broker and are responsible for assigning + partitions. Returns false if queried before the intiial sync has completed. + """ + @spec leader?(Supervisor.supervisor) :: boolean + def leader?(supervisor_pid) do + call_manager(supervisor_pid, :am_leader) + end + + @doc """ + Returns a list of topic and partition assignments for which this consumer is + responsible. + + These are assigned by the leader and communicated by the broker on sync. + """ + @spec assignments(Supervisor.supervisor) :: + [{topic :: binary, partition_id :: non_neg_integer}] + def assignments(supervisor_pid) do + call_manager(supervisor_pid, :assignments) + end + + @doc """ + Returns the pid of the `KafkaEx.GenConsumer.Supervisor` that supervises this + member's consumers. + + Returns `nil` if called before the initial sync. + """ + @spec consumer_supervisor_pid(Supervisor.supervisor) :: nil | pid + def consumer_supervisor_pid(supervisor_pid) do + call_manager(supervisor_pid, :consumer_supervisor_pid) + end + + @doc false # used by ConsumerGroup.Manager to set partition assignments def start_consumer(pid, consumer_module, group_name, assignments, opts) do child = supervisor( KafkaEx.GenConsumer.Supervisor, @@ -154,8 +221,8 @@ defmodule KafkaEx.ConsumerGroup do ) case Supervisor.start_child(pid, child) do - {:ok, _child} -> :ok - {:ok, _child, _info} -> :ok + {:ok, consumer_pid} -> {:ok, consumer_pid} + {:ok, consumer_pid, _info} -> {:ok, consumer_pid} end end @@ -183,4 +250,21 @@ defmodule KafkaEx.ConsumerGroup do supervise(children, strategy: :one_for_all) end + + defp call_manager(supervisor_pid, call) do + supervisor_pid + |> get_manager_pid + |> GenServer.call(call) + end + + defp get_manager_pid(supervisor_pid) do + {_, pid, _, _} = Enum.find( + Supervisor.which_children(supervisor_pid), + fn + ({KafkaEx.ConsumerGroup.Manager, _, _, _}) -> true + ({_, _, _, _}) -> false + end + ) + pid + end end diff --git a/lib/kafka_ex/consumer_group/manager.ex b/lib/kafka_ex/consumer_group/manager.ex index feb06e01..3a899f4d 100644 --- a/lib/kafka_ex/consumer_group/manager.ex +++ b/lib/kafka_ex/consumer_group/manager.ex @@ -32,6 +32,9 @@ defmodule KafkaEx.ConsumerGroup.Manager do :group_name, :topics, :member_id, + :leader_id, + :consumer_supervisor_pid, + :members, :generation_id, :assignments, :heartbeat_timer, @@ -102,7 +105,7 @@ defmodule KafkaEx.ConsumerGroup.Manager do consumer_opts: consumer_opts, group_name: group_name, topics: topics, - member_id: "", + member_id: nil, } Process.flag(:trap_exit, true) @@ -110,11 +113,38 @@ defmodule KafkaEx.ConsumerGroup.Manager do {:ok, state, 0} end + ###################################################################### + # handle_call clauses - mostly for ops queries + def handle_call(:generation_id, _from, state) do + {:reply, state.generation_id, state} + end + + def handle_call(:member_id, _from, state) do + {:reply, state.member_id, state} + end + + def handle_call(:leader_id, _from, state) do + {:reply, state.leader_id, state} + end + + def handle_call(:am_leader, _from, state) do + {:reply, state.leader_id && state.member_id == state.leader_id, state} + end + + def handle_call(:assignments, _from, state) do + {:reply, state.assignments, state} + end + + def handle_call(:consumer_supervisor_pid, _from, state) do + {:reply, state.consumer_supervisor_pid, state} + end + ###################################################################### + # If `member_id` and `generation_id` aren't set, we haven't yet joined the # group. `member_id` and `generation_id` are initialized by # `JoinGroupResponse`. def handle_info( - :timeout, %State{generation_id: nil, member_id: ""} = state + :timeout, %State{generation_id: nil, member_id: nil} = state ) do {:ok, new_state} = join(state) @@ -132,7 +162,7 @@ defmodule KafkaEx.ConsumerGroup.Manager do # When terminating, inform the group coordinator that this member is leaving # the group so that the group can rebalance without waiting for a session # timeout. - def terminate(_reason, %State{generation_id: nil, member_id: ""}), do: :ok + def terminate(_reason, %State{generation_id: nil, member_id: nil}), do: :ok def terminate(_reason, %State{} = state) do :ok = leave(state) end @@ -165,7 +195,7 @@ defmodule KafkaEx.ConsumerGroup.Manager do ) do join_request = %JoinGroupRequest{ group_name: group_name, - member_id: member_id, + member_id: member_id || "", topics: topics, session_timeout: session_timeout, } @@ -181,6 +211,7 @@ defmodule KafkaEx.ConsumerGroup.Manager do new_state = %State{ state | + leader_id: join_response.leader_id, member_id: join_response.member_id, generation_id: join_response.generation_id } @@ -350,7 +381,7 @@ defmodule KafkaEx.ConsumerGroup.Manager do } = state, assignments ) do - :ok = ConsumerGroup.start_consumer( + {:ok, consumer_supervisor_pid} = ConsumerGroup.start_consumer( pid, consumer_module, group_name, @@ -358,7 +389,11 @@ defmodule KafkaEx.ConsumerGroup.Manager do consumer_opts ) - state + %{ + state | + assignments: assignments, + consumer_supervisor_pid: consumer_supervisor_pid + } end # Stops consuming from the member's assigned partitions and commits offsets. diff --git a/lib/kafka_ex/gen_consumer.ex b/lib/kafka_ex/gen_consumer.ex index 3258cc44..dbadeb90 100644 --- a/lib/kafka_ex/gen_consumer.ex +++ b/lib/kafka_ex/gen_consumer.ex @@ -298,6 +298,15 @@ defmodule KafkaEx.GenConsumer do ) end + @doc """ + Returns the topic and partition id for this consumer process + """ + @spec topic_partition(GenServer.server) :: + {topic :: binary, partition_id :: non_neg_integer} + def topic_partition(gen_consumer) do + GenServer.call(gen_consumer, :topic_partition) + end + # GenServer callbacks def init({consumer_module, group_name, topic, partition, opts}) do @@ -332,6 +341,10 @@ defmodule KafkaEx.GenConsumer do {:ok, state, 0} end + def handle_call(:topic_partition, _from, state) do + {:reply, {state.topic, state.partition}, state, 0} + end + def handle_info( :timeout, %State{current_offset: nil, last_commit: nil} = state diff --git a/lib/kafka_ex/gen_consumer/supervisor.ex b/lib/kafka_ex/gen_consumer/supervisor.ex index 78a920ed..50b35702 100644 --- a/lib/kafka_ex/gen_consumer/supervisor.ex +++ b/lib/kafka_ex/gen_consumer/supervisor.ex @@ -53,6 +53,18 @@ defmodule KafkaEx.GenConsumer.Supervisor do end end + @doc """ + Returns a list of child pids + + Intended to be used for operational and testing purposes + """ + @spec child_pids(pid | atom) :: [pid] + def child_pids(supervisor_pid) do + supervisor_pid + |> Supervisor.which_children + |> Enum.map(fn({_, pid, _, _}) -> pid end) + end + def init({consumer_module, group_name, _assignments, _opts}) do children = [ worker(KafkaEx.GenConsumer, [consumer_module, group_name]) diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index 4f658147..e5796e96 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -2,6 +2,7 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do use ExUnit.Case alias KafkaEx.ConsumerGroup + alias KafkaEx.GenConsumer import TestHelper require Logger @@ -215,6 +216,53 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end test "basic startup, consume, and shutdown test", context do + generation_id1 = ConsumerGroup.generation_id(context[:consumer_group_pid1]) + generation_id2 = ConsumerGroup.generation_id(context[:consumer_group_pid2]) + assert generation_id1 == generation_id2 + + member1 = ConsumerGroup.member_id(context[:consumer_group_pid1]) + member2 = ConsumerGroup.member_id(context[:consumer_group_pid2]) + assert member1 != member2 + + leader1 = ConsumerGroup.leader_id(context[:consumer_group_pid1]) + leader2 = ConsumerGroup.leader_id(context[:consumer_group_pid2]) + assert leader1 == leader2 + + cond do + leader1 == member1 -> + assert ConsumerGroup.leader?(context[:consumer_group_pid1]) + refute ConsumerGroup.leader?(context[:consumer_group_pid2]) + leader1 == member2 -> + refute ConsumerGroup.leader?(context[:consumer_group_pid1]) + assert ConsumerGroup.leader?(context[:consumer_group_pid2]) + true -> + raise "Neither member is the leader" + end + + assignments1 = ConsumerGroup.assignments(context[:consumer_group_pid1]) + assignments2 = ConsumerGroup.assignments(context[:consumer_group_pid2]) + assert 2 == length(assignments1) + assert 2 == length(assignments2) + refute assignments1 == assignments2 + + consumer1_pid = + ConsumerGroup.consumer_supervisor_pid(context[:consumer_group_pid1]) + consumer1_assignments = consumer1_pid + |> GenConsumer.Supervisor.child_pids + |> Enum.map(&GenConsumer.topic_partition/1) + |> Enum.sort + + assert consumer1_assignments == Enum.sort(assignments1) + + consumer2_pid = + ConsumerGroup.consumer_supervisor_pid(context[:consumer_group_pid2]) + consumer2_assignments = consumer2_pid + |> GenConsumer.Supervisor.child_pids + |> Enum.map(&GenConsumer.topic_partition/1) + |> Enum.sort + + assert consumer2_assignments == Enum.sort(assignments2) + partition_range = 0..(@partition_count - 1) # the assign_partitions callback should have been called with all 4 From ae4b964733881bf5df1541f6c4d5b5bfb4e6e5b1 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 7 Aug 2017 11:56:10 -0400 Subject: [PATCH 44/47] Test assignments directly This should be more robust and exercises some of the ops functions --- lib/kafka_ex/consumer_group.ex | 14 +++ lib/kafka_ex/gen_consumer/supervisor.ex | 10 +++ .../consumer_group_implementation_test.exs | 89 ++++--------------- 3 files changed, 41 insertions(+), 72 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index f8e5ab6b..747b37a6 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -76,6 +76,7 @@ defmodule KafkaEx.ConsumerGroup do use Supervisor alias KafkaEx.ConsumerGroup.PartitionAssignment + alias KafkaEx.GenConsumer @typedoc """ Option values used when starting a consumer group @@ -212,6 +213,19 @@ defmodule KafkaEx.ConsumerGroup do call_manager(supervisor_pid, :consumer_supervisor_pid) end + @doc """ + Returns true if at least one child consumer process is alive + """ + @spec active?(Supervisor.supervisor) :: boolean + def active?(supervisor_pid) do + consumer_supervisor = consumer_supervisor_pid(supervisor_pid) + if consumer_supervisor && Process.alive?(consumer_supervisor) do + GenConsumer.Supervisor.active?(consumer_supervisor) + else + false + end + end + @doc false # used by ConsumerGroup.Manager to set partition assignments def start_consumer(pid, consumer_module, group_name, assignments, opts) do child = supervisor( diff --git a/lib/kafka_ex/gen_consumer/supervisor.ex b/lib/kafka_ex/gen_consumer/supervisor.ex index 50b35702..d89451d3 100644 --- a/lib/kafka_ex/gen_consumer/supervisor.ex +++ b/lib/kafka_ex/gen_consumer/supervisor.ex @@ -65,6 +65,16 @@ defmodule KafkaEx.GenConsumer.Supervisor do |> Enum.map(fn({_, pid, _, _}) -> pid end) end + @doc """ + Returns true if any child pids are alive + """ + @spec active?(Supervisor.supervisor) :: boolean + def active?(supervisor_pid) do + supervisor_pid + |> child_pids + |> Enum.any?(&Process.alive?/1) + end + def init({consumer_module, group_name, _assignments, _opts}) do children = [ worker(KafkaEx.GenConsumer, [consumer_module, group_name]) diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index e5796e96..dc02e984 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -54,20 +54,6 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do Enum.map(events, &(&1.source)) end - def on_assign_partitions(topic, members, partitions, assignments) do - event( - %Event{ - type: :assign_partitions, - key: topic, - payload: %{ - members: members, - partitions: partitions, - assignments: assignments - } - } - ) - end - def on_handled_message_set(message_set, topic, partition) do event( %Event{ @@ -86,13 +72,6 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do |> List.last end - def get_assigns(topic) do - all_events() - |> by_type(:assign_partitions) - |> by_key(topic) - |> payloads() - end - def last_handler(topic, partition) do all_events() |> by_type(:handled_message_set) @@ -100,11 +79,6 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do |> sources() |> List.last end - - def current_assignments(topic) do - last_assigns = List.last(get_assigns(topic)) - last_assigns.assignments - end end defmodule TestPartitioner do @@ -116,17 +90,7 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do "partition assignment: #{inspect members} #{inspect partitions}" end) - # this assumes we are only consuming from one topic - [{topic_name, _} | _] = partitions - - assignments = PartitionAssignment.round_robin(members, partitions) - TestObserver.on_assign_partitions( - topic_name, - members, - partitions, - assignments - ) - assignments + PartitionAssignment.round_robin(members, partitions) end end @@ -199,8 +163,8 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do # wait for both consumer groups to join wait_for(fn -> - assigns = TestObserver.get_assigns(@topic_name) || [] - length(assigns) > 0 && length(Map.get(List.last(assigns), :members)) == 2 + ConsumerGroup.active?(consumer_group_pid1) && + ConsumerGroup.active?(consumer_group_pid2) end) on_exit fn -> @@ -263,24 +227,10 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do assert consumer2_assignments == Enum.sort(assignments2) - partition_range = 0..(@partition_count - 1) + # all of the partitions should be accounted for + assert @partition_count == length(Enum.uniq(assignments1 ++ assignments2)) - # the assign_partitions callback should have been called with all 4 - # partitions - assigns = TestObserver.get_assigns(@topic_name) - assert length(assigns) == 2 - last_assigns = List.last(assigns) - # we should have two consumers in the most recent batch - assert 2 == length(last_assigns.members) - assert @partition_count == length(last_assigns.partitions) - for ix <- 0..(@partition_count - 1) do - assert {@topic_name, ix} in last_assigns.partitions - end - assignments = TestObserver.current_assignments(@topic_name) - # there should be two cgs with assignments - assert 2 == length(Map.keys(assignments)) - # each worker should have two partitions - assert Enum.all?(Map.values(assignments), fn(p) -> length(p) == 2 end) + partition_range = 0..(@partition_count - 1) starting_offsets = partition_range |> Enum.map(fn(px) -> {px, latest_offset_number(@topic_name, px)} end) @@ -327,19 +277,17 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end test "starting/stopping consumers rebalances assignments", context do - last_assigns = List.last(TestObserver.get_assigns(@topic_name)) - assert 2 == length(last_assigns.members) - Process.unlink(context[:consumer_group_pid1]) sync_stop(context[:consumer_group_pid1]) + # the other cg should get assigned all of the partitions wait_for(fn -> - last_assigns = List.last(TestObserver.get_assigns(@topic_name)) - 1 == length(last_assigns.members) + @partition_count == + length(ConsumerGroup.assignments(context[:consumer_group_pid2])) end) - last_assigns = List.last(TestObserver.get_assigns(@topic_name)) - assert 1 == length(last_assigns.members) + # and become the leader + assert ConsumerGroup.leader?(context[:consumer_group_pid2]) {:ok, consumer_group_pid3} = ConsumerGroup.start_link( TestConsumer, @@ -349,24 +297,21 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do partition_assignment_callback: &TestPartitioner.assign_partitions/2 ) + # the new worker should get assigned some partitions wait_for(fn -> - last_assigns = List.last(TestObserver.get_assigns(@topic_name)) - 2 == length(last_assigns.members) + ConsumerGroup.active?(consumer_group_pid3) end) - last_assigns = List.last(TestObserver.get_assigns(@topic_name)) - assert 2 == length(last_assigns.members) - Process.unlink(context[:consumer_group_pid2]) sync_stop(context[:consumer_group_pid2]) + # now the new cg should get all of the partitions wait_for(fn -> - last_assigns = List.last(TestObserver.get_assigns(@topic_name)) - 1 == length(last_assigns.members) + @partition_count == length(ConsumerGroup.assignments(consumer_group_pid3)) end) - last_assigns = List.last(TestObserver.get_assigns(@topic_name)) - assert 1 == length(last_assigns.members) + # and become the leader + assert ConsumerGroup.leader?(consumer_group_pid3) Process.unlink(consumer_group_pid3) sync_stop(consumer_group_pid3) From d25562b8cef58ab29d06e1d04f3341d4dbc5c91b Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 7 Aug 2017 19:53:31 -0400 Subject: [PATCH 45/47] Add ability for consumer to handle messages Cleaned up the tests a bit --- lib/kafka_ex/consumer_group.ex | 24 ++++ lib/kafka_ex/gen_consumer.ex | 98 ++++++++++++-- .../consumer_group_implementation_test.exs | 123 ++++++------------ 3 files changed, 152 insertions(+), 93 deletions(-) diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index 747b37a6..d020188a 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -213,6 +213,30 @@ defmodule KafkaEx.ConsumerGroup do call_manager(supervisor_pid, :consumer_supervisor_pid) end + @doc """ + Returns the pids of consumer processes + """ + @spec consumer_pids(Supervisor.supervisor) :: [pid] + def consumer_pids(supervisor_pid) do + supervisor_pid + |> consumer_supervisor_pid + |> GenConsumer.Supervisor.child_pids + end + + @doc """ + Returns a map from `{topic, partition_id}` to consumer pid + """ + @spec partition_consumer_map(Supervisor.supervisor) :: + %{{topic :: binary, partition_id :: non_neg_integer} => pid} + def partition_consumer_map(supervisor_pid) do + supervisor_pid + |> consumer_pids + |> Enum.map(fn(pid) -> + {GenConsumer.partition(pid), pid} + end) + |> Enum.into(%{}) + end + @doc """ Returns true if at least one child consumer process is alive """ diff --git a/lib/kafka_ex/gen_consumer.ex b/lib/kafka_ex/gen_consumer.ex index dbadeb90..1428a50e 100644 --- a/lib/kafka_ex/gen_consumer.ex +++ b/lib/kafka_ex/gen_consumer.ex @@ -118,12 +118,40 @@ defmodule KafkaEx.GenConsumer do often a `KafkaEx.GenConsumer` auto-commits. For high-volume topics, `:commit_threshold` is the dominant factor. - ## Integration with OTP + ## Handler state and interaction - A `KafkaEx.GenConsumer` is a specialized `GenServer`. It can be supervised, - registered, and debugged the same as any other `GenServer`. Use - `start_link/5` to start a `KafkaEx.GenConsumer` properly; do not use - `GenServer.start_link/3` directly to start a `KafkaEx.GenConsumer`. + Use the `c:init/2` to initialize consumer state and `c:handle_call/3` + to interact. + + Example: + + ``` + defmodule MyConsumer do + use KafkaEx.GenConsumer + + defmodule State do + defstruct messages: [], calls: 0 + end + + def init(_topic, _partition) do + {:ok, %State{}} + end + + def handle_message_set(message_set, state) do + {:async_commit, %{state | messages: state.messages ++ message_set}} + end + + def handle_call(:messages, _from, messages_so_far) + {:reply, state.messages, %{state | calls: state.calls + 1}} + end + end + + {:ok, pid} = GenConsumer.start_link(MyConsumer, "consumer_group", "topic", 0) + GenConsumer.call(pid, :messages) + ``` + + **NOTE** If you do not implement a `c:handle_call/3` callback, any calls to + `GenConsumer.call/3` that go to your consumer will cause a `MatchError`. ## Testing @@ -216,6 +244,15 @@ defmodule KafkaEx.GenConsumer do @callback handle_message_set(message_set :: [Message.t], state :: term) :: {:async_commit, new_state :: term} | {:sync_commit, new_state :: term} + @doc """ + Invoked by `KafkaEx.GenConsumer.call/3`. + + Note the default implementation will cause a `MatchError`. If you want to + interact with your consumer, you must implement a handle_call function. + """ + @callback handle_call(call :: term, from :: GenServer.from, state :: term) + :: {:reply, reply_value :: term, new_state :: term} + defmacro __using__(_opts) do quote do @behaviour KafkaEx.GenConsumer @@ -225,7 +262,12 @@ defmodule KafkaEx.GenConsumer do {:ok, nil} end - defoverridable [init: 2] + def handle_call(_call, _from, _consumer_state) do + # the user must implement this if they expect to recieve calls + :handle_call_not_implemented + end + + defoverridable [init: 2, handle_call: 3] end end @@ -301,10 +343,26 @@ defmodule KafkaEx.GenConsumer do @doc """ Returns the topic and partition id for this consumer process """ - @spec topic_partition(GenServer.server) :: + @spec partition(GenServer.server) :: {topic :: binary, partition_id :: non_neg_integer} - def topic_partition(gen_consumer) do - GenServer.call(gen_consumer, :topic_partition) + def partition(gen_consumer) do + GenServer.call(gen_consumer, :partition) + end + + @doc """ + Forwards a `GenServer.call/3` to the consumer implementation with the + consumer's state. + + The implementation must return a `GenServer.call/3`-compatible value of the + form `{:reply, reply_value, new_consumer_state}`. The GenConsumer will + turn this into an immediate timeout, which drives continued message + consumption. + + See the moduledoc for an example. + """ + @spec call(GenServer.server, term, timeout) :: term + def call(gen_consumer, message, timeout \\ 5000) do + GenServer.call(gen_consumer, {:consumer_call, message}, timeout) end # GenServer callbacks @@ -341,10 +399,30 @@ defmodule KafkaEx.GenConsumer do {:ok, state, 0} end - def handle_call(:topic_partition, _from, state) do + def handle_call(:partition, _from, state) do {:reply, {state.topic, state.partition}, state, 0} end + def handle_call( + {:consumer_call, message}, + from, + %State{ + consumer_module: consumer_module, + consumer_state: consumer_state + } = state + ) do + # NOTE we only support the {:reply, _, _} result format here + # which we turn into a timeout = 0 clause so that we continue to consume. + # any other GenServer flow control could have unintended consequences, + # so we leave that for later consideration + {:reply, reply, new_consumer_state} = consumer_module.handle_call( + message, + from, + consumer_state + ) + {:reply, reply, %{state | consumer_state: new_consumer_state}, 0} + end + def handle_info( :timeout, %State{current_offset: nil, last_commit: nil} = state diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index dc02e984..61695c06 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -14,82 +14,25 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do @partition_count 4 @consumer_group_name "consumer_group_implementation" - defmodule TestObserver do - defmodule Event do - defstruct type: nil, source: nil, key: nil, payload: nil - - def type?(%Event{type: type}, type), do: true - def type?(%Event{}, _type), do: false - - def key?(%Event{key: key}, key), do: true - def key?(%Event{}, _key), do: false - end + defmodule TestPartitioner do + alias KafkaEx.ConsumerGroup.PartitionAssignment def start_link do - Agent.start_link(fn -> [] end, name: __MODULE__) + Agent.start_link(fn -> 0 end, name: __MODULE__) end - def event(event = %Event{}) do - event = %{event | source: self()} - Agent.update(__MODULE__, fn(events) -> events ++ [event] end) - end - - def all_events() do + def calls do Agent.get(__MODULE__, &(&1)) end - def by_type(events, type) do - Enum.filter(events, &Event.type?(&1, type)) - end - - def by_key(events, key) do - Enum.filter(events, &Event.key?(&1, key)) - end - - def payloads(events) do - Enum.map(events, &(&1.payload)) - end - - def sources(events) do - Enum.map(events, &(&1.source)) - end - - def on_handled_message_set(message_set, topic, partition) do - event( - %Event{ - type: :handled_message_set, - key: {topic, partition}, - payload: message_set - } - ) - end - - def last_handled_message_set(topic, partition) do - all_events() - |> by_type(:handled_message_set) - |> by_key({topic, partition}) - |> payloads() - |> List.last - end - - def last_handler(topic, partition) do - all_events() - |> by_type(:handled_message_set) - |> by_key({topic, partition}) - |> sources() - |> List.last - end - end - - defmodule TestPartitioner do - alias KafkaEx.ConsumerGroup.PartitionAssignment - def assign_partitions(members, partitions) do Logger.debug(fn -> "Consumer #{inspect self()} got " <> "partition assignment: #{inspect members} #{inspect partitions}" end) + Agent.update(__MODULE__, &(&1 + 1)) + PartitionAssignment.round_robin(members, partitions) end end @@ -97,21 +40,31 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do defmodule TestConsumer do use KafkaEx.GenConsumer - alias KafkaEx.ConsumerGroupImplementationTest.TestObserver + alias KafkaEx.GenConsumer + + def last_message_set(pid) do + List.last(GenConsumer.call(pid, :message_sets)) || [] + end def init(topic, partition) do Logger.debug(fn -> "Initialized consumer #{inspect self()} for #{topic}:#{partition}" end) - {:ok, %{topic: topic, partition: partition}} + {:ok, %{topic: topic, partition: partition, message_sets: []}} + end + + def handle_call(:message_sets, _from, state) do + {:reply, state.message_sets, state} end def handle_message_set(message_set, state) do Logger.debug(fn -> "Consumer #{inspect self()} handled message set #{inspect message_set}" end) - TestObserver.on_handled_message_set(message_set, state.topic, state.partition) - {:async_commit, state} + { + :async_commit, + %{state | message_sets: state.message_sets ++ [message_set]} + } end end @@ -145,7 +98,7 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end setup do - {:ok, _} = TestObserver.start_link + {:ok, _} = TestPartitioner.start_link {:ok, consumer_group_pid1} = ConsumerGroup.start_link( TestConsumer, @consumer_group_name, @@ -180,6 +133,8 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end test "basic startup, consume, and shutdown test", context do + assert TestPartitioner.calls > 0 + generation_id1 = ConsumerGroup.generation_id(context[:consumer_group_pid1]) generation_id2 = ConsumerGroup.generation_id(context[:consumer_group_pid2]) assert generation_id1 == generation_id2 @@ -213,7 +168,7 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do ConsumerGroup.consumer_supervisor_pid(context[:consumer_group_pid1]) consumer1_assignments = consumer1_pid |> GenConsumer.Supervisor.child_pids - |> Enum.map(&GenConsumer.topic_partition/1) + |> Enum.map(&GenConsumer.partition/1) |> Enum.sort assert consumer1_assignments == Enum.sort(assignments1) @@ -222,7 +177,7 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do ConsumerGroup.consumer_supervisor_pid(context[:consumer_group_pid2]) consumer2_assignments = consumer2_pid |> GenConsumer.Supervisor.child_pids - |> Enum.map(&GenConsumer.topic_partition/1) + |> Enum.map(&GenConsumer.partition/1) |> Enum.sort assert consumer2_assignments == Enum.sort(assignments2) @@ -243,20 +198,23 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end) |> Enum.into(%{}) + consumers = Map.merge( + ConsumerGroup.partition_consumer_map(context[:consumer_group_pid1]), + ConsumerGroup.partition_consumer_map(context[:consumer_group_pid2]) + ) + # we actually consume the messages - for px <- partition_range do + last_offsets = partition_range + |> Enum.map(fn(px) -> + consumer_pid = Map.get(consumers, {@topic_name, px}) wait_for(fn -> - message_set = TestObserver.last_handled_message_set(@topic_name, px) + message_set = TestConsumer.last_message_set(consumer_pid) right_last_message?(message_set, messages[px], starting_offsets[px]) end) - end - - # each partition should be getting handled by a different consumer - handlers = Enum.map( - partition_range, - &(TestObserver.last_handler(@topic_name, &1)) - ) - assert handlers == Enum.uniq(handlers) + last_message = List.last(TestConsumer.last_message_set(consumer_pid)) + {px, last_message.offset} + end) + |> Enum.into(%{}) # stop the supervisors Process.unlink(context[:consumer_group_pid1]) @@ -269,9 +227,8 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do wait_for(fn -> ending_offset = latest_consumer_offset_number(@topic_name, px, @consumer_group_name) - message_set = TestObserver.last_handled_message_set(@topic_name, px) - last_message = List.last(message_set) - ending_offset == last_message.offset + 1 + last_offset = Map.get(last_offsets, px) + ending_offset == last_offset + 1 end) end end From ede2b09b6fd67decd1fea10aee0194a443fe5778 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Mon, 7 Aug 2017 20:01:38 -0400 Subject: [PATCH 46/47] More test cleanup --- .../consumer_group_implementation_test.exs | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index 61695c06..afb0de55 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -15,6 +15,9 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do @consumer_group_name "consumer_group_implementation" defmodule TestPartitioner do + # wraps an Agent that we use to capture the fact that the partitioner was + # called - normally one would not really need to do this + alias KafkaEx.ConsumerGroup.PartitionAssignment def start_link do @@ -38,6 +41,8 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end defmodule TestConsumer do + # test consumer - keeps track of messages handled + use KafkaEx.GenConsumer alias KafkaEx.GenConsumer @@ -50,7 +55,7 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do Logger.debug(fn -> "Initialized consumer #{inspect self()} for #{topic}:#{partition}" end) - {:ok, %{topic: topic, partition: partition, message_sets: []}} + {:ok, %{message_sets: []}} end def handle_call(:message_sets, _from, state) do @@ -73,9 +78,9 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do message end - def right_last_message?(nil, _, _), do: false - def right_last_message?([], _, _), do: false - def right_last_message?(message_set, expected_message, expected_offset) do + def correct_last_message?(nil, _, _), do: false + def correct_last_message?([], _, _), do: false + def correct_last_message?(message_set, expected_message, expected_offset) do Logger.debug(fn -> "Got message set: #{inspect message_set} " <> "expecting '#{expected_message}' @ offset #{expected_offset}" @@ -84,10 +89,6 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do message.value == expected_message && message.offset == expected_offset end - def topic_name do - @topic_name - end - def sync_stop(pid) when is_pid(pid) do wait_for(fn -> if Process.alive?(pid) do @@ -209,7 +210,7 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do consumer_pid = Map.get(consumers, {@topic_name, px}) wait_for(fn -> message_set = TestConsumer.last_message_set(consumer_pid) - right_last_message?(message_set, messages[px], starting_offsets[px]) + correct_last_message?(message_set, messages[px], starting_offsets[px]) end) last_message = List.last(TestConsumer.last_message_set(consumer_pid)) {px, last_message.offset} From ca7ff34c428a08bb5baa5b7b6d65962457948bf4 Mon Sep 17 00:00:00 2001 From: Dan Swain Date: Tue, 8 Aug 2017 21:23:21 -0400 Subject: [PATCH 47/47] Review feedback --- test/integration/consumer_group_implementation_test.exs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index afb0de55..3855ab33 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -163,7 +163,10 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do assignments2 = ConsumerGroup.assignments(context[:consumer_group_pid2]) assert 2 == length(assignments1) assert 2 == length(assignments2) - refute assignments1 == assignments2 + assert MapSet.disjoint?( + Enum.into(assignments1, MapSet.new), + Enum.into(assignments2, MapSet.new) + ) consumer1_pid = ConsumerGroup.consumer_supervisor_pid(context[:consumer_group_pid1])