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