Repository: dashbitco/broadway_kafka Branch: main Commit: 309b5a89a1bd Files: 20 Total size: 124.1 KB Directory structure: gitextract_obhv2y98/ ├── .formatter.exs ├── .github/ │ └── workflows/ │ └── ci.yml ├── .gitignore ├── CHANGELOG.md ├── Earthfile ├── README.md ├── docker-compose.yml ├── lib/ │ ├── broadway_kafka/ │ │ ├── acknowledger.ex │ │ ├── allocator.ex │ │ ├── brod_client.ex │ │ ├── kafka_client.ex │ │ └── producer.ex │ └── broadway_kafka.ex ├── mix.exs └── test/ ├── acknowledger_test.exs ├── allocator_test.exs ├── brod_client_test.exs ├── integration/ │ └── consume_test.exs ├── producer_test.exs └── test_helper.exs ================================================ FILE CONTENTS ================================================ ================================================ FILE: .formatter.exs ================================================ # Used by "mix format" [ inputs: ["{mix,.formatter}.exs", "{config,lib,test}/**/*.{ex,exs}"] ] ================================================ FILE: .github/workflows/ci.yml ================================================ name: CI on: pull_request: push: branches: - main jobs: integration-test: name: integration test runs-on: ubuntu-latest env: FORCE_COLOR: 1 strategy: fail-fast: false matrix: elixirbase: - "1.11.3-erlang-23.2.5-alpine-3.16.0" - "1.14.3-erlang-25.3-alpine-3.17.2" steps: - uses: actions/checkout@v4 - name: Download released earthly run: "sudo /bin/sh -c 'wget https://github.com/earthly/earthly/releases/download/v0.7.2/earthly-linux-amd64 -O /usr/local/bin/earthly && chmod +x /usr/local/bin/earthly'" - name: Kafka Brodway integration-test under ${{matrix.elixirbase}} run: earthly -P --ci --build-arg ELIXIR_BASE=${{matrix.elixirbase}} +integration-test test: runs-on: ubuntu-20.04 env: MIX_ENV: test strategy: fail-fast: false matrix: include: - pair: elixir: 1.7.4 otp: 21.3.8.17 - pair: elixir: 1.15.6 otp: 26.1.1 lint: lint steps: - uses: actions/checkout@v4 - uses: erlef/setup-beam@v1 with: otp-version: ${{matrix.pair.otp}} elixir-version: ${{matrix.pair.elixir}} - uses: actions/cache@v4 with: path: | deps _build key: ${{ runner.os }}-mix-${{matrix.pair.elixir}}-${{matrix.pair.otp}}-${{ hashFiles('**/mix.lock') }} restore-keys: | ${{ runner.os }}-mix-${{matrix.pair.elixir}}-${{matrix.pair.otp}}- - run: mix deps.get - run: mix format --check-formatted if: ${{ matrix.lint }} - run: mix deps.unlock --check-unused if: ${{ matrix.lint }} - run: mix deps.compile - run: mix compile --warnings-as-errors if: ${{ matrix.lint }} - run: mix test ================================================ FILE: .gitignore ================================================ # The directory Mix will write compiled artifacts to. /_build/ # If you run "mix test --cover", coverage assets end up here. /cover/ # The directory Mix downloads your dependencies sources to. /deps/ # Where 3rd-party dependencies like ExDoc output generated docs. /doc/ # Ignore .fetch files in case you like to edit your project deps locally. /.fetch # If the VM crashes, it generates a dump, let's ignore it too. erl_crash.dump # Also ignore archive artifacts (built via "mix archive.build"). *.ez ================================================ FILE: CHANGELOG.md ================================================ # Changelog ## v0.4.4 (2024-07-08) * Update brod dependency to ~> 3.6 or ~> 4.0 ## v0.4.3 (2024-06-12) * Support setting socket options on Kafka connection ## v0.4.2 (2024-04-01) * Add support for timestamp `:offset_reset_policy` * Supports reading sasl credentials from file * Add `:begin_offset` option * Do not block until coordinator exits to avoid deadlocks ## v0.4.1 (2023-03-14) * Disable `offset_commit_on_ack` during revoke assignment call ## v0.4.0 (2022-08-23) * Implement fairer distribution of messages across partitions * Add :request_timeout option to BrodClient * Send telemetry when assignments_revoked is executed ## v0.3.6 (2022-06-22) * Do not poll while draining * Properly shutdown Brod's group coordinators * Support `:rebalance_timeout_seconds` option * Support `:query_api_versions` option ## v0.3.5 (2022-05-12) * Do not send more messages per partition than `max_demand` * Support for custom `:sasl` authenticators by allowing :callback to be sent as opts ## v0.3.4 (2022-03-19) * Pass SSL configurations when validating offset ## v0.3.3 (2022-02-16) * Ensure `handle_info` does not crash when receiving an EXIT message in case Kafka goes offline ## v0.3.2 (2022-02-15) * Ensure reset_policy is only used when offset is `undefined` or `out_of_range` ## v0.3.1 (2022-01-19) * Allow `:sasl` option to be set to `:undefined` * Allow `:heartbeat_rate_seconds` option to group config * Handle `:offset_out_of_range` errors when resolving offset * Ensure `brod` processes terminate on disconnections ## v0.3.0 (2021-08-30) * Support `:client_id_prefix` to make it easier to identify Kafka connections * Add support for `:max_wait_time` in fetch * Require Broadway 1.0 ## v0.2.0 (2021-03-11) * Add an API for updating topics on producers * Support consuming compacted topics ## v0.1.4 (2020-07-25) * Relax Brod dependency ## v0.1.3 (2020-04-27) * Fix resetting offset on every assignment ## v0.1.2 (2020-04-02) * Add support for single string hosts configuration * Add support for tuple lists with string as key in hosts * Drop invalid support of topic/partition for topics option ## v0.1.1 (2020-02-28) * Add SASL authentication support * Allow boolean for client config ssl option * Append `headers` to message metadata ## v0.1.0 (2020-02-19) * Initial release ================================================ FILE: Earthfile ================================================ VERSION 0.7 all: BUILD \ --build-arg ELIXIR_BASE=1.11.3-erlang-23.2.5-alpine-3.16.0 \ --build-arg ELIXIR_BASE=1.14.3-erlang-25.3-alpine-3.17.2 \ +integration-test setup-base: ARG ELIXIR_BASE=1.13.4-erlang-24.3.4.2-alpine-3.16.0 FROM hexpm/elixir:$ELIXIR_BASE RUN apk add --no-progress --update build-base RUN mix local.rebar --force RUN mix local.hex --force ENV ELIXIR_ASSERT_TIMEOUT=10000 WORKDIR /src/broadway_kafka integration-test-base: FROM +setup-base RUN apk add --no-progress --update docker docker-compose git integration-test: FROM +integration-test-base COPY mix.exs mix.lock .formatter.exs docker-compose.yml ./ RUN mix deps.get COPY --dir lib test ./ # then run the tests WITH DOCKER --compose docker-compose.yml RUN set -e; \ mix test --only integration END ================================================ FILE: README.md ================================================ # BroadwayKafka [![Build Status](https://github.com/dashbitco/broadway_kafka/actions/workflows/ci.yml/badge.svg)](https://github.com/dashbitco/broadway_kafka/actions/workflows/ci.yml) A Kafka connector for [Broadway](https://github.com/dashbitco/broadway). Documentation can be found at [https://hexdocs.pm/broadway_kafka](https://hexdocs.pm/broadway_kafka). ## Installation Add `:broadway_kafka` to the list of dependencies in `mix.exs`: ```elixir def deps do [ {:broadway_kafka, "~> 0.4.1"} ] end ``` ## Usage Configure Broadway's producer using `BroadwayKafka.Producer`: ```elixir defmodule MyBroadway do use Broadway def start_link(_opts) do Broadway.start_link(__MODULE__, name: __MODULE__, producer: [ module: {BroadwayKafka.Producer, [ hosts: [localhost: 9092], group_id: "group_1", topics: ["test"], ]}, concurrency: 1 ], processors: [ default: [ concurrency: 10 ] ] ) end def handle_message(_, message, _) do IO.inspect(message.data, label: "Got message") message end end ``` ## License Copyright 2019 Plataformatec\ Copyright 2020 Dashbit Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. ================================================ FILE: docker-compose.yml ================================================ version: '3.9' services: zookeeper: image: wurstmeister/zookeeper ports: - "127.0.0.1:2181:2181" kafka: image: wurstmeister/kafka:2.13-2.7.1 ports: - "127.0.0.1:9092:9092" environment: KAFKA_LISTENERS: "INTERNAL://:29092,EXTERNAL://:9092" KAFKA_ADVERTISED_LISTENERS: "INTERNAL://kafka:29092,EXTERNAL://localhost:9092" KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: "INTERNAL:PLAINTEXT,EXTERNAL:PLAINTEXT" KAFKA_INTER_BROKER_LISTENER_NAME: "INTERNAL" KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181 volumes: - /var/run/docker.sock:/var/run/docker.sock depends_on: - zookeeper ================================================ FILE: lib/broadway_kafka/acknowledger.ex ================================================ defmodule BroadwayKafka.Acknowledger do @moduledoc false @behaviour Broadway.Acknowledger @type t :: %{key => value} @type key :: {:brod.group_generation_id(), :brod.topic(), :brod.partition()} @type value :: {pending_offsets, last_offset :: :brod.offset(), seen_offsets} @type pending_offsets :: [:brod.offset()] @type seen_offsets :: [:brod.offset()] @spec new() :: t def new(), do: %{} @doc """ Add keys to the acknowledger. """ @spec add(t, [{:brod.group_generation_id(), :brod.topic(), :brod.partition(), :brod.offset()}]) :: t def add(acknowledgers, list) do for entry <- list, {generation_id, topic, partition, offset} = entry, do: {{generation_id, topic, partition}, {:ordsets.new(), offset, :ordsets.new()}}, into: acknowledgers end @doc """ Whenever demand is pending, we will call: for key <- Acknowledger.keys(acks) do Process.send_after(self(), {:poll, key}, @timeout) end Then on :poll, one should call `last_offset/2`, see if the key is still valid. If so, we should poll Kafka and generate messages. The `ack_ref` of each message should be `{producer_pid, ack_key}`. Once poll is done, `update_last_offset/4` must be called every time messages are sent to the client. """ @spec keys(t) :: [key] def keys(acknowledgers), do: Map.keys(acknowledgers) @doc """ Returns the last offset for key. Returns nil if the key does not exist. """ @spec last_offset(t, key) :: :brod.offset() | nil def last_offset(acknowledgers, key) do case acknowledgers do %{^key => {_, offset, _}} -> offset %{} -> nil end end @doc """ Updates the last offset after every polling. """ @spec update_last_offset(t, key, :brod.offset(), seen_offsets) :: t def update_last_offset(acknowledgers, key, last_offset, new_offsets) do %{^key => {pending, _, seen}} = acknowledgers %{acknowledgers | key => {pending ++ new_offsets, last_offset, seen}} end @doc """ Receives a list of offsets update the current key. Returns `{drained?, new_offset, t}`. The `drained?` value should be used a quick check during draining before checking if all partitions are drained. {drained?, new_offset, acknowledgers} = Acknowledger.update_current_offset(state.acknowledgers, key, offsets) if new_offset do :brod_group_coordinator.ack(..., new_offset) end if drained? && state.draining && Acknowledger.all_drained?(acknowledgers) do ... end """ @spec update_current_offset(t, key, [:brod.offset()]) :: {drained? :: boolean, :brod.offset() | nil, t} def update_current_offset(acknowledgers, key, offsets) when is_list(offsets) do %{^key => {pending, last, seen}} = acknowledgers {new_pending, new_seen} = update_offsets(offsets, pending, seen) next = List.first(pending) || last new_next = List.first(new_pending) || last update = if new_next > next, do: new_next - 1, else: nil value = {new_pending, last, new_seen} {drained?(value), update, %{acknowledgers | key => value}} end # Discard older offsets defp update_offsets([offset | offsets], [current | _] = pending, seen) when offset < current, do: update_offsets(offsets, pending, seen) # Bump latest offset defp update_offsets([offset | offsets], [offset | pending], seen), do: update_offsets(offsets, pending, seen) # Bump from seen defp update_offsets(offsets, [current | pending], [current | seen]), do: update_offsets(offsets, pending, seen) # Merge any left over defp update_offsets(offsets, pending, seen), do: {pending, :ordsets.union(offsets, seen)} @doc """ Returns if all keys drained. """ @spec all_drained?(t) :: boolean() def all_drained?(map) do Enum.all?(map, fn {_, v} -> drained?(v) end) end defp drained?({[], _, []}), do: true defp drained?(_), do: false @doc """ The ack callback. It simply sends messages to the annotated producer. """ def ack({producer_pid, key}, successful, failed) do offsets = Enum.map(successful ++ failed, fn %{acknowledger: {_, _, %{offset: offset}}} -> offset end) send(producer_pid, {:ack, key, Enum.sort(offsets)}) end end ================================================ FILE: lib/broadway_kafka/allocator.ex ================================================ defmodule BroadwayKafka.Allocator do @moduledoc false # The allocator is responsible to allocate Kafka partitions # for each layer of stages in Broadway. We have an allocator # for each processor (vertical) and an allocator for each # batcher (horizontal). # # The allocator is stateful, as it must avoid rellocations, # as that would imply lost of ordering. use GenServer @doc """ Starts a new allocator. """ def start_link({name, producers, processors}) when is_atom(name) and producers > 0 and processors > 0 do GenServer.start_link(__MODULE__, {name, producers, processors}, name: name) end @doc """ Fetches the partition for a given `key`. """ def fetch!(name, key) when is_atom(name), do: :ets.lookup_element(name, key, 2) @doc """ Allocates the given `new_entries` for `producer`. """ def allocate(name, producer, new_entries) when is_atom(name) and producer >= 0 and is_list(new_entries), do: GenServer.call(name, {:allocate, producer, new_entries}, :infinity) @doc """ Returns the allocation map with partitions as keys and a list of allocated keys as values. Used for testing. """ def to_map(name), do: GenServer.call(name, :to_map, :infinity) @impl true def init({name, producers, processors}) when producers > 0 and processors > 0 do partitions = for i <- 0..(processors - 1), into: %{}, do: {i, %{}} producers = for i <- 0..(producers - 1), into: %{}, do: {i, %{}} old_producers = producers keys = :ets.new(name, [:named_table, :set, :protected, read_concurrency: true]) old_keys = %{} # `producers` - a map from the producer index to all partitions it holds # `old_producers` - a map from the producer index to all partitions it previously held # `partitions` - a map with the processor index pointing with a map as set of partitions # `keys` - an ets set from the partition pointing to the processor index # `old_keys` - a map from the partition pointing to its previous processor index {:ok, {producers, old_producers, partitions, keys, old_keys}} end @impl true def handle_call(:to_map, _from, {_, _, partitions, _, _} = state) do map = for {partition, map} <- partitions, into: %{} do {partition, Map.keys(map)} end {:reply, map, state} end @impl true def handle_call( {:allocate, producer, new_entries}, _from, {producers, old_producers, partitions, keys, old_keys} ) do # Update the producer allocation old_entries = Map.fetch!(old_producers, producer) current_entries = Map.fetch!(producers, producer) old_producers = Map.put(producers, producer, current_entries) producers = Map.put(producers, producer, new_entries) # We remove the current entries that are no longer used to_remove_entries = Enum.reject(current_entries, &in_a_producer?(producers, &1)) partitions = remove_unused(to_remove_entries, partitions, keys) # We remove the old entries that are no longer used fun = &(in_a_producer?(producers, &1) or in_a_producer?(old_producers, &1)) old_keys = Map.drop(old_keys, Enum.reject(old_entries, fun)) # Now we reject anything that is currently assigned and # split into unseen and seen entries. {seen_entries, unseen_entries} = new_entries |> Enum.reject(&:ets.member(keys, &1)) |> Enum.split_with(&Map.has_key?(old_keys, &1)) # We first assign the ones that we know the location # so we can do a better distribution for the unseen ones partitions = assign_seen_entries(seen_entries, partitions, old_keys, keys) # Now we assign unseen ones {_, partitions, old_keys} = assign_unseen_entries(unseen_entries, partitions, old_keys, keys) {:reply, :ok, {producers, old_producers, partitions, keys, old_keys}} end defp in_a_producer?(producers, entry) do Enum.any?(producers, fn {_, entries} -> entry in entries end) end defp remove_unused(to_remove, partitions, keys) do Enum.reduce(to_remove, partitions, fn entry, partitions -> [{^entry, partition}] = :ets.take(keys, entry) {true, partitions} = pop_in(partitions[partition][entry]) partitions end) end defp assign_seen_entries(seen_entries, partitions, old_keys, keys) do Enum.reduce(seen_entries, partitions, fn entry, partitions -> partition = Map.fetch!(old_keys, entry) :ets.insert(keys, {entry, partition}) put_in(partitions[partition][entry], true) end) end defp assign_unseen_entries(unseen_entries, partitions, old_keys, keys) do # Get the partitions with fewer allocations first and assign from there sorted = sort_partitions_by_allocation(partitions) Enum.reduce(unseen_entries, {sorted, partitions, old_keys}, fn entry, {[{size, partition} | sorted], partitions, old_keys} -> sorted = add_to_sorted({size + 1, partition}, sorted) partitions = put_in(partitions[partition][entry], true) old_keys = put_in(old_keys[entry], partition) :ets.insert(keys, {entry, partition}) {sorted, partitions, old_keys} end) end defp sort_partitions_by_allocation(partitions) do partitions |> Enum.map(fn {partition, map} -> {map_size(map), partition} end) |> Enum.sort() end defp add_to_sorted({size, partition}, [{next_size, _} | _] = rest) when size <= next_size, do: [{size, partition} | rest] defp add_to_sorted(tuple, [pair | rest]), do: [pair | add_to_sorted(tuple, rest)] defp add_to_sorted(tuple, []), do: [tuple] end ================================================ FILE: lib/broadway_kafka/brod_client.ex ================================================ defmodule BroadwayKafka.BrodClient do @moduledoc false require Logger @behaviour BroadwayKafka.KafkaClient # We only accept :commit_to_kafka_v2 for now so we hard coded the value # to avoid problems in case :brod's default policy changes in the future @offset_commit_policy :commit_to_kafka_v2 @supported_group_config_options [ :offset_commit_interval_seconds, :rejoin_delay_seconds, :session_timeout_seconds, :heartbeat_rate_seconds, :rebalance_timeout_seconds ] @supported_fetch_config_options [ :min_bytes, :max_bytes, :max_wait_time ] @supported_client_config_options [ :ssl, :sasl, :connect_timeout, :request_timeout, :client_id_prefix, :query_api_versions, :extra_sock_opts, :allow_topic_auto_creation ] @default_receive_interval 2000 # Private option. Not exposed to the user @default_reconnect_timeout 1000 @default_offset_commit_on_ack true @offset_reset_policy_values [:earliest, :latest] @default_offset_reset_policy :latest @begin_offset_values [:assigned, :reset] @default_begin_offset :assigned @default_shared_client false @impl true def init(opts) do with {:ok, hosts} <- validate(opts, :hosts, required: true), {:ok, group_id} <- validate(opts, :group_id, required: true), {:ok, topics} <- validate(opts, :topics, required: true), {:ok, receive_interval} <- validate(opts, :receive_interval, default: @default_receive_interval), {:ok, reconnect_timeout} <- validate(opts, :reconnect_timeout, default: @default_reconnect_timeout), {:ok, offset_commit_on_ack} <- validate(opts, :offset_commit_on_ack, default: @default_offset_commit_on_ack), {:ok, offset_reset_policy} <- validate(opts, :offset_reset_policy, default: @default_offset_reset_policy), {:ok, begin_offset} <- validate(opts, :begin_offset, default: @default_begin_offset), {:ok, shared_client} <- validate(opts, :shared_client, default: @default_shared_client), {:ok, group_config} <- validate_group_config(opts), {:ok, fetch_config} <- validate_fetch_config(opts), {:ok, client_config} <- validate_client_config(opts) do config = %{ hosts: parse_hosts(hosts), group_id: group_id, topics: topics, receive_interval: receive_interval, reconnect_timeout: reconnect_timeout, offset_commit_on_ack: offset_commit_on_ack, offset_reset_policy: offset_reset_policy, begin_offset: begin_offset, group_config: [{:offset_commit_policy, @offset_commit_policy} | group_config], fetch_config: Map.new(fetch_config), client_config: client_config, shared_client: shared_client, shared_client_id: build_shared_client_id(opts) } {:ok, shared_client_child_spec(config), config} end end @impl true def setup(stage_pid, client_id, callback_module, config) do with :ok <- do_start_brod_client(config.hosts, client_id, config.client_config), {:ok, group_coordinator} <- start_link_group_coordinator(stage_pid, client_id, callback_module, config) do Process.monitor(client_id) ref = Process.monitor(group_coordinator) Process.unlink(group_coordinator) {:ok, group_coordinator, ref} end end @impl true def fetch(client_id, topic, partition, offset, opts, _config) do :brod.fetch(client_id, topic, partition, offset, opts) end @impl true def ack(group_coordinator, generation_id, topic, partition, offset, config) do if group_coordinator do :brod_group_coordinator.ack(group_coordinator, generation_id, topic, partition, offset) if config.offset_commit_on_ack do :brod_group_coordinator.commit_offsets(group_coordinator, [{{topic, partition}, offset}]) end end :ok end @impl true def connected?(client_id) do try do match?({:ok, _}, :brod_client.get_metadata(client_id, :all)) catch _type, _reason -> false end end @impl true def disconnect(client_id) do :ok = :brod.stop_client(client_id) :ok end @impl true def resolve_offset(topic, partition, current_offset, offset_reset_policy, config) do policy = offset_reset_policy_value(offset_reset_policy) if current_offset == :undefined do lookup_offset(config.hosts, topic, partition, policy, config.client_config) else case :brod.fetch({config.hosts, config.client_config}, topic, partition, current_offset) do {:ok, _} -> current_offset {:error, :offset_out_of_range} -> lookup_offset(config.hosts, topic, partition, policy, config.client_config) {:error, reason} -> raise "cannot resolve offset (hosts=#{inspect(config.hosts)} topic=#{topic} " <> "partition=#{partition}). Reason: #{inspect(reason)}" end end end defp shared_client_child_spec(%{shared_client: false}), do: [] defp shared_client_child_spec(%{shared_client: true} = config) do [ %{ id: config.shared_client_id, start: {:brod, :start_link_client, [config.hosts, config.shared_client_id, config.client_config]} } ] end defp lookup_offset(hosts, topic, partition, policy, client_config) do case :brod.resolve_offset(hosts, topic, partition, policy, client_config) do {:ok, -1} -> # `:brod.resolve_offset` returns -1 when asked to resolve a timestamp newer # than all the messages in the partition. # -1 is not a valid offset you can use with `:brod.fetch` so we need to # resolve the latest offset instead lookup_offset(hosts, topic, partition, :latest, client_config) {:ok, offset} -> offset {:error, reason} -> raise "cannot resolve begin offset (hosts=#{inspect(hosts)} topic=#{topic} " <> "partition=#{partition}). Reason: #{inspect(reason)}" end end @impl true def update_topics(group_coordinator, topics) do if group_coordinator do :brod_group_coordinator.update_topics(group_coordinator, topics) end :ok end defp start_link_group_coordinator(stage_pid, client_id, callback_module, config) do :brod_group_coordinator.start_link( client_id, config.group_id, config.topics, config.group_config, callback_module, stage_pid ) end defp validate(opts, key, options \\ []) when is_list(opts) do has_key = Keyword.has_key?(opts, key) required = Keyword.get(options, :required, false) default = Keyword.get(options, :default) cond do has_key -> validate_option(key, opts[key]) required -> {:error, "#{inspect(key)} is required"} default != nil -> validate_option(key, default) true -> {:ok, nil} end end defp validate_option(:hosts, value) do if supported_hosts?(value) do {:ok, value} else validation_error( :hosts, "a list of host/port pairs or a single string of comma separated HOST:PORT pairs", value ) end end defp validate_option(:group_id, value) when not is_binary(value) or value == "", do: validation_error(:group_id, "a non empty string", value) defp validate_option(:topics, value) do if is_list(value) && Enum.all?(value, &is_binary/1) do {:ok, value} else validation_error(:topics, "a list of strings", value) end end defp validate_option(:receive_interval, value) when not is_integer(value) or value < 0, do: validation_error(:receive_interval, "a non-negative integer", value) defp validate_option(:reconnect_timeout, value) when not is_integer(value) or value < 0, do: validation_error(:reconnect_timeout, "a non-negative integer", value) defp validate_option(:offset_commit_on_ack, value) when not is_boolean(value), do: validation_error(:offset_commit_on_ack, "a boolean", value) defp validate_option(:offset_reset_policy, {:timestamp, timestamp}) when is_integer(timestamp) and timestamp > 0 do {:ok, {:timestamp, timestamp}} end defp validate_option(:offset_reset_policy, value) when value not in @offset_reset_policy_values do validation_error( :offset_reset_policy, "one of #{inspect(@offset_reset_policy_values)} or `{:timestamp, timestamp}` where timestamp is a non-negative integer", value ) end defp validate_option(:begin_offset, value) when value not in @begin_offset_values do validation_error(:begin_offset, "one of #{inspect(@begin_offset_values)}", value) end defp validate_option(:offset_commit_interval_seconds, value) when not is_integer(value) or value < 1, do: validation_error(:offset_commit_interval_seconds, "a positive integer", value) defp validate_option(:rejoin_delay_seconds, value) when not is_integer(value) or value < 0, do: validation_error(:rejoin_delay_seconds, "a non-negative integer", value) defp validate_option(:session_timeout_seconds, value) when not is_integer(value) or value < 1, do: validation_error(:session_timeout_seconds, "a positive integer", value) defp validate_option(:heartbeat_rate_seconds, value) when not is_integer(value) or value < 1, do: validation_error(:heartbeat_rate_seconds, "a positive integer", value) defp validate_option(:rebalance_timeout_seconds, value) when not is_integer(value) or value < 1, do: validation_error(:rebalance_timeout_seconds, "a positive integer", value) defp validate_option(:min_bytes, value) when not is_integer(value) or value < 1, do: validation_error(:min_bytes, "a positive integer", value) defp validate_option(:max_bytes, value) when not is_integer(value) or value < 1, do: validation_error(:max_bytes, "a positive integer", value) defp validate_option(:max_wait_time, value) when not is_integer(value) or value < 1, do: validation_error(:max_wait_time, "a positive integer", value) defp validate_option(:client_id_prefix, value) when not is_binary(value), do: validation_error(:client_id_prefix, "a string", value) defp validate_option(:shared_client, value) when not is_boolean(value), do: validation_error(:shared_client, "a boolean", value) defp validate_option(:sasl, :undefined), do: {:ok, :undefined} defp validate_option(:sasl, value = {:callback, _callback_module, _opts}), do: {:ok, value} defp validate_option(:sasl, {mechanism, username, password} = value) when mechanism in [:plain, :scram_sha_256, :scram_sha_512] and is_binary(username) and is_binary(password) do {:ok, value} end defp validate_option(:sasl, {mechanism, path} = value) when mechanism in [:plain, :scram_sha_256, :scram_sha_512] and is_binary(path) do {:ok, value} end defp validate_option(:sasl, value) do validation_error( :sasl, "a tuple of SASL mechanism, username and password, or mechanism and path", value ) end defp validate_option(:query_api_versions, value) when not is_boolean(value), do: validation_error(:query_api_versions, "a boolean", value) defp validate_option(:allow_topic_auto_creation, value) when not is_boolean(value), do: validation_error(:allow_topic_auto_creation, "a boolean", value) defp validate_option(:ssl, value) when is_boolean(value), do: {:ok, value} defp validate_option(:ssl, value) do if Keyword.keyword?(value) do {:ok, value} else validation_error(:ssl, "a keyword list of SSL/TLS client options", value) end end defp validate_option(:connect_timeout, value) when not is_integer(value) or value < 1, do: validation_error(:connect_timeout, "a positive integer", value) defp validate_option(:request_timeout, value) when not is_integer(value) or value < 1000, do: validation_error(:request_timeout, "a positive integer >= 1000", value) defp validate_option(_, value), do: {:ok, value} defp validation_error(option, expected, value) do {:error, "expected #{inspect(option)} to be #{expected}, got: #{inspect(value)}"} end defp validate_group_config(opts) do with {:ok, [_ | _] = config} <- validate_supported_opts(opts, :group_config, @supported_group_config_options), {:ok, _} <- validate(config, :offset_commit_interval_seconds), {:ok, _} <- validate(config, :rejoin_delay_seconds), {:ok, _} <- validate(config, :session_timeout_seconds), {:ok, _} <- validate(config, :heartbeat_rate_seconds), {:ok, _} <- validate(config, :rebalance_timeout_seconds) do {:ok, config} end end defp validate_fetch_config(opts) do with {:ok, [_ | _] = config} <- validate_supported_opts(opts, :fetch_config, @supported_fetch_config_options), {:ok, _} <- validate(config, :min_bytes), {:ok, _} <- validate(config, :max_bytes), {:ok, _} <- validate(config, :max_wait_time) do {:ok, config} end end defp validate_client_config(opts) do with {:ok, [_ | _] = config} <- validate_supported_opts(opts, :client_config, @supported_client_config_options), {:ok, _} <- validate(config, :client_id_prefix), {:ok, _} <- validate(config, :sasl), {:ok, _} <- validate(config, :ssl), {:ok, _} <- validate(config, :connect_timeout), {:ok, _} <- validate(config, :request_timeout), {:ok, _} <- validate(config, :query_api_versions), {:ok, _} <- validate(config, :allow_topic_auto_creation) do {:ok, config} end end defp validate_supported_opts(all_opts, group_name, supported_opts) do opts = Keyword.get(all_opts, group_name, []) opts |> Keyword.keys() |> Enum.reject(fn k -> k in supported_opts end) |> case do [] -> {:ok, opts} keys -> {:error, "Unsupported options #{inspect(keys)} for #{inspect(group_name)}"} end end defp offset_reset_policy_value(policy) do case policy do :earliest -> -2 :latest -> -1 {:timestamp, timestamp} when is_integer(timestamp) and timestamp >= 0 -> timestamp end end defp supported_hosts?(hosts_single_binary) when is_binary(hosts_single_binary) do String.match?(hosts_single_binary, ~r/^(.+:[\d]+)(,.+:[\d]+)?$/) end defp supported_hosts?([{key, _value} | rest]) when is_binary(key) or is_atom(key), do: supported_hosts?(rest) defp supported_hosts?([]), do: true defp supported_hosts?(_other), do: false defp parse_hosts(hosts_single_binary) when is_binary(hosts_single_binary) do hosts_single_binary |> String.split(",") |> Enum.map(fn host_port -> [host, port] = String.split(host_port, ":") {host, String.to_integer(port)} end) end defp parse_hosts(hosts), do: hosts defp build_shared_client_id(opts) do if opts[:shared_client] do prefix = get_in(opts, [:client_config, :client_id_prefix]) broadway_name = opts[:broadway][:name] :"#{prefix}#{Module.concat(broadway_name, SharedClient)}" end end defp do_start_brod_client(hosts, client_id, client_config) do case :brod.start_client(hosts, client_id, client_config) do :ok -> :ok # Because we are starting the client on the broadway supervison tree # instead of the :brod supervisor, the already_started error # is not properly handled by :brod.start_client/3 for shared clients # So we must handle it here. {:error, {{:already_started, _}, _}} -> :ok error -> error end end end ================================================ FILE: lib/broadway_kafka/kafka_client.ex ================================================ defmodule BroadwayKafka.KafkaClient do @moduledoc false @typep config :: %{ hosts: [:brod.endpoint()], client_id: :brod.client(), group_id: :brod.group_id(), reconnect_timeout: non_neg_integer, offset_commit_on_ack: boolean, topics: [:brod.topic()], group_config: keyword, client_config: keyword, shared_client: boolean(), shared_client_id: atom() | nil } @typep offset_reset_policy :: :earliest | :latest @typep brod_group_coordinator :: pid() | nil @callback init(opts :: any) :: {:ok, config} | {:error, any} @callback setup( stage_pid :: pid, client_id :: :brod.client(), callback_module :: module, config ) :: {:ok, group_coordinator :: brod_group_coordinator()} | {:error, any} @callback ack( group_coordinator :: brod_group_coordinator(), generation_id :: integer, topic :: binary, partition :: integer, offset :: integer, config ) :: :ok @callback fetch( client_id :: atom, topic :: binary, partition :: integer, offset :: integer, opts :: any, config :: any ) :: {:ok, {offset :: integer, [:brod.message()]}} | {:error, any()} @callback resolve_offset( topic :: binary, partition :: integer, offset :: integer, offset_reset_policy :: offset_reset_policy(), config ) :: offset :: integer | no_return() @callback update_topics(brod_group_coordinator(), [:brod.topic()]) :: :ok @callback connected?(:brod.client()) :: boolean @callback disconnect(:brod.client()) :: :ok end ================================================ FILE: lib/broadway_kafka/producer.ex ================================================ defmodule BroadwayKafka.Producer do @moduledoc """ A Kafka connector for Broadway. BroadwayKafka can subscribe as a consumer to one or more topics and process streams of records within the same consumer group. Communication is done through Kafka's [Consumer API](https://kafka.apache.org/documentation.html#consumerapi) using the [:brod](https://github.com/klarna/brod/) client. ## Options * `:hosts` - Required. A list of host and port tuples or a single string of comma separated HOST:PORT pairs to use for establishing the initial connection to Kafka, e.g. [localhost: 9092]. Examples: # Keyword ["kafka-vm1": 9092, "kafka-vm2": 9092, "kafka-vm3": 9092] # List of tuples [{"kafka-vm1", 9092}, {"kafka-vm2", 9092}, {"kafka-vm3", 9092}] # String "kafka-vm1:9092,kafka-vm2:9092,kafka-vm3:9092" * `:group_id` - Required. A unique string that identifies the consumer group the producer will belong to. * `:topics` - Required. A list of topics that the producer will subscribe to. * `:receive_interval` - Optional. The duration (in milliseconds) for which the producer waits before making a request for more messages. Default is 2000 (2 seconds). * `:offset_commit_on_ack` - Optional. Tells Broadway to send or not an offset commit request after each acknowledgement. Default is `true`. Setting this value to `false` can increase performance since commit requests will respect the `:offset_commit_interval_seconds` option. However, setting long commit intervals might lead to a large number of duplicated records to be processed after a server restart or connection loss. If that's the case, make sure your logic is idempotent when consuming records to avoid inconsistencies. Also, bear in mind the the negative performance impact might be insignificant if you're using batchers since only one commit request will be performed per batch. * `:offset_reset_policy` - Optional. Defines the offset to be used when there's no initial offset in Kafka or if the current offset has expired. Possible values are `:earliest`, `:latest` or {:timestamp, timestamp} (in milliseconds). Default is `:latest`. * `:begin_offset` - Optional. Defines how to get the initial offset for the consumers. The possible values are `:assigned` or `:reset`. When set to `:assigned` the starting offset will be the ones returned in the kafka partition assignments (the lastest committed offsets for the consumer group). When set to `:reset`, the starting offset will be dictated by the `:offset_reset_policy` option, either starting from the `:earliest` or the `:latest` offsets of the topic. Default is `:assigned`. * `:shared_client` - Optional. When false, it starts one client per producer. When true, it starts a single shared client across all producers (which may reduce memory/resource usage). May cause severe performance degradation, see ["Shared Client Performance"](#module-shared-client-performance) for details. Default is `false`. * `:group_config` - Optional. A list of options used to configure the group coordinator. See the ["Group config options"](#module-group-config-options) section below for a list of all available options. * `:fetch_config` - Optional. A list of options used when fetching messages. See the ["Fetch config options"](#module-fetch-config-options) section below for a list of all available options. * `:client_config` - Optional. A list of options used when creating the client. See the ["Client config options"](#module-client-config-options) section below for a list of all available options. ## Group config options The available options that will be passed to `:brod`'s group coordinator. * `:offset_commit_interval_seconds` - Optional. The time interval between two OffsetCommitRequest messages. Default is 5. * `:rejoin_delay_seconds` - Optional. Delay in seconds before rejoining the group. Default is 1. * `:session_timeout_seconds` - Optional. Time in seconds the group coordinator broker waits before considering a member 'down' if no heartbeat or any kind of request is received. A group member may also consider the coordinator broker 'down' if no heartbeat response is received in the past N seconds. Default is 30 seconds. * `:heartbeat_rate_seconds` - Optional. Time in seconds for member to 'ping' group coordinator. Heartbeats are used to ensure that the consumer's session stays active and to facilitate rebalancing when new consumers join or leave the group. The value must be set lower than `:session_timeout_seconds`, typically equal to or lower than 1/3 of that value. It can be adjusted even lower to control the expected time for normal rebalances. Default is 5 seconds. * `:rebalance_timeout_seconds` - Optional. Time in seconds for each worker to join the group once a rebalance has begun. If the timeout is exceeded, then the worker will be removed from the group, which will cause offset commit failures. Default is 30. ## Fetch config options The available options that will be internally passed to `:brod.fetch/5`. * `:min_bytes` - Optional. The minimum amount of data to be fetched from the server. If not enough data is available the request will wait for that much data to accumulate before answering. Default is 1 byte. Setting this value greater than 1 can improve server throughput a bit at the cost of additional latency. * `:max_bytes` - Optional. The maximum amount of data to be fetched at a time from a single partition. Default is 1048576 (1 MiB). Setting greater values can improve server throughput at the cost of more memory consumption. * `:max_wait_time` - Optional. Time in millisecond. Max number of milliseconds allowed for the broker to collect `min_bytes` of messages in fetch response. Default is 1000ms. ## Client config options The available options that will be internally passed to `:brod.start_client/3`. * `:client_id_prefix` - Optional. A string that will be used to build the client id passed to `:brod`. The example value `client_id_prefix: :"\#{Node.self()} -"` would generate the following connection log from our integration tests: 20:41:37.717 [info] :supervisor: {:local, :brod_sup} :started: [ pid: #PID<0.286.0>, id: :"nonode@nohost - Elixir.BroadwayKafka.ConsumerTest.MyBroadway.Broadway.Producer_0.Client", mfargs: {:brod_client, :start_link, [ [localhost: 9092], :"nonode@nohost - Elixir.BroadwayKafka.ConsumerTest.MyBroadway.Broadway.Producer_0.Client", [client_id_prefix: :"nonode@nohost - "] ]}, restart_type: {:permanent, 10}, shutdown: 5000, child_type: :worker ] * `:sasl` - Optional. A a tuple of mechanism which can be `:plain`, `:scram_sha_256` or `:scram_sha_512`, username and password. See the `:brod`'s [`Authentication Support`](https://github.com/klarna/brod#authentication-support) documentation for more information. Default is no sasl options. * `:ssl` - Optional. A boolean or a list of options to use when connecting via SSL/TLS. See the [`tls_client_option`](http://erlang.org/doc/man/ssl.html#type-tls_client_option) documentation for more information. Default is no ssl options. * `:connect_timeout` - Optional. Time in milliseconds to be used as a timeout for `:brod`'s communication with Kafka. Default is to use `:brod`'s default timeout which is currently 5 seconds. * `:request_timeout` - Optional. Time in milliseconds to be used as a timeout for waiting response from Kafka. Default is to use `:brod`'s default timeout which is currently 240 seconds. * `:extra_sock_opts` - Optional. `gen_tcp` socket options. [More info](https://www.erlang.org/doc/man/gen_tcp.html#type-option). Set to `[:inet6]` if your Kafka broker uses IPv6. > **Note**: Currently, Broadway does not support all options provided by `:brod`. If you have a scenario where you need any extra option that is not listed above, please open an issue, so we can consider adding it. ## Example Broadway.start_link(MyBroadway, name: MyBroadway, producer: [ module: {BroadwayKafka.Producer, [ hosts: [localhost: 9092], group_id: "group_1", topics: ["test"], ]}, concurrency: 1 ], processors: [ default: [ concurrency: 10 ] ] ) ## Concurrency and partitioning The concurrency model provided by Kafka is based on partitioning, i.e., the more partitions you have, the more concurrency you get. However, in order to take advantage of this model you need to set up the `:concurrency` options for your processors and batchers accordingly. Having less concurrency than topic/partitions assigned will result in individual processors handling more than one partition, decreasing the overall level of concurrency. Therefore, if you want to always be able to process messages at maximum concurrency (assuming you have enough resources to do it), you should increase the concurrency up front to make sure you have enough processors to handle the extra messages received from new partitions assigned. > **Note**: Even if you don't plan to add more partitions to a Kafka topic, your pipeline can still receive more assignments than planned. For instance, if another consumer crashes, the server will reassign all its topic/partition to other available consumers, including any Broadway producer subscribed to the same topic. ## Handling failed messages `BroadwayKafka` never stops the flow of the stream, i.e. it will **always ack** the messages even when they fail. Unlike queue-based connectors, where you can mark a single message as failed. In Kafka that's not possible due to its single offset per topic/partition ack strategy. If you want to reprocess failed messages, you need to roll your own strategy. A possible way to do that is to implement `c:Broadway.handle_failed/2` and send failed messages to a separated stream or queue for later processing. ## Message metadata When producing messages, the following information will be passed to [`Broadway.Message`](`t:Broadway.Message.t/0`)'s metadata. * `topic` - The topic the message was published. * `partition` - The topic partition. * `offset` - The offset assigned to the message inside the partition. * `key` - The partition key. * `ts` - A timestamp associated with the message. * `headers` - The headers of the message. ## Telemetry This producer emits a few [Telemetry](https://github.com/beam-telemetry/telemetry) events which are listed below. * `[:broadway_kafka, :assignments_revoked, :start | :stop | :exception]` spans - these events are emitted in "span style" when receiving assignments revoked call from consumer group coordinator See `:telemetry.span/3`. ## Shared Client Performance Enabling shared client may drastically decrease performance. Since connection is handled by a single process, producers may block each other waiting for the client response. This is more likely to be an issue if the producers on your pipeline are fetching message from multiple topics and specially if there are very low traffic topics, which may block on batch wait times. To mitigate this, you can split your topics between multiple pipelines, but notice that this will increase the resource usage as well. By creating one new client/connection for each pipeline, you effectively diminishing the `shared_client` resource usage gains. So make sure to measure if you enable this option. """ use GenStage require Logger import Record, only: [defrecordp: 2, extract: 2] alias Broadway.{Message, Acknowledger, Producer} alias BroadwayKafka.Allocator alias BroadwayKafka.Acknowledger @behaviour Producer @behaviour :brod_group_member defrecordp :kafka_message, extract(:kafka_message, from_lib: "brod/include/brod.hrl") defrecordp :brod_received_assignment, extract(:brod_received_assignment, from_lib: "brod/include/brod.hrl") @impl GenStage def init(opts) do Process.flag(:trap_exit, true) config = opts[:initialized_client_config] draining_after_revoke_flag = self() |> drain_after_revoke_table_name!() |> drain_after_revoke_table_init!() prefix = get_in(config, [:client_config, :client_id_prefix]) {_, producer_name} = Process.info(self(), :registered_name) client_id = config[:shared_client_id] || :"#{prefix}#{Module.concat([producer_name, Client])}" max_demand = with [{_first, processor_opts}] <- opts[:broadway][:processors], max_demand when is_integer(max_demand) <- processor_opts[:max_demand] do max_demand else _ -> 10 end state = %{ client: opts[:client] || BroadwayKafka.BrodClient, client_id: client_id, group_coordinator: nil, receive_timer: nil, receive_interval: config.receive_interval, reconnect_timeout: config.reconnect_timeout, acks: Acknowledger.new(), config: config, allocator_names: allocator_names(opts[:broadway]), revoke_caller: nil, draining_after_revoke_flag: draining_after_revoke_flag, demand: 0, shutting_down?: false, buffer: :queue.new(), max_demand: max_demand, shared_client: config.shared_client } {:producer, connect(state)} end defp allocator_names(broadway_config) do broadway_name = broadway_config[:name] broadway_index = broadway_config[:index] processors_allocators = for {name, _} <- broadway_config[:processors] do Module.concat([broadway_name, "Allocator_processor_#{name}"]) end batchers_allocators = for {name, _} <- broadway_config[:batchers] do Module.concat([broadway_name, "Allocator_batcher_consumer_#{name}"]) end {broadway_index, processors_allocators, batchers_allocators} end @impl GenStage def handle_demand(incoming_demand, %{demand: demand} = state) do maybe_schedule_poll(%{state | demand: demand + incoming_demand}, 0) end @impl GenStage def handle_call(:drain_after_revoke, _from, %{group_coordinator: nil} = state) do set_draining_after_revoke!(state.draining_after_revoke_flag, false) {:reply, :ok, [], state} end def handle_call(:drain_after_revoke, from, %{revoke_caller: nil} = state) do state = reset_buffer(state) if Acknowledger.all_drained?(state.acks) do set_draining_after_revoke!(state.draining_after_revoke_flag, false) {:reply, :ok, [], %{state | acks: Acknowledger.new()}} else {:noreply, [], %{state | revoke_caller: from}} end end @impl GenStage def handle_cast({:update_topics, topics}, state) do state.client.update_topics(state.group_coordinator, topics) {:noreply, [], state} end @impl GenStage def handle_info({:poll, key}, %{acks: acks, demand: demand, max_demand: max_demand} = state) do # We only poll if: # # 1. We are not shutting down # 2. We are not waiting for draining after receivd revoke assignment # 3. We know the key being acked # # Note the key may be out of date when polling has been scheduled and # assignments were revoked afterwards, which is why check 3 is necessary. offset = Acknowledger.last_offset(acks, key) if not state.shutting_down? and not is_draining_after_revoke?(state.draining_after_revoke_flag) and offset != nil do messages = fetch_messages_from_kafka(state, key, offset) to_send = min(demand, max_demand) {new_acks, not_sent, messages, pending} = split_demand(messages, acks, key, to_send) new_buffer = enqueue_many(state.buffer, key, pending) new_demand = demand - to_send + not_sent new_state = %{state | acks: new_acks, demand: new_demand, buffer: new_buffer} {:noreply, messages, new_state} else {:noreply, [], state} end end def handle_info(:maybe_schedule_poll, state) do maybe_schedule_poll(%{state | receive_timer: nil}, state.receive_interval) end def handle_info({:put_assignments, group_generation_id, assignments}, state) do list = Enum.map(assignments, fn assignment -> brod_received_assignment( topic: topic, partition: partition, begin_offset: assigned_begin_offset ) = assignment offset_reset_policy = state.config[:offset_reset_policy] begin_offset = case state.config[:begin_offset] do :assigned -> assigned_begin_offset :reset -> :undefined end offset = state.client.resolve_offset( topic, partition, begin_offset, offset_reset_policy, state.config ) {group_generation_id, topic, partition, offset} end) topics_partitions = Enum.map(list, fn {_, topic, partition, _} -> {topic, partition} end) {broadway_index, processors_allocators, batchers_allocators} = state.allocator_names for allocator_name <- processors_allocators do Allocator.allocate(allocator_name, broadway_index, topics_partitions) end for allocator_name <- batchers_allocators do Allocator.allocate(allocator_name, broadway_index, topics_partitions) end {:noreply, [], %{state | acks: Acknowledger.add(state.acks, list)}} end def handle_info({:ack, key, offsets}, state) do %{group_coordinator: group_coordinator, client: client, acks: acks, config: config} = state {generation_id, topic, partition} = key {drained?, new_offset, updated_acks} = Acknowledger.update_current_offset(acks, key, offsets) if new_offset do try do client.ack( group_coordinator, generation_id, topic, partition, new_offset, disable_offset_commit_during_revoke_call(config, state) ) catch kind, reason -> Logger.error(Exception.format(kind, reason, __STACKTRACE__)) end end new_state = if drained? && state.revoke_caller && Acknowledger.all_drained?(updated_acks) do set_draining_after_revoke!(state.draining_after_revoke_flag, false) GenStage.reply(state.revoke_caller, :ok) %{state | revoke_caller: nil, acks: Acknowledger.new()} else %{state | acks: updated_acks} end {:noreply, [], new_state} end def handle_info({:DOWN, _ref, _, {client_id, _}, _reason}, %{client_id: client_id} = state) do if coord = state.group_coordinator do Process.exit(coord, :shutdown) end state = reset_buffer(state) schedule_reconnect(state.reconnect_timeout) {:noreply, [], %{state | group_coordinator: nil}} end def handle_info({:DOWN, _ref, _, coord, _reason}, %{group_coordinator: coord} = state) do state = reset_buffer(state) schedule_reconnect(state.reconnect_timeout) {:noreply, [], %{state | group_coordinator: nil}} end def handle_info({:EXIT, _pid, _reason}, state) do {:noreply, [], state} end def handle_info(:reconnect, state) do if state.client.connected?(state.client_id) do {:noreply, [], connect(state)} else schedule_reconnect(state.reconnect_timeout) {:noreply, [], state} end end def handle_info(_, state) do {:noreply, [], state} end @impl Producer def prepare_for_draining(state) do # On draining, we will continue scheduling the polls, but they will be a no-op. {:noreply, [], %{state | shutting_down?: true}} end @impl Producer def prepare_for_start(_module, opts) do broadway_name = opts[:name] producers_concurrency = opts[:producer][:concurrency] [first_processor_entry | other_processors_entries] = opts[:processors] {allocator, updated_processor_entry} = build_allocator_spec_and_consumer_entry( broadway_name, :processors, "processor", producers_concurrency, first_processor_entry ) {allocators, updated_batchers_entries} = Enum.reduce(opts[:batchers], {[allocator], []}, fn entry, {allocators, entries} -> {allocator, updated_entry} = build_allocator_spec_and_consumer_entry( broadway_name, :batchers, "batcher_consumer", producers_concurrency, entry ) {[allocator | allocators], [updated_entry | entries]} end) updated_opts = opts |> Keyword.put(:processors, [updated_processor_entry | other_processors_entries]) |> Keyword.put(:batchers, updated_batchers_entries) {producer_mod, producer_opts} = opts[:producer][:module] client = producer_opts[:client] || BroadwayKafka.BrodClient case client.init(Keyword.put(producer_opts, :broadway, opts)) do {:error, message} -> raise ArgumentError, "invalid options given to #{client}.init/1, " <> message {:ok, extra_child_specs, config} -> new_producer_opts = Keyword.put(producer_opts, :initialized_client_config, config) updated_opts = put_in(updated_opts, [:producer, :module], {producer_mod, new_producer_opts}) {allocators ++ extra_child_specs, updated_opts} end end @impl :brod_group_member def get_committed_offsets(_pid, _topics_partitions) do raise "not implemented" end @impl :brod_group_member def assignments_received(pid, _group_member_id, group_generation_id, received_assignments) do send(pid, {:put_assignments, group_generation_id, received_assignments}) :ok end @impl :brod_group_member def assignments_revoked(producer_pid) do maybe_process_name = fn pid when is_pid(pid) -> pid name when is_atom(name) -> Process.whereis(name) end producer_pid |> maybe_process_name.() |> drain_after_revoke_table_name!() |> set_draining_after_revoke!(true) metadata = %{producer: maybe_process_name.(producer_pid)} :telemetry.span([:broadway_kafka, :assignments_revoked], metadata, fn -> GenStage.call(producer_pid, :drain_after_revoke, :infinity) {:ok, metadata} end) end @impl GenStage def terminate(_reason, state) do %{client: client, group_coordinator: group_coordinator, client_id: client_id} = state group_coordinator && Process.exit(group_coordinator, :shutdown) if state.shared_client == false do client.disconnect(client_id) end :ok end defp maybe_schedule_poll(%{demand: 0} = state, _interval) do {:noreply, [], state} end defp maybe_schedule_poll(state, interval) do %{buffer: buffer, demand: demand, acks: acks, receive_timer: receive_timer} = state case dequeue_many(buffer, acks, demand, []) do {acks, 0, events, buffer} -> {:noreply, events, %{state | demand: 0, buffer: buffer, acks: acks}} {acks, demand, events, buffer} -> receive_timer = receive_timer || schedule_poll(state, interval) state = %{ state | demand: demand, buffer: buffer, receive_timer: receive_timer, acks: acks } {:noreply, events, state} end end defp schedule_poll(state, interval) do for key <- Acknowledger.keys(state.acks) do Process.send_after(self(), {:poll, key}, interval) end Process.send_after(self(), :maybe_schedule_poll, interval) end defp fetch_messages_from_kafka(state, key, offset) do %{ client: client, client_id: client_id, config: config } = state {generation_id, topic, partition} = key case client.fetch(client_id, topic, partition, offset, config[:fetch_config], config) do {:ok, {_watermark_offset, kafka_messages}} -> Enum.map(kafka_messages, fn k_msg -> wrap_message(k_msg, topic, partition, generation_id) end) {:error, reason} -> raise "cannot fetch records from Kafka (topic=#{topic} partition=#{partition} " <> "offset=#{offset}). Reason: #{inspect(reason)}" end end defp wrap_message(kafka_msg, topic, partition, generation_id) do kafka_message(value: data, offset: offset, key: key, ts: ts, headers: headers) = kafka_msg ack_data = %{offset: offset} ack_ref = {self(), {generation_id, topic, partition}} message = %Message{ data: data, metadata: %{ topic: topic, partition: partition, offset: offset, key: key, ts: ts, headers: headers }, acknowledger: {Acknowledger, ack_ref, ack_data} } Message.put_batch_key(message, {topic, partition}) end defp connect(state) do %{client: client, client_id: client_id, config: config} = state case client.setup(self(), client_id, __MODULE__, config) do {:ok, coord_pid, _coord_ref} -> %{state | group_coordinator: coord_pid} error -> raise "Cannot connect to Kafka. Reason #{inspect(error)}" end end defp build_allocator_spec_and_consumer_entry( broadway_name, group, prefix, producers_concurrency, consumer_entry ) do {consumer_name, consumer_config} = consumer_entry validate_partition_by(group, consumer_name, consumer_config) consumer_concurrency = consumer_config[:concurrency] allocator_name = Module.concat([broadway_name, "Allocator_#{prefix}_#{consumer_name}"]) partition_by = &Allocator.fetch!(allocator_name, {&1.metadata.topic, &1.metadata.partition}) new_config = Keyword.put(consumer_config, :partition_by, partition_by) allocator = {BroadwayKafka.Allocator, {allocator_name, producers_concurrency, consumer_concurrency}} allocator_spec = Supervisor.child_spec(allocator, id: allocator_name) {allocator_spec, {consumer_name, new_config}} end defp validate_partition_by(group, consumer_name, consumer_config) do if Keyword.has_key?(consumer_config, :partition_by) do raise ArgumentError, "cannot set option :partition_by for #{group} #{inspect(consumer_name)}. " <> "The option will be set automatically by BroadwayKafka.Producer" end end ## Buffer handling defp split_demand(list, acks, key, demand) do {rest, demand, reversed, acc} = reverse_split_demand(list, demand, [], []) acks = update_last_offset(acks, key, reversed) {acks, demand, Enum.reverse(acc), rest} end defp reverse_split_demand(rest, 0, reversed, acc) do {rest, 0, reversed, acc} end defp reverse_split_demand([], demand, reversed, acc) do {[], demand, reversed, acc} end defp reverse_split_demand([head | tail], demand, reversed, acc) do reverse_split_demand(tail, demand - 1, [head | reversed], [head | acc]) end defp enqueue_many(queue, _key, []), do: queue defp enqueue_many(queue, key, list), do: :queue.in({key, list}, queue) defp dequeue_many(queue, acks, demand, acc) when demand > 0 do case :queue.out(queue) do {{:value, {key, list}}, queue} -> {rest, demand, reversed, acc} = reverse_split_demand(list, demand, [], acc) acks = update_last_offset(acks, key, reversed) case {demand, rest} do {0, []} -> {acks, demand, Enum.reverse(acc), queue} {0, _} -> {acks, demand, Enum.reverse(acc), :queue.in({key, rest}, queue)} {_, []} -> dequeue_many(queue, acks, demand, acc) end {:empty, queue} -> {acks, demand, Enum.reverse(acc), queue} end end defp update_last_offset(acks, key, [message | _] = reversed) do last = message.metadata.offset + 1 offsets = Enum.reduce(reversed, [], &[&1.metadata.offset | &2]) Acknowledger.update_last_offset(acks, key, last, offsets) end defp update_last_offset(acks, _key, []) do acks end defp reset_buffer(state) do put_in(state.buffer, :queue.new()) end defp schedule_reconnect(timeout) do Process.send_after(self(), :reconnect, timeout) end defp drain_after_revoke_table_name!(pid) do {_, producer_name} = Process.info(pid, :registered_name) Module.concat([producer_name, DrainingAfterRevoke]) end defp drain_after_revoke_table_init!(table_name) do table_name = :ets.new(table_name, [:named_table, :public, :set]) set_draining_after_revoke!(table_name, false) table_name end defp set_draining_after_revoke!(table_name, value) do :ets.insert(table_name, {:draining, value}) end defp is_draining_after_revoke?(table_name) do :ets.lookup_element(table_name, :draining, 2) end defp disable_offset_commit_during_revoke_call(config, state) do offset_commit_on_ack = not is_draining_after_revoke?(state.draining_after_revoke_flag) and state.config.offset_commit_on_ack %{config | offset_commit_on_ack: offset_commit_on_ack} end end ================================================ FILE: lib/broadway_kafka.ex ================================================ defmodule BroadwayKafka do @moduledoc """ Helpers for BroadwayKafka. You can find the Broadway producer in `BroadwayKafka.Producer`. """ @doc """ Sequentially updates topics in all Broadway producers in the pipeline given by `name`. ## Examples BroadwayKafka.update_topics(MyBroadway, ["topic_a", "topic_b"]) :ok BroadwayKafka.update_topics(MyBroadway, []) :ok """ def update_topics(name, topics) do each_producer(name, &GenServer.cast(&1, {:update_topics, topics})) end defp each_producer(server, fun) when is_function(fun, 1) do server |> Broadway.producer_names() |> Enum.each(fun) end end ================================================ FILE: mix.exs ================================================ defmodule BroadwayKafka.MixProject do use Mix.Project @version "0.4.4" @description "A Kafka connector for Broadway" def project do [ app: :broadway_kafka, version: @version, elixir: "~> 1.7", name: "BroadwayKafka", description: @description, start_permanent: Mix.env() == :prod, deps: deps(), docs: docs(), package: package() ] end def application do [ extra_applications: [:logger] ] end defp deps do [ {:broadway, "~> 1.0"}, {:brod, "~> 3.16 or ~> 4.0"}, {:telemetry, "~> 0.4.3 or ~> 1.0"}, {:ex_doc, ">= 0.19.0", only: :docs} ] end defp docs do [ main: "BroadwayKafka.Producer", source_ref: "v#{@version}", source_url: "https://github.com/dashbitco/broadway_kafka" ] end defp package do %{ licenses: ["Apache-2.0"], links: %{"GitHub" => "https://github.com/dashbitco/broadway_kafka"} } end end ================================================ FILE: test/acknowledger_test.exs ================================================ defmodule BroadwayKafka.AcknowledgerTest do use ExUnit.Case, async: true alias BroadwayKafka.Acknowledger, as: Ack @foo {1, "foo", 1} @bar {1, "bar", 2} @ack Ack.add(Ack.new(), [{1, "foo", 1, 10}, {1, "bar", 2, 0}]) test "new" do assert Ack.new() == %{} end test "add" do assert @ack == %{ @foo => {[], 10, []}, @bar => {[], 0, []} } end test "keys" do assert Ack.keys(@ack) |> Enum.sort() == [@bar, @foo] end test "last_offset" do assert Ack.last_offset(@ack, @foo) == 10 assert Ack.last_offset(@ack, @bar) == 0 end test "update_current_offset" do ack = Ack.update_last_offset(@ack, @foo, 20, Enum.to_list(10..19)) assert {true, 19, _} = Ack.update_current_offset(ack, @foo, Enum.to_list(9..19)) ack = Ack.update_last_offset(@ack, @foo, 20, Enum.to_list(10..19)) assert {false, 10, ack} = Ack.update_current_offset(ack, @foo, [10, 13, 14]) assert {true, 19, _} = Ack.update_current_offset(ack, @foo, [11, 12, 15, 16, 17, 18, 19]) ack = Ack.update_last_offset(@ack, @foo, 20, Enum.to_list(10..19)) assert {false, nil, ack} = Ack.update_current_offset(ack, @foo, [13, 14]) assert {false, nil, ack} = Ack.update_current_offset(ack, @foo, [11, 12, 15, 16, 17, 18, 19]) assert {true, 19, _} = Ack.update_current_offset(ack, @foo, [10]) ack = Ack.update_last_offset(@ack, @foo, 20, Enum.to_list(10..19)) assert {false, nil, ack} = Ack.update_current_offset(ack, @foo, [13, 14]) assert {false, 16, ack} = Ack.update_current_offset(ack, @foo, [10, 11, 12, 15, 16, 18, 19]) assert {true, 19, _} = Ack.update_current_offset(ack, @foo, [17]) end test "update_current_offset with gaps" do ack = Ack.update_last_offset(@ack, @foo, 20, [11, 13, 15, 17, 19]) assert {true, 19, _} = Ack.update_current_offset(ack, @foo, [9, 11, 13, 15, 17, 19]) ack = Ack.update_last_offset(@ack, @foo, 20, [11, 13, 15, 17, 19]) assert {false, 12, ack} = Ack.update_current_offset(ack, @foo, [11, 15]) assert {true, 19, _} = Ack.update_current_offset(ack, @foo, [13, 17, 19]) ack = Ack.update_last_offset(@ack, @foo, 20, [11, 13, 15, 17, 19]) assert {false, nil, ack} = Ack.update_current_offset(ack, @foo, [13]) assert {false, nil, ack} = Ack.update_current_offset(ack, @foo, [15, 17, 19]) assert {true, 19, _} = Ack.update_current_offset(ack, @foo, [11]) ack = Ack.update_last_offset(@ack, @foo, 20, [11, 13, 15, 17, 19]) assert {false, nil, ack} = Ack.update_current_offset(ack, @foo, [13]) assert {false, 16, ack} = Ack.update_current_offset(ack, @foo, [11, 15, 19]) assert {true, 19, _} = Ack.update_current_offset(ack, @foo, [17]) end test "all_drained?" do ack = @ack assert Ack.all_drained?(ack) ack = Ack.update_last_offset(ack, @foo, 100, Enum.to_list(10..99)) refute Ack.all_drained?(ack) assert {false, 49, ack} = Ack.update_current_offset(ack, @foo, Enum.to_list(10..49)) refute Ack.all_drained?(ack) assert {true, 99, ack} = Ack.update_current_offset(ack, @foo, Enum.to_list(50..99)) assert Ack.all_drained?(ack) end # Some poor man's property based testing. describe "property based testing" do # We generate a list from 10..99 and we break it into 1..9 random parts. test "drained?" do ack = Ack.update_last_offset(@ack, @foo, 100, Enum.to_list(10..99)) for n_parts <- 1..9 do groups = Enum.group_by(10..99, fn _ -> :rand.uniform(n_parts) end) offsets = Map.values(groups) {drained?, _, ack} = Enum.reduce(offsets, {false, :unused, ack}, fn offset, {false, _, ack} -> Ack.update_current_offset(ack, @foo, Enum.sort(offset)) end) assert drained? assert Ack.all_drained?(ack) end end end end ================================================ FILE: test/allocator_test.exs ================================================ defmodule BroadwayKafka.AllocatorTest do use ExUnit.Case, async: true import BroadwayKafka.Allocator test "evenly allocates partitions at once" do start_link({:a1, 1, 4}) allocate(:a1, 0, [:a, :b]) assert verify!(:a1) == %{0 => [:a], 1 => [:b], 2 => [], 3 => []} start_link({:a2, 1, 4}) allocate(:a2, 0, [:a, :b, :c, :d]) assert verify!(:a2) == %{0 => [:a], 1 => [:b], 2 => [:c], 3 => [:d]} start_link({:a3, 1, 4}) allocate(:a3, 0, [:a, :b, :c, :d, :e, :f]) assert verify!(:a3) == %{0 => [:a], 1 => [:b], 2 => [:c, :f], 3 => [:d, :e]} start_link({:a4, 1, 4}) allocate(:a4, 0, [:a, :b, :c, :d, :e, :f, :g, :h]) assert verify!(:a4) == %{0 => [:a, :h], 1 => [:b, :g], 2 => [:c, :f], 3 => [:d, :e]} start_link({:a5, 1, 4}) allocate(:a5, 0, [:a, :b, :c, :d, :e, :f, :g, :h, :i, :j]) assert verify!(:a5) == %{0 => [:a, :h, :i], 1 => [:b, :g, :j], 2 => [:c, :f], 3 => [:d, :e]} end test "re-allocates partitions keeping current allocation", %{test: name} do start_link({name, 1, 4}) allocate(name, 0, [:a, :b, :c, :d]) assert verify!(name) == %{0 => [:a], 1 => [:b], 2 => [:c], 3 => [:d]} allocate(name, 0, [:c, :d, :e, :f]) assert verify!(name) == %{0 => [:e], 1 => [:f], 2 => [:c], 3 => [:d]} end test "re-allocates partitions keeping last generation allocation", %{test: name} do start_link({name, 1, 4}) allocate(name, 0, [:a, :b, :c, :d]) allocate(name, 0, [:e, :f, :g, :h]) allocate(name, 0, [:c, :b, :d, :a, :f, :g, :h, :e]) assert verify!(name) == %{0 => [:a, :e], 1 => [:b, :f], 2 => [:c, :g], 3 => [:d, :h]} end test "does not remove entries when they are reallocated" do start_link({:b1, 2, 4}) allocate(:b1, 0, [:a, :b, :c, :d]) allocate(:b1, 0, [:e, :f, :g, :h]) allocate(:b1, 1, [:a, :b, :c, :d]) assert verify!(:b1) == %{0 => [:a, :e], 1 => [:b, :f], 2 => [:c, :g], 3 => [:d, :h]} start_link({:b2, 2, 4}) allocate(:b2, 0, [:a, :b, :c, :d]) allocate(:b2, 1, [:a, :b, :c, :d]) allocate(:b2, 0, [:e, :f, :g, :h]) assert verify!(:b2) == %{0 => [:a, :e], 1 => [:b, :f], 2 => [:c, :g], 3 => [:d, :h]} end test "rebalances highly skewed partitions considering previous allocations", %{test: name} do start_link({name, 1, 4}) allocate(name, 0, [:a, :b, :c, :d, :e, :f, :g, :h, :i, :j]) allocate(name, 0, [:a, :c, :f, :h, :i, :j, :k, :l, :m, :n]) assert verify!(name) == %{0 => [:a, :h, :i], 1 => [:j, :m, :n], 2 => [:c, :f], 3 => [:k, :l]} end defp verify!(name) do map = to_map(name) for {partition, entries} <- map, entry <- entries, do: ^partition = fetch!(name, entry) # ensure the value list is sorted, to be able to reliably assert Map.new(map, fn {k, v} -> {k, Enum.sort(v)} end) end end ================================================ FILE: test/brod_client_test.exs ================================================ defmodule BroadwayKafka.BrodClientTest do use ExUnit.Case alias BroadwayKafka.BrodClient @opts [ group_id: "group", hosts: [host: 9092], topics: ["topic"], group_config: [], fetch_config: [], client_config: [] ] describe "validate init options" do test ":hosts is a required value" do opts = Keyword.delete(@opts, :hosts) assert BrodClient.init(opts) == {:error, ":hosts is required"} end test ":hosts is valid as a keyword list or a list of tuples or a single binary" do expected_msg = "expected :hosts to be a list of host/port pairs or a single " <> "string of comma separated HOST:PORT pairs, got: " opts = Keyword.put(@opts, :hosts, :an_atom) assert BrodClient.init(opts) == {:error, expected_msg <> ":an_atom"} opts = Keyword.put(@opts, :hosts, "host") assert BrodClient.init(opts) == {:error, expected_msg <> ~s/"host"/} opts = Keyword.put(@opts, :hosts, "host:9092,") assert BrodClient.init(opts) == {:error, expected_msg <> ~s/"host:9092,"/} opts = Keyword.put(@opts, :hosts, host: 9092) assert {:ok, [], %{hosts: [host: 9092]}} = BrodClient.init(opts) opts = Keyword.put(@opts, :hosts, [{"host", 9092}]) assert {:ok, [], %{hosts: [{"host", 9092}]}} = BrodClient.init(opts) opts = Keyword.put(@opts, :hosts, "host:9092") assert {:ok, [], %{hosts: [{"host", 9092}]}} = BrodClient.init(opts) opts = Keyword.put(@opts, :hosts, "host1:9092,host2:9092") assert {:ok, [], %{hosts: [{"host1", 9092}, {"host2", 9092}]}} = BrodClient.init(opts) end test ":group_id is a required string" do opts = Keyword.delete(@opts, :group_id) assert BrodClient.init(opts) == {:error, ":group_id is required"} opts = Keyword.put(@opts, :group_id, :an_atom) assert BrodClient.init(opts) == {:error, "expected :group_id to be a non empty string, got: :an_atom"} opts = Keyword.put(@opts, :group_id, "my_group") assert {:ok, [], %{group_id: "my_group"}} = BrodClient.init(opts) end test ":topics is a required list of strings" do opts = Keyword.delete(@opts, :topics) assert BrodClient.init(opts) == {:error, ":topics is required"} opts = Keyword.put(@opts, :topics, :an_atom) assert BrodClient.init(opts) == {:error, "expected :topics to be a list of strings, got: :an_atom"} opts = Keyword.put(@opts, :topics, ["topic_1", "topic_2"]) assert {:ok, [], %{topics: ["topic_1", "topic_2"]}} = BrodClient.init(opts) end test ":receive_interval is a non-negative integer with default value 2000" do opts = Keyword.delete(@opts, :receive_interval) assert {:ok, [], %{receive_interval: 2000}} = BrodClient.init(opts) opts = Keyword.put(@opts, :receive_interval, :an_atom) assert BrodClient.init(opts) == {:error, "expected :receive_interval to be a non-negative integer, got: :an_atom"} opts = Keyword.put(@opts, :receive_interval, 1000) assert {:ok, [], %{receive_interval: 1000}} = BrodClient.init(opts) end test ":reconnect_timeout is a non-negative integer with default value 1000" do assert {:ok, [], %{reconnect_timeout: 1000}} = BrodClient.init(@opts) opts = Keyword.put(@opts, :reconnect_timeout, :an_atom) assert BrodClient.init(opts) == {:error, "expected :reconnect_timeout to be a non-negative integer, got: :an_atom"} opts = Keyword.put(@opts, :reconnect_timeout, 2000) assert {:ok, [], %{reconnect_timeout: 2000}} = BrodClient.init(opts) end test ":offset_commit_on_ack is a boolean with default value true" do assert {:ok, [], %{offset_commit_on_ack: true}} = BrodClient.init(@opts) opts = Keyword.put(@opts, :offset_commit_on_ack, :an_atom) assert BrodClient.init(opts) == {:error, "expected :offset_commit_on_ack to be a boolean, got: :an_atom"} opts = Keyword.put(@opts, :offset_commit_on_ack, false) assert {:ok, [], %{offset_commit_on_ack: false}} = BrodClient.init(opts) end test ":offset_reset_policy can be :earliest or :latest. Default is :latest" do assert {:ok, [], %{offset_reset_policy: :latest}} = BrodClient.init(@opts) opts = Keyword.put(@opts, :offset_reset_policy, :an_atom) assert BrodClient.init(opts) == {:error, "expected :offset_reset_policy to be one of [:earliest, :latest] or `{:timestamp, timestamp}` where timestamp is a non-negative integer, got: :an_atom"} opts = Keyword.put(@opts, :offset_reset_policy, :earliest) assert {:ok, [], %{offset_reset_policy: :earliest}} = BrodClient.init(opts) opts = Keyword.put(@opts, :offset_reset_policy, :latest) assert {:ok, [], %{offset_reset_policy: :latest}} = BrodClient.init(opts) end test ":begin_offset can be :assigned or :reset. Default is :assigned" do assert {:ok, [], %{begin_offset: :assigned}} = BrodClient.init(@opts) opts = Keyword.put(@opts, :begin_offset, :an_atom) assert BrodClient.init(opts) == {:error, "expected :begin_offset to be one of [:assigned, :reset], got: :an_atom"} opts = Keyword.put(@opts, :begin_offset, :assigned) assert {:ok, [], %{begin_offset: :assigned}} = BrodClient.init(opts) opts = Keyword.put(@opts, :begin_offset, :reset) assert {:ok, [], %{begin_offset: :reset}} = BrodClient.init(opts) end test ":offset_commit_interval_seconds is an optional non-negative integer" do opts = put_in(@opts, [:group_config, :offset_commit_interval_seconds], :an_atom) assert BrodClient.init(opts) == {:error, "expected :offset_commit_interval_seconds to be " <> "a positive integer, got: :an_atom"} opts = put_in(@opts, [:group_config, :offset_commit_interval_seconds], 3) {:ok, [], %{group_config: group_config}} = BrodClient.init(opts) assert group_config[:offset_commit_interval_seconds] == 3 end test ":rejoin_delay_seconds is an optional non-negative integer" do opts = put_in(@opts, [:group_config, :rejoin_delay_seconds], :an_atom) assert BrodClient.init(opts) == {:error, "expected :rejoin_delay_seconds to be a non-negative integer, got: :an_atom"} opts = put_in(@opts, [:group_config, :rejoin_delay_seconds], 3) {:ok, [], %{group_config: group_config}} = BrodClient.init(opts) assert group_config[:rejoin_delay_seconds] == 3 end test ":session_timeout_seconds is an optional positive integer" do opts = put_in(@opts, [:group_config, :session_timeout_seconds], :an_atom) assert BrodClient.init(opts) == {:error, "expected :session_timeout_seconds to be a positive integer, got: :an_atom"} opts = put_in(@opts, [:group_config, :session_timeout_seconds], 3) {:ok, [], %{group_config: group_config}} = BrodClient.init(opts) assert group_config[:session_timeout_seconds] == 3 end test ":heartbeat_rate_seconds is an optional positive integer" do opts = put_in(@opts, [:group_config, :heartbeat_rate_seconds], :an_atom) assert BrodClient.init(opts) == {:error, "expected :heartbeat_rate_seconds to be a positive integer, got: :an_atom"} opts = put_in(@opts, [:group_config, :heartbeat_rate_seconds], 3) {:ok, [], %{group_config: group_config}} = BrodClient.init(opts) assert group_config[:heartbeat_rate_seconds] == 3 end test ":rebalance_timeout_seconds is an optional positive integer" do opts = put_in(@opts, [:group_config, :rebalance_timeout_seconds], :an_atom) assert BrodClient.init(opts) == {:error, "expected :rebalance_timeout_seconds to be a positive integer, got: :an_atom"} opts = put_in(@opts, [:group_config, :rebalance_timeout_seconds], 3) {:ok, [], %{group_config: group_config}} = BrodClient.init(opts) assert group_config[:rebalance_timeout_seconds] == 3 end test ":min_bytes is an optional positive integer" do opts = put_in(@opts, [:fetch_config, :min_bytes], :an_atom) assert BrodClient.init(opts) == {:error, "expected :min_bytes to be a positive integer, got: :an_atom"} opts = put_in(@opts, [:fetch_config, :min_bytes], 3) {:ok, [], %{fetch_config: fetch_config}} = BrodClient.init(opts) assert fetch_config[:min_bytes] == 3 end test ":max_bytes is optional non-negative integer" do opts = put_in(@opts, [:fetch_config, :max_bytes], :an_atom) assert BrodClient.init(opts) == {:error, "expected :max_bytes to be a positive integer, got: :an_atom"} opts = put_in(@opts, [:fetch_config, :max_bytes], 3) {:ok, [], %{fetch_config: fetch_config}} = BrodClient.init(opts) assert fetch_config[:max_bytes] == 3 end test ":max_wait_time is optional non-negative integer" do opts = put_in(@opts, [:fetch_config, :max_wait_time], :an_atom) assert BrodClient.init(opts) == {:error, "expected :max_wait_time to be a positive integer, got: :an_atom"} {:ok, [], %{fetch_config: fetch_config}} = BrodClient.init(@opts) assert not Map.has_key?(fetch_config, :max_wait_time) opts = put_in(@opts, [:fetch_config, :max_wait_time], 3) {:ok, [], %{fetch_config: fetch_config}} = BrodClient.init(opts) assert fetch_config[:max_wait_time] == 3 end test ":client_id_prefix is an optional atom value" do opts = put_in(@opts, [:client_config, :client_id_prefix], :wrong_type) assert BrodClient.init(opts) == {:error, "expected :client_id_prefix to be a string, got: :wrong_type"} opts = put_in(@opts, [:client_config, :client_id_prefix], "a string") assert {:ok, [], %{ client_config: [ client_id_prefix: "a string" ] }} = BrodClient.init(opts) end test ":sasl is an optional tuple of SASL mechanism, username and password" do opts = put_in(@opts, [:client_config, :sasl], :an_atom) assert BrodClient.init(opts) == {:error, "expected :sasl to be a tuple of SASL mechanism, username and password, or mechanism and path, got: :an_atom"} opts = put_in(@opts, [:client_config, :sasl], {:an_atom, "username", "password"}) assert BrodClient.init(opts) == {:error, "expected :sasl to be a tuple of SASL mechanism, username and password, or mechanism and path, got: {:an_atom, \"username\", \"password\"}"} opts = put_in(@opts, [:client_config, :sasl], {:plain, "username", "password"}) assert {:ok, [], %{ client_config: [ sasl: {:plain, "username", "password"} ] }} = BrodClient.init(opts) opts = put_in(@opts, [:client_config, :sasl], {:plain, "filepath"}) assert {:ok, [], %{ client_config: [ sasl: {:plain, "filepath"} ] }} = BrodClient.init(opts) end test ":sasl is an optional tuple of :callback, SASL Authentication Plugin module and opts" do opts = put_in(@opts, [:client_config, :sasl], {:callback, FakeSaslMechanismPlugin, {}}) assert {:ok, [], %{ client_config: [ sasl: {:callback, FakeSaslMechanismPlugin, {}} ] }} = BrodClient.init(opts) end test ":ssl is an optional boolean or keyword list" do opts = put_in(@opts, [:client_config, :ssl], :an_atom) assert BrodClient.init(opts) == {:error, "expected :ssl to be a keyword list of SSL/TLS client options, got: :an_atom"} opts = put_in(@opts, [:client_config, :ssl], cacertfile: "ca.crt", keyfile: "client.key", certfile: "client.crt" ) assert {:ok, [], %{ client_config: [ ssl: [cacertfile: "ca.crt", keyfile: "client.key", certfile: "client.crt"] ] }} = BrodClient.init(opts) opts = put_in(@opts, [:client_config, :ssl], true) assert {:ok, [], %{ client_config: [ssl: true] }} = BrodClient.init(opts) end test ":connect_timeout is an optional positive integer" do opts = put_in(@opts, [:client_config, :connect_timeout], "5000") assert BrodClient.init(opts) == {:error, "expected :connect_timeout to be a positive integer, got: \"5000\""} opts = put_in(@opts, [:client_config, :connect_timeout], 5000) assert {:ok, [], %{ client_config: [ connect_timeout: 5000 ] }} = BrodClient.init(opts) end test ":request_timeout is an optional positive integer >= 1000" do opts = put_in(@opts, [:client_config, :request_timeout], "5000") assert BrodClient.init(opts) == {:error, "expected :request_timeout to be a positive integer >= 1000, got: \"5000\""} opts = put_in(@opts, [:client_config, :request_timeout], 300) assert BrodClient.init(opts) == {:error, "expected :request_timeout to be a positive integer >= 1000, got: 300"} opts = put_in(@opts, [:client_config, :request_timeout], 5000) assert {:ok, [], %{ client_config: [ request_timeout: 5000 ] }} = BrodClient.init(opts) end test ":query_api_versions is an optional boolean" do opts = put_in(@opts, [:client_config, :query_api_versions], "true") assert BrodClient.init(opts) == {:error, "expected :query_api_versions to be a boolean, got: \"true\""} opts = put_in(@opts, [:client_config, :query_api_versions], false) assert {:ok, [], %{client_config: [query_api_versions: false]}} = BrodClient.init(opts) end test ":allow_topic_auto_creation is an optional positive boolean" do opts = put_in(@opts, [:client_config, :allow_topic_auto_creation], "false") assert BrodClient.init(opts) == {:error, "expected :allow_topic_auto_creation to be a boolean, got: \"false\""} opts = put_in(@opts, [:client_config, :allow_topic_auto_creation], false) assert {:ok, [], %{ client_config: [ allow_topic_auto_creation: false ] }} = BrodClient.init(opts) end test ":shared_client is an optional boolean" do opts = Keyword.put(@opts, :shared_client, "true") assert BrodClient.init(opts) == {:error, "expected :shared_client to be a boolean, got: \"true\""} opts = @opts |> Keyword.put(:shared_client, true) |> Keyword.put(:broadway, name: :my_broadway_name) |> put_in([:client_config, :client_id_prefix], "my_prefix.") assert {:ok, _specs, %{shared_client: true}} = BrodClient.init(opts) end test "return shared_client_id when :shared_client is true" do opts = @opts |> Keyword.put(:shared_client, true) |> Keyword.put(:broadway, name: :my_broadway_name) |> put_in([:client_config, :client_id_prefix], "my_prefix.") assert {:ok, child_specs, %{ shared_client: true, shared_client_id: :"my_prefix.Elixir.my_broadway_name.SharedClient" }} = BrodClient.init(opts) assert [ %{ id: shared_client_id, start: {:brod, :start_link_client, [hosts, shared_client_id, client_config]} } ] = child_specs assert [{:host, 9092}] = hosts assert :"my_prefix.Elixir.my_broadway_name.SharedClient" = shared_client_id assert [client_id_prefix: "my_prefix."] = client_config opts = @opts |> Keyword.put(:shared_client, false) |> Keyword.put(:broadway, name: :my_broadway_name) |> put_in([:client_config, :client_id_prefix], "my_prefix.") assert {:ok, [], %{ shared_client: false, shared_client_id: nil }} = BrodClient.init(opts) end end defmodule FakeSaslMechanismPlugin do @behaviour :kpro_auth_backend @impl true def auth(_host, _sock, _mod, _client_id, _timeout, _sasl_opts = {}) do :ok end end end ================================================ FILE: test/integration/consume_test.exs ================================================ defmodule BroadwayKafka.ConsumerTest.Config do def n_messages do 10_000 end def last_messages do (n_messages() - 2)..n_messages() |> Enum.map(&to_string/1) end end defmodule BroadwayKafka.ConsumerTest do @moduledoc """ Kafka integration tests. # Setup 1. Run Docker $ docker compose up -d # Running only integration tests mix test --only integration # Running all tests mix test --include integration """ use ExUnit.Case require Logger @moduletag :integration alias BroadwayKafka.ConsumerTest.Config defmodule MyBroadway do use Broadway alias BroadwayKafka.ConsumerTest.Config def start_link(context) do Broadway.start_link(__MODULE__, name: __MODULE__, context: context, producer: [ module: {BroadwayKafka.Producer, [ hosts: [localhost: 9092], group_id: "brod_my_group", topics: ["test"], receive_interval: 100, group_config: [ offset_commit_interval_seconds: 1, rejoin_delay_seconds: 2 ], fetch_config: [ max_bytes: 10_240 ] ]}, concurrency: 3 ], processors: [ default: [ concurrency: 3 ] ], batchers: [ default: [ batch_size: 20, batch_timeout: 50, concurrency: 4 ] ] ) end def handle_message(_, message, %{caller_pid: caller_pid}) do if message.data in Config.last_messages() do send(caller_pid, {:last_message, message.metadata.partition, message.data}) end message end def handle_batch(_, messages, _info, %{messages_agent: messages_agent}) do Agent.update(messages_agent, fn list -> list ++ messages end) last_message = List.last(messages) last_offset = last_message.metadata.offset partition = last_message.metadata.partition IO.puts( "Batch handled with #{length(messages)} messages. " <> "Partition: #{partition} Last offset: #{last_offset}" ) messages end end setup_all do topic = "test" hosts = [localhost: 9092] reset_topic(topic) {broadway_pid, messages_agent} = start_broadway() # Let's wait for the assignments before start sending messages wait_for_assignments(MyBroadway) IO.puts("Sending messages...") send_messages(Config.n_messages(), hosts, topic) [last_message_2, last_message_0, last_message_1] = Config.last_messages() receive do {:last_message, 2, ^last_message_2} -> IO.puts("Got last message from partition 2") end receive do {:last_message, 0, ^last_message_0} -> IO.puts("Got last message from partition 0") end receive do {:last_message, 1, ^last_message_1} -> IO.puts("Got last message from partition 1") end # Let's wait a bit to see if we get more messages Process.sleep(2000) messages = Agent.get(messages_agent, & &1) on_exit(fn -> stop_broadway(broadway_pid) end) {:ok, %{broadway_pid: broadway_pid, messages: messages}} end test "number of processed messages = total messages ", %{messages: messages} do assert length(messages) == Config.n_messages() end test "messages are not duplicated", %{messages: messages} do messages_with_count = Enum.reduce(messages, %{}, fn msg, acc -> Map.update(acc, msg.data, %{count: 1, list: [msg]}, fn %{count: count, list: list} -> %{list: [msg | list], count: count + 1} end) end) duplicated_messages = Enum.filter(messages_with_count, fn {_k, v} -> v.count > 1 end) assert duplicated_messages == [] end test "order of messages and offsets", %{messages: messages} do assert get_ordering_proplems(messages) == [] end defp reset_topic(topic) do brokers = [{"localhost", 9092}] :brod.delete_topics(brokers, [topic], 1_000) topic_config = [ %{ num_partitions: 3, replication_factor: 1, name: topic, assignments: [], configs: [] } ] wait_until_create_topic(brokers, topic_config, %{timeout: 1_000}) end defp wait_until_create_topic(brokers, topic_config, opts) do case :brod.create_topics(brokers, topic_config, opts) do :ok -> :ok _error -> Process.sleep(10) wait_until_create_topic(brokers, topic_config, opts) end end defp send_messages(n_messages, hosts, topic) do client_id = :test_client :ok = :brod.start_client(hosts, client_id, _client_config = []) :ok = :brod.start_producer(client_id, topic, _producer_config = []) Enum.each(1..n_messages, fn i -> partition = rem(i, 3) :ok = :brod.produce_sync(client_id, topic, partition, _key = "", "#{i}") end) :brod.stop_client(client_id) end defp start_broadway() do {:ok, messages_agent} = Agent.start_link(fn -> [] end) context = %{messages_agent: messages_agent, caller_pid: self()} {:ok, broadway_pid} = MyBroadway.start_link(context) {broadway_pid, messages_agent} end defp stop_broadway(pid) do ref = Process.monitor(pid) Process.exit(pid, :normal) receive do {:DOWN, ^ref, _, _, _} -> :ok end end defp get_ordering_proplems(messages) do init_acc = %{last_messages: %{0 => nil, 1 => nil, 2 => nil}, problems: []} %{problems: ordering_problems} = Enum.reduce(messages, init_acc, fn msg, acc -> %{last_messages: last_messages, problems: problems} = acc partition = msg.metadata.partition problems = case last_messages[partition] do nil -> problems last_message -> if String.to_integer(msg.data) <= String.to_integer(last_message.data) do message = "Data out of order #{msg.data}->#{last_message.data} in partition #{partition}" [message | problems] else problems end end last_messages = Map.put(last_messages, partition, msg) %{acc | problems: Enum.reverse(problems), last_messages: last_messages} end) Enum.reverse(ordering_problems) end defp wait_for_assignments(broadway_name) do producers = broadway_name |> Broadway.producer_names() |> Enum.map(fn producer -> pid = Process.whereis(producer) :erlang.trace(pid, true, [:receive, tracer: self()]) pid end) Enum.each(producers, fn pid -> receive do {:trace, ^pid, :receive, {:put_assignments, _, _}} -> IO.puts("Assignment received. Producer: #{inspect(pid)}") end end) end end defmodule BroadwayKafka.ConsumerSharedClientTest do @moduledoc """ Kafka integration tests. # Setup 1. Run Docker $ docker compose up -d # Running only integration tests mix test --only integration # Running all tests mix test --include integration """ use ExUnit.Case require Logger @moduletag :integration alias BroadwayKafka.ConsumerTest.Config defmodule MyBroadway do use Broadway alias BroadwayKafka.ConsumerTest.Config def start_link(context) do Broadway.start_link(__MODULE__, name: __MODULE__, context: context, producer: [ module: {BroadwayKafka.Producer, [ hosts: [localhost: 9092], group_id: "brod_my_group_2", topics: ["test_2"], receive_interval: 100, group_config: [ offset_commit_interval_seconds: 1, rejoin_delay_seconds: 2 ], fetch_config: [ max_bytes: 10_240 ], shared_client: true ]}, concurrency: 3 ], processors: [ default: [ concurrency: 3 ] ], batchers: [ default: [ batch_size: 20, batch_timeout: 50, concurrency: 4 ] ] ) end def handle_message(_, message, %{caller_pid: caller_pid}) do if message.data in Config.last_messages() do send(caller_pid, {:last_message, message.metadata.partition, message.data}) end message end def handle_batch(_, messages, _info, %{messages_agent: messages_agent}) do Agent.update(messages_agent, fn list -> list ++ messages end) last_message = List.last(messages) last_offset = last_message.metadata.offset partition = last_message.metadata.partition IO.puts( "Batch handled with #{length(messages)} messages. " <> "Partition: #{partition} Last offset: #{last_offset}" ) messages end end setup_all do topic = "test_2" hosts = [localhost: 9092] reset_topic(topic) {broadway_pid, messages_agent} = start_broadway() # Let's wait for the assignments before start sending messages wait_for_assignments(MyBroadway) IO.puts("Sending messages...") send_messages(Config.n_messages(), hosts, topic) [last_message_2, last_message_0, last_message_1] = Config.last_messages() receive do {:last_message, 2, ^last_message_2} -> IO.puts("Got last message from partition 2") end receive do {:last_message, 0, ^last_message_0} -> IO.puts("Got last message from partition 0") end receive do {:last_message, 1, ^last_message_1} -> IO.puts("Got last message from partition 1") end # Let's wait a bit to see if we get more messages Process.sleep(2000) messages = Agent.get(messages_agent, & &1) on_exit(fn -> stop_broadway(broadway_pid) end) {:ok, %{broadway_pid: broadway_pid, messages: messages}} end test "number of processed messages = total messages ", %{messages: messages} do assert length(messages) == Config.n_messages() end test "messages are not duplicated", %{messages: messages} do messages_with_count = Enum.reduce(messages, %{}, fn msg, acc -> Map.update(acc, msg.data, %{count: 1, list: [msg]}, fn %{count: count, list: list} -> %{list: [msg | list], count: count + 1} end) end) duplicated_messages = Enum.filter(messages_with_count, fn {_k, v} -> v.count > 1 end) assert duplicated_messages == [] end test "order of messages and offsets", %{messages: messages} do assert get_ordering_proplems(messages) == [] end defp reset_topic(topic) do brokers = [{"localhost", 9092}] :brod.delete_topics(brokers, [topic], 1_000) topic_config = [ %{ num_partitions: 3, replication_factor: 1, name: topic, assignments: [], configs: [] } ] wait_until_create_topic(brokers, topic_config, %{timeout: 1_000}) end defp wait_until_create_topic(brokers, topic_config, opts) do case :brod.create_topics(brokers, topic_config, opts) do :ok -> :ok _error -> Process.sleep(10) wait_until_create_topic(brokers, topic_config, opts) end end defp send_messages(n_messages, hosts, topic) do client_id = :test_client :ok = :brod.start_client(hosts, client_id, _client_config = []) :ok = :brod.start_producer(client_id, topic, _producer_config = []) Enum.each(1..n_messages, fn i -> partition = rem(i, 3) :ok = :brod.produce_sync(client_id, topic, partition, _key = "", "#{i}") end) :brod.stop_client(client_id) end defp start_broadway() do {:ok, messages_agent} = Agent.start_link(fn -> [] end) context = %{messages_agent: messages_agent, caller_pid: self()} {:ok, broadway_pid} = MyBroadway.start_link(context) {broadway_pid, messages_agent} end defp stop_broadway(pid) do ref = Process.monitor(pid) Process.exit(pid, :normal) receive do {:DOWN, ^ref, _, _, _} -> :ok end end defp get_ordering_proplems(messages) do init_acc = %{last_messages: %{0 => nil, 1 => nil, 2 => nil}, problems: []} %{problems: ordering_problems} = Enum.reduce(messages, init_acc, fn msg, acc -> %{last_messages: last_messages, problems: problems} = acc partition = msg.metadata.partition problems = case last_messages[partition] do nil -> problems last_message -> if String.to_integer(msg.data) <= String.to_integer(last_message.data) do message = "Data out of order #{msg.data}->#{last_message.data} in partition #{partition}" [message | problems] else problems end end last_messages = Map.put(last_messages, partition, msg) %{acc | problems: Enum.reverse(problems), last_messages: last_messages} end) Enum.reverse(ordering_problems) end defp wait_for_assignments(broadway_name) do producers = broadway_name |> Broadway.producer_names() |> Enum.map(fn producer -> pid = Process.whereis(producer) :erlang.trace(pid, true, [:receive, tracer: self()]) pid end) Enum.each(producers, fn pid -> receive do {:trace, ^pid, :receive, {:put_assignments, _, _}} -> IO.puts("Assignment received. Producer: #{inspect(pid)}") end end) end end ================================================ FILE: test/producer_test.exs ================================================ defmodule BroadwayKafka.ProducerTest do use ExUnit.Case import ExUnit.CaptureLog import Record, only: [defrecordp: 2, extract: 2] defrecordp :brod_received_assignment, extract(:brod_received_assignment, from_lib: "brod/include/brod.hrl") defmodule MessageServer do def start_link() do Agent.start_link(fn -> %{} end) end def push_messages(server, messages, opts) do topic = Keyword.fetch!(opts, :topic) partition = Keyword.fetch!(opts, :partition) key = key(topic, partition) Agent.update(server, fn queue -> Map.put(queue, key, (queue[key] || []) ++ messages) end) end def take_messages(server, topic, partition, amount) do key = key(topic, partition) Agent.get_and_update(server, fn queue -> {messages, rest} = Enum.split(queue[key] || [], amount) {messages, Map.put(queue, key, rest)} end) end defp key(topic, partition) do "#{topic}-#{partition}" end end defmodule FakeKafkaClient do @behaviour BroadwayKafka.KafkaClient import Record, only: [defrecordp: 2, extract: 2] defrecordp :kafka_message, extract(:kafka_message, from_lib: "brod/include/brod.hrl") @impl true def init(opts), do: {:ok, opts[:child_specs], Map.new(opts)} @impl true def setup(_stage_pid, client_id, _callback_module, config) do if !Process.whereis(client_id) do {:ok, _pid} = Agent.start(fn -> Map.put(config, :connected, true) end, name: client_id) Process.monitor(client_id) end send(config[:test_pid], {:setup, client_id}) {pid, ref} = spawn_monitor(fn -> Process.sleep(:infinity) end) {:ok, pid, ref} end @impl true def fetch(_client_id, topic, partition, offset, _opts, config) do n_messages = config[:max_bytes] messages = MessageServer.take_messages(config[:message_server], topic, partition, n_messages) send(config[:test_pid], {:messages_fetched, length(messages)}) kafka_messages = for {msg, i} <- Enum.with_index(messages, offset) do kafka_message( value: msg, offset: i, key: :fake_key, ts: :fake_ts, headers: :fake_headers ) end {:ok, {offset + length(kafka_messages), kafka_messages}} end @impl true def ack(_group_coordinator, _generation_id, topic, partition, offset, config) do info = %{offset: offset, topic: topic, partition: partition, pid: self()} ack_raises_on_offset = config[:ack_raises_on_offset] if ack_raises_on_offset && ack_raises_on_offset == offset do raise "Ack failed on offset #{offset}" end send(config[:test_pid], {:ack, info}) end @impl true def connected?(client_id) do connected? = if pid = Process.whereis(client_id) do Process.alive?(pid) && Agent.get(client_id, fn config -> config.connected end) end connected? end @impl true def disconnect(client_id) do test_pid = Agent.get(client_id, fn config -> config.test_pid end) send(test_pid, :disconnected) :ok end @impl true def resolve_offset(_topic, _partition, offset, _offset_reset_policy, _config) do offset end @impl true def update_topics(_client_id, _topics) do :ok end end defmodule Forwarder do use Broadway def handle_message(_, message, %{test_pid: test_pid}) do meta = message.metadata content = %{ data: message.data, topic: meta.topic, partition: meta.partition, offset: meta.offset, meta: meta, pid: self() } send(test_pid, {:message_handled, content}) message end def handle_batch(_, messages, batch_info, %{test_pid: test_pid}) do %{batch_key: {topic, partition}} = batch_info content = %{ topic: topic, partition: partition, offset: List.last(messages).metadata.offset, pid: self() } send(test_pid, {:batch_handled, content}) messages end end defmacro assert_receive_in_order({type, content} = pattern, opts) do offsets = Keyword.fetch!(opts, :offsets) timeout = Keyword.get(opts, :timeout, 200) quote do for offset <- unquote(offsets) do receive do {unquote(type), unquote(content) = received_message} -> assert received_message.offset == offset after unquote(timeout) -> raise "no message matching #{unquote(Macro.to_string(pattern))}" <> "after #{unquote(timeout)}ms" end end end end test "do not allow users to set :partition_by for processors" do Process.flag(:trap_exit, true) {:error, {%ArgumentError{message: message}, _}} = Broadway.start_link(Forwarder, name: new_unique_name(), producer: [module: {BroadwayKafka.Producer, []}], processors: [default: [partition_by: fn msg -> msg.data end]] ) assert message == "cannot set option :partition_by for processors :default. " <> "The option will be set automatically by BroadwayKafka.Producer" end test "do not allow users to set :partition_by for batchers" do Process.flag(:trap_exit, true) {:error, {%ArgumentError{message: message}, _}} = Broadway.start_link(Forwarder, name: new_unique_name(), producer: [module: {BroadwayKafka.Producer, []}], processors: [default: []], batchers: [default: [partition_by: fn msg -> msg.data end]] ) assert message == "cannot set option :partition_by for batchers :default. " <> "The option will be set automatically by BroadwayKafka.Producer" end test "append kafka metadata to message" do {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server) producer = get_producer(pid) put_assignments(producer, [[topic: "topic", partition: 0]]) MessageServer.push_messages(message_server, 1..1, topic: "topic", partition: 0) assert_receive {:message_handled, %{data: 1, meta: meta}} assert meta == %{ topic: "topic", partition: 0, offset: 1, key: :fake_key, ts: :fake_ts, headers: :fake_headers } stop_broadway(pid) end test "single producer receiving messages from a single topic/partition" do {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server) producer = get_producer(pid) put_assignments(producer, [[topic: "topic", partition: 0]]) MessageServer.push_messages(message_server, 1..5, topic: "topic", partition: 0) for msg <- 1..5 do assert_receive {:message_handled, %{data: ^msg, partition: 0}} end stop_broadway(pid) end test "start all child process returned by config" do {:ok, message_server} = MessageServer.start_link() parent_pid = self() child_specs = [ Supervisor.child_spec( {Task, fn -> send(parent_pid, :child_started_1) end}, id: :child_started_1 ), Supervisor.child_spec( {Task, fn -> send(parent_pid, :child_started_2) end}, id: :child_started_2 ) ] {:ok, pid} = start_broadway(message_server, shared_client: true, child_specs: child_specs) assert_receive :child_started_1 assert_receive :child_started_2 stop_broadway(pid) end test "should not disconnect client if shared_client true" do {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server, shared_client: false) stop_broadway(pid) assert_receive :disconnected {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server, shared_client: true) stop_broadway(pid) refute_receive :disconnected end test "single producer receiving messages from multiple topic/partitions" do {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server) producer = get_producer(pid) put_assignments(producer, [ [topic: "topic_1", partition: 0], [topic: "topic_1", partition: 1], [topic: "topic_2", partition: 0], [topic: "topic_2", partition: 1] ]) MessageServer.push_messages(message_server, 1..5, topic: "topic_1", partition: 0) MessageServer.push_messages(message_server, 6..10, topic: "topic_1", partition: 1) MessageServer.push_messages(message_server, 11..15, topic: "topic_2", partition: 0) MessageServer.push_messages(message_server, 16..20, topic: "topic_2", partition: 1) for msg <- 1..5 do assert_receive {:message_handled, %{data: ^msg}} end for msg <- 6..10 do assert_receive {:message_handled, %{data: ^msg}} end for msg <- 11..15 do assert_receive {:message_handled, %{data: ^msg}} end for msg <- 16..20 do assert_receive {:message_handled, %{data: ^msg}} end stop_broadway(pid) end test "fetch messages by chunks according to :max_bytes" do {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server) producer = get_producer(pid) put_assignments(producer, [[topic: "topic", partition: 0]]) MessageServer.push_messages(message_server, 1..25, topic: "topic", partition: 0) assert_receive {:messages_fetched, 10} assert_receive {:messages_fetched, 10} assert_receive {:messages_fetched, 5} stop_broadway(pid) end test "keep trying to receive new messages when the queue is empty" do {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server) producer = get_producer(pid) put_assignments(producer, [[topic: "topic", partition: 0]]) assert_receive {:messages_fetched, 0} MessageServer.push_messages(message_server, 1..10, topic: "topic", partition: 0) assert_receive {:messages_fetched, 10} assert_receive {:messages_fetched, 0} stop_broadway(pid) end test "messages with the same topic/partition are processed in the same processor" do {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server, producers_concurrency: 2, processors_concurrency: 4) producer_1 = get_producer(pid, 0) producer_2 = get_producer(pid, 1) put_assignments(producer_1, [ [topic: "topic_1", partition: 0, begin_offset: 100], [topic: "topic_2", partition: 1, begin_offset: 400] ]) put_assignments(producer_2, [ [topic: "topic_1", partition: 1, begin_offset: 200], [topic: "topic_2", partition: 0, begin_offset: 300] ]) MessageServer.push_messages(message_server, 1..10, topic: "topic_1", partition: 0) MessageServer.push_messages(message_server, 1..10, topic: "topic_1", partition: 1) MessageServer.push_messages(message_server, 1..10, topic: "topic_2", partition: 0) MessageServer.push_messages(message_server, 1..10, topic: "topic_2", partition: 1) assert_receive {:message_handled, %{topic: "topic_1", partition: 0, pid: processor_1}} assert_receive {:message_handled, %{topic: "topic_1", partition: 1, pid: processor_2}} assert_receive {:message_handled, %{topic: "topic_2", partition: 0, pid: processor_3}} assert_receive {:message_handled, %{topic: "topic_2", partition: 1, pid: processor_4}} processors = Enum.uniq([processor_1, processor_2, processor_3, processor_4]) assert length(processors) == 4 assert_receive_in_order( {:message_handled, %{topic: "topic_1", partition: 0, pid: ^processor_1}}, offsets: 101..109 ) assert_receive_in_order( {:message_handled, %{topic: "topic_1", partition: 1, pid: ^processor_2}}, offsets: 201..209 ) assert_receive_in_order( {:message_handled, %{topic: "topic_2", partition: 0, pid: ^processor_3}}, offsets: 301..309 ) assert_receive_in_order( {:message_handled, %{topic: "topic_2", partition: 1, pid: ^processor_4}}, offsets: 401..409 ) stop_broadway(pid) end test "batches with the same topic/partition are processed in the same batch consumer" do {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server, producers_concurrency: 2, processors_concurrency: 4, batchers_concurrency: 4 ) producer_1 = get_producer(pid, 0) producer_2 = get_producer(pid, 1) put_assignments(producer_1, [ [topic: "topic_1", partition: 0, begin_offset: 101], [topic: "topic_2", partition: 1, begin_offset: 401] ]) put_assignments(producer_2, [ [topic: "topic_1", partition: 1, begin_offset: 201], [topic: "topic_2", partition: 0, begin_offset: 301] ]) MessageServer.push_messages(message_server, 1..50, topic: "topic_1", partition: 0, offset: 110 ) MessageServer.push_messages(message_server, 1..50, topic: "topic_1", partition: 1, offset: 210 ) MessageServer.push_messages(message_server, 1..50, topic: "topic_2", partition: 0, offset: 310 ) MessageServer.push_messages(message_server, 1..50, topic: "topic_2", partition: 1, offset: 410 ) assert_receive {:batch_handled, %{topic: "topic_1", partition: 0, pid: consumer_1}} assert_receive {:batch_handled, %{topic: "topic_1", partition: 1, pid: consumer_2}} assert_receive {:batch_handled, %{topic: "topic_2", partition: 0, pid: consumer_3}} assert_receive {:batch_handled, %{topic: "topic_2", partition: 1, pid: consumer_4}} consumers = Enum.uniq([consumer_1, consumer_2, consumer_3, consumer_4]) assert length(consumers) == 4 assert_receive_in_order( {:batch_handled, %{topic: "topic_1", partition: 0, pid: ^consumer_1}}, offsets: [120, 130, 140, 150] ) assert_receive_in_order( {:batch_handled, %{topic: "topic_1", partition: 1, pid: ^consumer_2}}, offsets: [220, 230, 240, 250] ) assert_receive_in_order( {:batch_handled, %{topic: "topic_2", partition: 0, pid: ^consumer_3}}, offsets: [320, 330, 340, 350] ) assert_receive_in_order( {:batch_handled, %{topic: "topic_2", partition: 1, pid: ^consumer_4}}, offsets: [420, 430, 440, 450] ) stop_broadway(pid) end test "messages from the same topic/partition are acknowledged in order" do {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server, producers_concurrency: 2, processors_concurrency: 4 ) producer_1 = get_producer(pid, 0) producer_2 = get_producer(pid, 1) put_assignments(producer_1, [ [topic: "topic_1", partition: 0, begin_offset: 101], [topic: "topic_2", partition: 1, begin_offset: 401] ]) put_assignments(producer_2, [ [topic: "topic_1", partition: 1, begin_offset: 201], [topic: "topic_2", partition: 0, begin_offset: 301] ]) MessageServer.push_messages(message_server, 1..20, topic: "topic_1", partition: 0) MessageServer.push_messages(message_server, 1..20, topic: "topic_1", partition: 1) MessageServer.push_messages(message_server, 1..20, topic: "topic_2", partition: 0) MessageServer.push_messages(message_server, 1..20, topic: "topic_2", partition: 1) assert_receive_in_order( {:ack, %{topic: "topic_1", partition: 0}}, offsets: [105, 110, 115, 120] ) assert_receive_in_order( {:ack, %{topic: "topic_1", partition: 1}}, offsets: [205, 210, 215, 220] ) assert_receive_in_order( {:ack, %{topic: "topic_2", partition: 0}}, offsets: [305, 310, 315, 320] ) assert_receive_in_order( {:ack, %{topic: "topic_2", partition: 1}}, offsets: [405, 410, 415, 420] ) stop_broadway(pid) end test "batches from the same topic/partition are acknowledged in order" do {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server, producers_concurrency: 2, processors_concurrency: 4, batchers_concurrency: 4 ) producer_1 = get_producer(pid, 0) producer_2 = get_producer(pid, 1) put_assignments(producer_1, [ [topic: "topic_1", partition: 0, begin_offset: 101], [topic: "topic_2", partition: 1, begin_offset: 401] ]) put_assignments(producer_2, [ [topic: "topic_1", partition: 1, begin_offset: 201], [topic: "topic_2", partition: 0, begin_offset: 301] ]) MessageServer.push_messages(message_server, 1..40, topic: "topic_1", partition: 0) MessageServer.push_messages(message_server, 1..40, topic: "topic_1", partition: 1) MessageServer.push_messages(message_server, 1..40, topic: "topic_2", partition: 0) MessageServer.push_messages(message_server, 1..40, topic: "topic_2", partition: 1) assert_receive_in_order( {:ack, %{topic: "topic_1", partition: 0}}, offsets: [110, 120, 130, 140] ) assert_receive_in_order( {:ack, %{topic: "topic_1", partition: 1}}, offsets: [210, 220, 230, 240] ) assert_receive_in_order( {:ack, %{topic: "topic_2", partition: 0}}, offsets: [310, 320, 330, 340] ) assert_receive_in_order( {:ack, %{topic: "topic_2", partition: 1}}, offsets: [410, 420, 430, 440] ) stop_broadway(pid) end test "continue fetching messages after rebalancing" do {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server) producer = get_producer(pid) put_assignments(producer, [[topic: "topic", partition: 0]]) assert_receive {:messages_fetched, 0} BroadwayKafka.Producer.assignments_revoked(producer) put_assignments(producer, [[topic: "topic", partition: 0]]) assert_receive {:messages_fetched, 0} assert_receive {:messages_fetched, 0} stop_broadway(pid) end test "stop trying to receive new messages after start draining" do {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server) producer = get_producer(pid) put_assignments(producer, [[topic: "topic", partition: 0]]) assert_receive {:messages_fetched, 0} :sys.suspend(producer) flush_messages_received() task = Task.async(fn -> Broadway.Topology.ProducerStage.drain(producer) end) :sys.resume(producer) Task.await(task) refute_receive {:messages_fetched, 0}, 10 stop_broadway(pid) end test "if connection is lost, reconnect when :brod client is ready again" do {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server) assert_receive {:setup, client_id} Process.exit(Process.whereis(client_id), :kill) refute_receive {:setup, _} {:ok, _} = Agent.start_link(fn -> %{test_pid: self(), connected: false} end, name: client_id) refute_receive {:setup, _} Agent.update(client_id, fn state -> Map.put(state, :connected, true) end) assert_receive {:setup, ^client_id} stop_broadway(pid) end test "keep the producer alive on ack errors and log the exception" do {:ok, message_server} = MessageServer.start_link() {:ok, pid} = start_broadway(message_server, ack_raises_on_offset: 4) producer = get_producer(pid) producer_pid = Process.whereis(producer) put_assignments(producer, [[topic: "topic", partition: 0]]) MessageServer.push_messages(message_server, 1..2, topic: "topic", partition: 0) assert_receive {:ack, %{topic: "topic", partition: 0, pid: ^producer_pid}} assert capture_log(fn -> MessageServer.push_messages(message_server, 3..4, topic: "topic", partition: 0) refute_receive {:ack, %{topic: "topic", partition: 0, pid: ^producer_pid}} end) =~ "(RuntimeError) Ack failed on offset" MessageServer.push_messages(message_server, 5..6, topic: "topic", partition: 0) assert_receive {:ack, %{topic: "topic", partition: 0, pid: ^producer_pid}} stop_broadway(pid) end defp start_broadway(message_server, opts \\ []) do producers_concurrency = Keyword.get(opts, :producers_concurrency, 1) processors_concurrency = Keyword.get(opts, :processors_concurrency, 1) batchers_concurrency = Keyword.get(opts, :batchers_concurrency) ack_raises_on_offset = Keyword.get(opts, :ack_raises_on_offset, nil) batchers = if batchers_concurrency do [default: [concurrency: batchers_concurrency, batch_size: 10, batch_timeout: 10]] else [] end {:ok, pid} = Broadway.start_link(Forwarder, name: new_unique_name(), context: %{test_pid: self()}, producer: [ module: {BroadwayKafka.Producer, [ client: FakeKafkaClient, hosts: [], test_pid: self(), message_server: message_server, receive_interval: 0, reconnect_timeout: 10, max_bytes: 10, offset_commit_on_ack: false, begin_offset: :assigned, ack_raises_on_offset: ack_raises_on_offset, shared_client: opts[:shared_client] || false, child_specs: opts[:child_specs] || [] ]}, concurrency: producers_concurrency ], processors: [ default: [concurrency: processors_concurrency] ], batchers: batchers ) {:ok, pid} end defp put_assignments(producer, assignments) do group_member_id = System.unique_integer([:positive]) group_generation_id = System.unique_integer([:positive]) kafka_assignments = for assignment <- assignments do begin_offset = Keyword.get(assignment, :begin_offset, 1) brod_received_assignment( topic: assignment[:topic], partition: assignment[:partition], begin_offset: begin_offset ) end BroadwayKafka.Producer.assignments_received( producer, group_member_id, group_generation_id, kafka_assignments ) end defp new_unique_name() do :"Broadway#{System.unique_integer([:positive, :monotonic])}" end defp get_producer(broadway, index \\ 0) do {_, name} = Process.info(broadway, :registered_name) :"#{name}.Broadway.Producer_#{index}" end defp stop_broadway(pid) do ref = Process.monitor(pid) Process.exit(pid, :normal) receive do {:DOWN, ^ref, _, _, _} -> :ok end end defp flush_messages_received() do receive do {:messages_fetched, 0} -> flush_messages_received() after 0 -> :ok end end end ================================================ FILE: test/test_helper.exs ================================================ ExUnit.start(exclude: [:integration])