diff --git a/config/config.exs b/config/config.exs index 14d170b7..58b4ce58 100644 --- a/config/config.exs +++ b/config/config.exs @@ -16,7 +16,7 @@ config :kafka_ex, brokers: [ {"localhost", 9092}, {"localhost", 9093}, - {"localhost", 9094}, + {"localhost", 9094} ], # # OR: @@ -57,15 +57,16 @@ config :kafka_ex, # see SSL OPTION DESCRIPTIONS - CLIENT SIDE at http://erlang.org/doc/man/ssl.html # for supported options ssl_options: [ - cacertfile: System.cwd <> "/ssl/ca-cert", - certfile: System.cwd <> "/ssl/cert.pem", - keyfile: System.cwd <> "/ssl/key.pem", + cacertfile: System.cwd() <> "/ssl/ca-cert", + certfile: System.cwd() <> "/ssl/cert.pem", + keyfile: System.cwd() <> "/ssl/key.pem" ], # set this to the version of the kafka broker that you are using # include only major.minor.patch versions. must be at least 0.8.0 kafka_version: "0.10.1" -env_config = Path.expand("#{Mix.env}.exs", __DIR__) +env_config = Path.expand("#{Mix.env()}.exs", __DIR__) + if File.exists?(env_config) do import_config(env_config) end diff --git a/lib/kafka_ex.ex b/lib/kafka_ex.ex index ab5c7a2a..9f97fbc1 100644 --- a/lib/kafka_ex.ex +++ b/lib/kafka_ex.ex @@ -28,17 +28,20 @@ defmodule KafkaEx do alias KafkaEx.Server alias KafkaEx.Stream - @type uri() :: [{binary|[char], number}] + @type uri() :: [{binary | [char], number}] @type worker_init :: [worker_setting] - @type ssl_options :: [{:cacertfile, binary} | - {:certfile, binary} | - {:keyfile, binary} | - {:password, binary}] - @type worker_setting :: {:uris, uri} | - {:consumer_group, binary | :no_consumer_group} | - {:metadata_update_interval, non_neg_integer} | - {:consumer_group_update_interval, non_neg_integer} | - {:ssl_options, ssl_options} + @type ssl_options :: [ + {:cacertfile, binary} + | {:certfile, binary} + | {:keyfile, binary} + | {:password, binary} + ] + @type worker_setting :: + {:uris, uri} + | {:consumer_group, binary | :no_consumer_group} + | {:metadata_update_interval, non_neg_integer} + | {:consumer_group_update_interval, non_neg_integer} + | {:ssl_options, ssl_options} @doc """ create_worker creates KafkaEx workers @@ -66,11 +69,13 @@ defmodule KafkaEx do {:error, :invalid_consumer_group} ``` """ - @spec create_worker(atom, KafkaEx.worker_init) :: Supervisor.on_start_child + @spec create_worker(atom, KafkaEx.worker_init()) :: + Supervisor.on_start_child() def create_worker(name, worker_init \\ []) do case build_worker_options(worker_init) do {:ok, worker_init} -> KafkaEx.Supervisor.start_child([worker_init, name]) + {:error, error} -> {:error, error} end @@ -81,8 +86,10 @@ defmodule KafkaEx do Returns `:ok` on success or `:error` if `worker` is not a valid worker """ - @spec stop_worker(atom | pid) :: :ok | - {:error, :not_found} | {:error, :simple_one_for_one} + @spec stop_worker(atom | pid) :: + :ok + | {:error, :not_found} + | {:error, :simple_one_for_one} def stop_worker(worker) do KafkaEx.Supervisor.stop_child(worker) end @@ -93,16 +100,16 @@ defmodule KafkaEx do Worker may be an atom or pid. The default worker is used by default. """ @spec consumer_group(atom | pid) :: binary | :no_consumer_group - def consumer_group(worker \\ Config.default_worker) do + def consumer_group(worker \\ Config.default_worker()) do Server.call(worker, :consumer_group) end @doc """ Sends a request to join a consumer group. """ - @spec join_group(JoinGroupRequest.t, Keyword.t) :: JoinGroupResponse.t + @spec join_group(JoinGroupRequest.t(), Keyword.t()) :: JoinGroupResponse.t() def join_group(request, opts \\ []) do - worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + worker_name = Keyword.get(opts, :worker_name, Config.default_worker()) timeout = Keyword.get(opts, :timeout) Server.call(worker_name, {:join_group, request, timeout}, opts) end @@ -110,9 +117,9 @@ defmodule KafkaEx do @doc """ Sends a request to synchronize with a consumer group. """ - @spec sync_group(SyncGroupRequest.t, Keyword.t) :: SyncGroupResponse.t + @spec sync_group(SyncGroupRequest.t(), Keyword.t()) :: SyncGroupResponse.t() def sync_group(request, opts \\ []) do - worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + worker_name = Keyword.get(opts, :worker_name, Config.default_worker()) timeout = Keyword.get(opts, :timeout) Server.call(worker_name, {:sync_group, request, timeout}, opts) end @@ -120,9 +127,10 @@ defmodule KafkaEx do @doc """ Sends a request to leave a consumer group. """ - @spec leave_group(LeaveGroupRequest.t, Keyword.t) :: LeaveGroupResponse.t + @spec leave_group(LeaveGroupRequest.t(), Keyword.t()) :: + LeaveGroupResponse.t() def leave_group(request, opts \\ []) do - worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + worker_name = Keyword.get(opts, :worker_name, Config.default_worker()) timeout = Keyword.get(opts, :timeout) Server.call(worker_name, {:leave_group, request, timeout}, opts) end @@ -130,9 +138,9 @@ defmodule KafkaEx do @doc """ Sends a heartbeat to maintain membership in a consumer group. """ - @spec heartbeat(HeartbeatRequest.t, Keyword.t) :: HeartbeatResponse.t + @spec heartbeat(HeartbeatRequest.t(), Keyword.t()) :: HeartbeatResponse.t() def heartbeat(request, opts \\ []) do - worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + worker_name = Keyword.get(opts, :worker_name, Config.default_worker()) timeout = Keyword.get(opts, :timeout) Server.call(worker_name, {:heartbeat, request, timeout}, opts) end @@ -157,16 +165,19 @@ defmodule KafkaEx do topic: "foo"}]} ``` """ - @spec metadata(Keyword.t) :: MetadataResponse.t + @spec metadata(Keyword.t()) :: MetadataResponse.t() def metadata(opts \\ []) do - worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + worker_name = Keyword.get(opts, :worker_name, Config.default_worker()) topic = Keyword.get(opts, :topic, "") Server.call(worker_name, {:metadata, topic}, opts) end - @spec consumer_group_metadata(atom, binary) :: ConsumerMetadataResponse.t + @spec consumer_group_metadata(atom, binary) :: ConsumerMetadataResponse.t() def consumer_group_metadata(worker_name, supplied_consumer_group) do - Server.call(worker_name, {:consumer_group_metadata, supplied_consumer_group}) + Server.call( + worker_name, + {:consumer_group_metadata, supplied_consumer_group} + ) end @doc """ @@ -179,8 +190,10 @@ defmodule KafkaEx do [%KafkaEx.Protocol.Offset.Response{partition_offsets: [%{error_code: 0, offsets: [16], partition: 0}], topic: "foo"}] ``` """ - @spec latest_offset(binary, integer, atom|pid) :: [OffsetResponse.t] | :topic_not_found - def latest_offset(topic, partition, name \\ Config.default_worker), do: offset(topic, partition, :latest, name) + @spec latest_offset(binary, integer, atom | pid) :: + [OffsetResponse.t()] | :topic_not_found + def latest_offset(topic, partition, name \\ Config.default_worker()), + do: offset(topic, partition, :latest, name) @doc """ Get the offset of the earliest message still persistent in Kafka @@ -192,8 +205,10 @@ defmodule KafkaEx do [%KafkaEx.Protocol.Offset.Response{partition_offsets: [%{error_code: 0, offset: [0], partition: 0}], topic: "foo"}] ``` """ - @spec earliest_offset(binary, integer, atom|pid) :: [OffsetResponse.t] | :topic_not_found - def earliest_offset(topic, partition, name \\ Config.default_worker), do: offset(topic, partition, :earliest, name) + @spec earliest_offset(binary, integer, atom | pid) :: + [OffsetResponse.t()] | :topic_not_found + def earliest_offset(topic, partition, name \\ Config.default_worker()), + do: offset(topic, partition, :earliest, name) @doc """ Get the offset of the message sent at the specified date/time @@ -205,8 +220,13 @@ defmodule KafkaEx do [%KafkaEx.Protocol.Offset.Response{partition_offsets: [%{error_code: 0, offset: [256], partition: 0}], topic: "foo"}] ``` """ - @spec offset(binary, number, :calendar.datetime | :earliest | :latest, atom|pid) :: [OffsetResponse.t] | :topic_not_found - def offset(topic, partition, time, name \\ Config.default_worker) do + @spec offset( + binary, + number, + :calendar.datetime() | :earliest | :latest, + atom | pid + ) :: [OffsetResponse.t()] | :topic_not_found + def offset(topic, partition, time, name \\ Config.default_worker()) do Server.call(name, {:offset, topic, partition, time}) end @@ -238,33 +258,44 @@ defmodule KafkaEx do ] ``` """ - @spec fetch(binary, number, Keyword.t) :: [FetchResponse.t] | :topic_not_found + @spec fetch(binary, number, Keyword.t()) :: + [FetchResponse.t()] | :topic_not_found def fetch(topic, partition, opts \\ []) do - worker_name = Keyword.get(opts, :worker_name, Config.default_worker) - supplied_offset = Keyword.get(opts, :offset) - wait_time = Keyword.get(opts, :wait_time, @wait_time) - min_bytes = Keyword.get(opts, :min_bytes, @min_bytes) - max_bytes = Keyword.get(opts, :max_bytes, @max_bytes) - auto_commit = Keyword.get(opts, :auto_commit, true) - - retrieved_offset = current_offset(supplied_offset, partition, topic, worker_name) - - Server.call(worker_name, {:fetch, - %FetchRequest{ - auto_commit: auto_commit, - topic: topic, partition: partition, - offset: retrieved_offset, wait_time: wait_time, - min_bytes: min_bytes, max_bytes: max_bytes - } - }, opts) + worker_name = Keyword.get(opts, :worker_name, Config.default_worker()) + supplied_offset = Keyword.get(opts, :offset) + wait_time = Keyword.get(opts, :wait_time, @wait_time) + min_bytes = Keyword.get(opts, :min_bytes, @min_bytes) + max_bytes = Keyword.get(opts, :max_bytes, @max_bytes) + auto_commit = Keyword.get(opts, :auto_commit, true) + + retrieved_offset = + current_offset(supplied_offset, partition, topic, worker_name) + + Server.call( + worker_name, + {:fetch, + %FetchRequest{ + auto_commit: auto_commit, + topic: topic, + partition: partition, + offset: retrieved_offset, + wait_time: wait_time, + min_bytes: min_bytes, + max_bytes: max_bytes + }}, + opts + ) end - @spec offset_commit(atom, OffsetCommitRequest.t) :: [OffsetCommitResponse.t] + @spec offset_commit(atom, OffsetCommitRequest.t()) :: [ + OffsetCommitResponse.t() + ] def offset_commit(worker_name, offset_commit_request) do Server.call(worker_name, {:offset_commit, offset_commit_request}) end - @spec offset_fetch(atom, OffsetFetchRequest.t) :: [OffsetFetchResponse.t] | :topic_not_found + @spec offset_fetch(atom, OffsetFetchRequest.t()) :: + [OffsetFetchResponse.t()] | :topic_not_found def offset_fetch(worker_name, offset_fetch_request) do Server.call(worker_name, {:offset_fetch, offset_fetch_request}) end @@ -284,9 +315,17 @@ defmodule KafkaEx do {:ok, 9773} ``` """ - @spec produce(ProduceRequest.t, Keyword.t) :: nil | :ok | {:ok, integer} | {:error, :closed} | {:error, :inet.posix} | {:error, any} | iodata | :leader_not_available + @spec produce(ProduceRequest.t(), Keyword.t()) :: + nil + | :ok + | {:ok, integer} + | {:error, :closed} + | {:error, :inet.posix()} + | {:error, any} + | iodata + | :leader_not_available def produce(produce_request, opts \\ []) do - worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + worker_name = Keyword.get(opts, :worker_name, Config.default_worker()) Server.call(worker_name, {:produce, produce_request}, opts) end @@ -308,12 +347,28 @@ defmodule KafkaEx do {:ok, 9771} ``` """ - @spec produce(binary, number, binary, Keyword.t) :: nil | :ok | {:ok, integer} | {:error, :closed} | {:error, :inet.posix} | {:error, any} | iodata | :leader_not_available + @spec produce(binary, number, binary, Keyword.t()) :: + nil + | :ok + | {:ok, integer} + | {:error, :closed} + | {:error, :inet.posix()} + | {:error, any} + | iodata + | :leader_not_available def produce(topic, partition, value, opts \\ []) do - key = Keyword.get(opts, :key, "") - required_acks = Keyword.get(opts, :required_acks, 0) - timeout = Keyword.get(opts, :timeout, 100) - produce_request = %ProduceRequest{topic: topic, partition: partition, required_acks: required_acks, timeout: timeout, compression: :none, messages: [%Message{key: key, value: value}]} + key = Keyword.get(opts, :key, "") + required_acks = Keyword.get(opts, :required_acks, 0) + timeout = Keyword.get(opts, :timeout, 100) + + produce_request = %ProduceRequest{ + topic: topic, + partition: partition, + required_acks: required_acks, + timeout: timeout, + compression: :none, + messages: [%Message{key: key, value: value}] + } produce(produce_request, opts) end @@ -433,16 +488,16 @@ defmodule KafkaEx do - `auto_commit` (boolean): If true, the stream automatically commits offsets of fetched messages. See discussion above. """ - @spec stream(binary, integer, Keyword.t) :: KafkaEx.Stream.t + @spec stream(binary, integer, Keyword.t()) :: KafkaEx.Stream.t() def stream(topic, partition, opts \\ []) do - auto_commit = Keyword.get(opts, :auto_commit, true) - consumer_group = Keyword.get(opts, :consumer_group) - max_bytes = Keyword.get(opts, :max_bytes, @max_bytes) - min_bytes = Keyword.get(opts, :min_bytes, @min_bytes) - supplied_offset = Keyword.get(opts, :offset) - worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + auto_commit = Keyword.get(opts, :auto_commit, true) + consumer_group = Keyword.get(opts, :consumer_group) + max_bytes = Keyword.get(opts, :max_bytes, @max_bytes) + min_bytes = Keyword.get(opts, :min_bytes, @min_bytes) + supplied_offset = Keyword.get(opts, :offset) + worker_name = Keyword.get(opts, :worker_name, Config.default_worker()) no_wait_at_logend = Keyword.get(opts, :no_wait_at_logend, false) - wait_time = Keyword.get(opts, :wait_time, @wait_time) + wait_time = Keyword.get(opts, :wait_time, @wait_time) retrieved_offset = if consumer_group && !supplied_offset do @@ -452,25 +507,31 @@ defmodule KafkaEx do consumer_group: consumer_group } - fetched_offset = worker_name - |> KafkaEx.offset_fetch(request) - |> KafkaEx.Protocol.OffsetFetch.Response.last_offset + fetched_offset = + worker_name + |> KafkaEx.offset_fetch(request) + |> KafkaEx.Protocol.OffsetFetch.Response.last_offset() fetched_offset + 1 else current_offset(supplied_offset, partition, topic, worker_name) end - fetch_request = %FetchRequest{ + fetch_request = %FetchRequest{ auto_commit: auto_commit, - topic: topic, partition: partition, - offset: retrieved_offset, wait_time: wait_time, - min_bytes: min_bytes, max_bytes: max_bytes + topic: topic, + partition: partition, + offset: retrieved_offset, + wait_time: wait_time, + min_bytes: min_bytes, + max_bytes: max_bytes } %Stream{ - worker_name: worker_name, fetch_request: fetch_request, - consumer_group: consumer_group, no_wait_at_logend: no_wait_at_logend + worker_name: worker_name, + fetch_request: fetch_request, + consumer_group: consumer_group, + no_wait_at_logend: no_wait_at_logend } end @@ -484,18 +545,19 @@ defmodule KafkaEx do Note this happens automatically when using `KafkaEx.create_worker`. """ @spec build_worker_options(worker_init) :: - {:ok, worker_init} | {:error, :invalid_consumer_group} + {:ok, worker_init} | {:error, :invalid_consumer_group} def build_worker_options(worker_init) do defaults = [ uris: Config.brokers(), consumer_group: Config.consumer_group(), use_ssl: Config.use_ssl(), - ssl_options: Config.ssl_options(), + ssl_options: Config.ssl_options() ] worker_init = Keyword.merge(defaults, worker_init) supplied_consumer_group = Keyword.get(worker_init, :consumer_group) + if valid_consumer_group?(supplied_consumer_group) do {:ok, worker_init} else @@ -506,20 +568,25 @@ defmodule KafkaEx do defp current_offset(supplied_offset, partition, topic, worker_name) do case supplied_offset do nil -> - last_offset = worker_name - |> offset_fetch(%OffsetFetchRequest{topic: topic, partition: partition}) - |> OffsetFetchResponse.last_offset + last_offset = + worker_name + |> offset_fetch(%OffsetFetchRequest{ + topic: topic, + partition: partition + }) + |> OffsetFetchResponse.last_offset() if last_offset < 0 do topic |> earliest_offset(partition, worker_name) - |> OffsetResponse.extract_offset + |> OffsetResponse.extract_offset() else last_offset + 1 end - _ -> supplied_offset - end + _ -> + supplied_offset + end end @doc """ @@ -533,36 +600,42 @@ defmodule KafkaEx do @doc """ Retrieve supported api versions for each api key. """ - @spec api_versions(Keyword.t) :: ApiVersionsResponse.t + @spec api_versions(Keyword.t()) :: ApiVersionsResponse.t() def api_versions(opts \\ []) do - worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + worker_name = Keyword.get(opts, :worker_name, Config.default_worker()) Server.call(worker_name, {:api_versions}) end - @doc """ Create topics. Must provide a list of CreateTopicsRequest, each containing all the information needed for the creation of a new topic. """ - @spec create_topics([CreateTopicsRequest.t], Keyword.t) :: CreateTopicsResponse.t + @spec create_topics([CreateTopicsRequest.t()], Keyword.t()) :: + CreateTopicsResponse.t() def create_topics(requests, opts \\ []) do - worker_name = Keyword.get(opts, :worker_name, Config.default_worker) + worker_name = Keyword.get(opts, :worker_name, Config.default_worker()) timeout = Keyword.get(opts, :timeout, 4000) Server.call(worker_name, {:create_topics, requests, timeout}) end -#OTP API + # OTP API def start(_type, _args) do max_restarts = Application.get_env(:kafka_ex, :max_restarts, 10) max_seconds = Application.get_env(:kafka_ex, :max_seconds, 60) - {:ok, pid} = KafkaEx.Supervisor.start_link(Config.server_impl, max_restarts, max_seconds) - if Config.disable_default_worker do + {:ok, pid} = + KafkaEx.Supervisor.start_link( + Config.server_impl(), + max_restarts, + max_seconds + ) + + if Config.disable_default_worker() do {:ok, pid} else - case KafkaEx.create_worker(Config.default_worker, []) do + case KafkaEx.create_worker(Config.default_worker(), []) do {:error, reason} -> {:error, reason} - {:ok, _} -> {:ok, pid} + {:ok, _} -> {:ok, pid} end end end diff --git a/lib/kafka_ex/api_versions.ex b/lib/kafka_ex/api_versions.ex index 2328eb3a..b3ac3da0 100644 --- a/lib/kafka_ex/api_versions.ex +++ b/lib/kafka_ex/api_versions.ex @@ -22,10 +22,17 @@ defmodule KafkaEx.ApiVersions do api_key -> case api_versions_map[api_key] do - %{min_version: min} when min > max_implemented_version -> :no_version_supported - %{max_version: max} when max < min_implemented_version -> :no_version_supported - %{max_version: max} -> {:ok, Enum.min([max_implemented_version, max])} - _ -> :unknown_message_for_server + %{min_version: min} when min > max_implemented_version -> + :no_version_supported + + %{max_version: max} when max < min_implemented_version -> + :no_version_supported + + %{max_version: max} -> + {:ok, Enum.min([max_implemented_version, max])} + + _ -> + :unknown_message_for_server end end end diff --git a/lib/kafka_ex/compression.ex b/lib/kafka_ex/compression.ex index 0dc50c70..fa816afa 100644 --- a/lib/kafka_ex/compression.ex +++ b/lib/kafka_ex/compression.ex @@ -30,7 +30,7 @@ defmodule KafkaEx.Compression do end def decompress(@snappy_attribute, data) do - << _snappy_header :: 64, _snappy_version_info :: 64, rest :: binary>> = data + <<_snappy_header::64, _snappy_version_info::64, rest::binary>> = data snappy_decompress_chunk(rest, <<>>) end @@ -44,6 +44,7 @@ defmodule KafkaEx.Compression do {:ok, compressed_data} = :snappy.compress(data) {compressed_data, @snappy_attribute} end + def compress(:gzip, data) do compressed_data = :zlib.gzip(data) {compressed_data, @gzip_attribute} @@ -52,9 +53,11 @@ defmodule KafkaEx.Compression do def snappy_decompress_chunk(<<>>, so_far) do so_far end - def snappy_decompress_chunk(<< valsize :: 32-unsigned, - value :: size(valsize)-binary, - rest :: binary>>, so_far) do + + def snappy_decompress_chunk( + <>, + so_far + ) do {:ok, decompressed_value} = :snappy.decompress(value) snappy_decompress_chunk(rest, so_far <> decompressed_value) end diff --git a/lib/kafka_ex/config.ex b/lib/kafka_ex/config.ex index 1b5ba86d..18c8857a 100644 --- a/lib/kafka_ex/config.ex +++ b/lib/kafka_ex/config.ex @@ -1,11 +1,13 @@ defmodule KafkaEx.Config do @moduledoc """ - Configuring KafkaEx + Configuring KafkaEx - ``` - """ <> File.read!(Path.expand("../../config/config.exs", __DIR__)) <> """ - ``` - """ + ``` + """ <> + File.read!(Path.expand("../../config/config.exs", __DIR__)) <> + """ + ``` + """ require Logger @@ -37,21 +39,23 @@ defmodule KafkaEx.Config do @doc false def server_impl do :kafka_ex - |> Application.get_env(:kafka_version, :default) - |> server + |> Application.get_env(:kafka_version, :default) + |> server end @doc false def brokers do :kafka_ex - |> Application.get_env(:brokers) - |> brokers() + |> Application.get_env(:brokers) + |> brokers() end defp brokers(nil), do: nil + defp brokers(list) when is_list(list), do: list + defp brokers(csv) when is_binary(csv) do for line <- String.split(csv, ","), into: [] do case line |> trim() |> String.split(":") do @@ -59,20 +63,23 @@ defmodule KafkaEx.Config do msg = "Port not set for kafka broker #{host}" Logger.warn(msg) raise msg + [host, port] -> {port, _} = Integer.parse(port) {host, port} end end end + defp brokers({mod, fun, args}) when is_atom(mod) and is_atom(fun) do apply(mod, fun, args) end + defp brokers(fun) when is_function(fun, 0) do fun.() end - if Version.match?(System.version, "<1.3.0") do + if Version.match?(System.version(), "<1.3.0") do defp trim(string), do: String.strip(string) else defp trim(string), do: String.trim(string) @@ -83,18 +90,21 @@ defmodule KafkaEx.Config do defp server("0.9.0"), do: KafkaEx.Server0P9P0 defp server(_), do: KafkaEx.Server0P10AndLater - # ssl_options should be an empty list by default if use_ssl is false defp ssl_options(false, []), do: [] # emit a warning if use_ssl is false but options are present # (this is not a fatal error and can occur if one disables ssl in the # default option set) defp ssl_options(false, options) do - Logger.warn("Ignoring ssl_options #{inspect options} because " <> - "use_ssl is false. If you do not intend to use ssl and want to " <> - "remove this warning, set `ssl_options: []` in the KafkaEx config.") + Logger.warn( + "Ignoring ssl_options #{inspect(options)} because " <> + "use_ssl is false. If you do not intend to use ssl and want to " <> + "remove this warning, set `ssl_options: []` in the KafkaEx config." + ) + [] end + # verify that options is at least a keyword list defp ssl_options(true, options) do if Keyword.keyword?(options) do diff --git a/lib/kafka_ex/consumer_group.ex b/lib/kafka_ex/consumer_group.ex index f679b329..b2b318ca 100644 --- a/lib/kafka_ex/consumer_group.ex +++ b/lib/kafka_ex/consumer_group.ex @@ -102,14 +102,15 @@ defmodule KafkaEx.ConsumerGroup do and 30000 by default). See [https://kafka.apache.org/documentation/#configuration](https://kafka.apache.org/documentation/#configuration). """ - @type option :: KafkaEx.GenConsumer.option - | {:heartbeat_interval, pos_integer} - | {:session_timeout, pos_integer} - | {:partition_assignment_callback, PartitionAssignment.callback} - | {:gen_server_opts, GenServer.options} - | {:name, Supervisor.name} - | {:max_restarts, non_neg_integer} - | {:max_seconds, non_neg_integer} + @type option :: + KafkaEx.GenConsumer.option() + | {:heartbeat_interval, pos_integer} + | {:session_timeout, pos_integer} + | {:partition_assignment_callback, PartitionAssignment.callback()} + | {:gen_server_opts, GenServer.options()} + | {:name, Supervisor.name()} + | {:max_restarts, non_neg_integer} + | {:max_seconds, non_neg_integer} @type options :: [option] @@ -134,7 +135,7 @@ defmodule KafkaEx.ConsumerGroup do This function has the same return values as `Supervisor.start_link/3`. """ - @spec start_link(module, binary, [binary], options) :: Supervisor.on_start + @spec start_link(module, binary, [binary], options) :: Supervisor.on_start() def start_link(consumer_module, group_name, topics, opts \\ []) do {supervisor_opts, module_opts} = Keyword.split(opts, [:name, :strategy, :max_restarts, :max_seconds]) @@ -152,7 +153,7 @@ defmodule KafkaEx.ConsumerGroup do The generation id is provided by the broker on sync. Returns `nil` if queried before the initial sync has completed. """ - @spec generation_id(Supervisor.supervisor) :: integer | nil + @spec generation_id(Supervisor.supervisor()) :: integer | nil def generation_id(supervisor_pid) do call_manager(supervisor_pid, :generation_id) end @@ -163,7 +164,7 @@ defmodule KafkaEx.ConsumerGroup do The id is assigned by the broker. Returns `nil` if queried before the initial sync has completed. """ - @spec member_id(Supervisor.supervisor) :: binary | nil + @spec member_id(Supervisor.supervisor()) :: binary | nil def member_id(supervisor_pid) do call_manager(supervisor_pid, :member_id) end @@ -174,7 +175,7 @@ defmodule KafkaEx.ConsumerGroup do This is provided by the broker on sync. Returns `nil` if queried before the initial sync has completed """ - @spec leader_id(Supervisor.supervisor) :: binary | nil + @spec leader_id(Supervisor.supervisor()) :: binary | nil def leader_id(supervisor_pid) do call_manager(supervisor_pid, :leader_id) end @@ -185,7 +186,7 @@ defmodule KafkaEx.ConsumerGroup do Leaders are elected by the broker and are responsible for assigning partitions. Returns false if queried before the intiial sync has completed. """ - @spec leader?(Supervisor.supervisor) :: boolean + @spec leader?(Supervisor.supervisor()) :: boolean def leader?(supervisor_pid) do call_manager(supervisor_pid, :am_leader) end @@ -196,8 +197,9 @@ defmodule KafkaEx.ConsumerGroup do These are assigned by the leader and communicated by the broker on sync. """ - @spec assignments(Supervisor.supervisor) :: - [{topic :: binary, partition_id :: non_neg_integer}] + @spec assignments(Supervisor.supervisor()) :: [ + {topic :: binary, partition_id :: non_neg_integer} + ] def assignments(supervisor_pid) do call_manager(supervisor_pid, :assignments) end @@ -208,7 +210,7 @@ defmodule KafkaEx.ConsumerGroup do Returns `nil` if called before the initial sync. """ - @spec consumer_supervisor_pid(Supervisor.supervisor) :: nil | pid + @spec consumer_supervisor_pid(Supervisor.supervisor()) :: nil | pid def consumer_supervisor_pid(supervisor_pid) do call_manager(supervisor_pid, :consumer_supervisor_pid) end @@ -216,17 +218,17 @@ defmodule KafkaEx.ConsumerGroup do @doc """ Returns the pids of consumer processes """ - @spec consumer_pids(Supervisor.supervisor) :: [pid] + @spec consumer_pids(Supervisor.supervisor()) :: [pid] def consumer_pids(supervisor_pid) do supervisor_pid |> consumer_supervisor_pid - |> GenConsumer.Supervisor.child_pids + |> GenConsumer.Supervisor.child_pids() end @doc """ Returns the name of the consumer group """ - @spec group_name(Supervisor.supervisor) :: binary + @spec group_name(Supervisor.supervisor()) :: binary def group_name(supervisor_pid) do call_manager(supervisor_pid, :group_name) end @@ -234,12 +236,13 @@ defmodule KafkaEx.ConsumerGroup do @doc """ Returns a map from `{topic, partition_id}` to consumer pid """ - @spec partition_consumer_map(Supervisor.supervisor) :: - %{{topic :: binary, partition_id :: non_neg_integer} => pid} + @spec partition_consumer_map(Supervisor.supervisor()) :: %{ + {topic :: binary, partition_id :: non_neg_integer} => pid + } def partition_consumer_map(supervisor_pid) do supervisor_pid |> consumer_pids - |> Enum.map(fn(pid) -> + |> Enum.map(fn pid -> {GenConsumer.partition(pid), pid} end) |> Enum.into(%{}) @@ -248,9 +251,10 @@ defmodule KafkaEx.ConsumerGroup do @doc """ Returns true if at least one child consumer process is alive """ - @spec active?(Supervisor.supervisor) :: boolean + @spec active?(Supervisor.supervisor()) :: boolean def active?(supervisor_pid) do consumer_supervisor = consumer_supervisor_pid(supervisor_pid) + if consumer_supervisor && Process.alive?(consumer_supervisor) do GenConsumer.Supervisor.active?(consumer_supervisor) else @@ -264,25 +268,29 @@ defmodule KafkaEx.ConsumerGroup do Intended for introspection usage only. """ - @spec get_manager_pid(Supervisor.supervisor) :: pid + @spec get_manager_pid(Supervisor.supervisor()) :: pid def get_manager_pid(supervisor_pid) do - {_, pid, _, _} = Enum.find( - Supervisor.which_children(supervisor_pid), - fn - ({KafkaEx.ConsumerGroup.Manager, _, _, _}) -> true - ({_, _, _, _}) -> false - end - ) + {_, pid, _, _} = + Enum.find( + Supervisor.which_children(supervisor_pid), + fn + {KafkaEx.ConsumerGroup.Manager, _, _, _} -> true + {_, _, _, _} -> false + end + ) + pid end - @doc false # used by ConsumerGroup.Manager to set partition assignments + # used by ConsumerGroup.Manager to set partition assignments + @doc false def start_consumer(pid, consumer_module, group_name, assignments, opts) do - child = supervisor( - KafkaEx.GenConsumer.Supervisor, - [consumer_module, group_name, assignments, opts], - id: :consumer - ) + child = + supervisor( + KafkaEx.GenConsumer.Supervisor, + [consumer_module, group_name, assignments, opts], + id: :consumer + ) case Supervisor.start_child(pid, child) do {:ok, consumer_pid} -> {:ok, consumer_pid} @@ -290,7 +298,8 @@ defmodule KafkaEx.ConsumerGroup do end end - @doc false # used by ConsumerGroup to pause consumption during rebalance + # used by ConsumerGroup to pause consumption during rebalance + @doc false def stop_consumer(pid) do case Supervisor.terminate_child(pid, :consumer) do :ok -> @@ -309,7 +318,7 @@ defmodule KafkaEx.ConsumerGroup do worker( KafkaEx.ConsumerGroup.Manager, [consumer_module, group_name, topics, opts] - ), + ) ] supervise(children, strategy: :one_for_all) diff --git a/lib/kafka_ex/consumer_group/manager.ex b/lib/kafka_ex/consumer_group/manager.ex index c2810f92..fb2c3701 100644 --- a/lib/kafka_ex/consumer_group/manager.ex +++ b/lib/kafka_ex/consumer_group/manager.ex @@ -34,8 +34,9 @@ defmodule KafkaEx.ConsumerGroup.Manager do :members, :generation_id, :assignments, - :heartbeat_timer, + :heartbeat_timer ] + @type t :: %__MODULE__{} end @@ -49,8 +50,8 @@ defmodule KafkaEx.ConsumerGroup.Manager do @doc false # use `KafkaEx.ConsumerGroup.start_link/4` instead - @spec start_link(module, binary, [binary], KafkaEx.GenConsumer.options) :: - GenServer.on_start + @spec start_link(module, binary, [binary], KafkaEx.GenConsumer.options()) :: + GenServer.on_start() def start_link(consumer_module, group_name, topics, opts \\ []) do gen_server_opts = Keyword.get(opts, :gen_server_opts, []) consumer_opts = Keyword.drop(opts, [:gen_server_opts]) @@ -65,38 +66,47 @@ defmodule KafkaEx.ConsumerGroup.Manager do # GenServer callbacks def init({consumer_module, group_name, topics, opts}) do - heartbeat_interval = Keyword.get( - opts, - :heartbeat_interval, - Application.get_env(:kafka_ex, :heartbeat_interval, @heartbeat_interval) - ) - session_timeout = Keyword.get( - opts, - :session_timeout, - Application.get_env(:kafka_ex, :session_timeout, @session_timeout) - ) - partition_assignment_callback = Keyword.get( - opts, - :partition_assignment_callback, - &PartitionAssignment.round_robin/2 - ) - - supervisor_pid = Keyword.fetch!(opts, :supervisor_pid) - consumer_opts = Keyword.drop( - opts, - [ - :supervisor_pid, + heartbeat_interval = + Keyword.get( + opts, :heartbeat_interval, + Application.get_env(:kafka_ex, :heartbeat_interval, @heartbeat_interval) + ) + + session_timeout = + Keyword.get( + opts, :session_timeout, - :partition_assignment_callback - ] - ) + Application.get_env(:kafka_ex, :session_timeout, @session_timeout) + ) + + partition_assignment_callback = + Keyword.get( + opts, + :partition_assignment_callback, + &PartitionAssignment.round_robin/2 + ) + + supervisor_pid = Keyword.fetch!(opts, :supervisor_pid) + + consumer_opts = + Keyword.drop( + opts, + [ + :supervisor_pid, + :heartbeat_interval, + :session_timeout, + :partition_assignment_callback + ] + ) worker_opts = Keyword.take(opts, [:uris]) - {:ok, worker_name} = KafkaEx.create_worker( - :no_name, - [consumer_group: group_name] ++ worker_opts - ) + + {:ok, worker_name} = + KafkaEx.create_worker( + :no_name, + [consumer_group: group_name] ++ worker_opts + ) state = %State{ supervisor_pid: supervisor_pid, @@ -108,7 +118,7 @@ defmodule KafkaEx.ConsumerGroup.Manager do consumer_opts: consumer_opts, group_name: group_name, topics: topics, - member_id: nil, + member_id: nil } Process.flag(:trap_exit, true) @@ -145,21 +155,26 @@ defmodule KafkaEx.ConsumerGroup.Manager do def handle_call(:group_name, _from, state) do {:reply, state.group_name, state} end + ###################################################################### # If `member_id` and `generation_id` aren't set, we haven't yet joined the # group. `member_id` and `generation_id` are initialized by # `JoinGroupResponse`. def handle_info( - :timeout, %State{generation_id: nil, member_id: nil} = state - ) do + :timeout, + %State{generation_id: nil, member_id: nil} = state + ) do {:ok, new_state} = join(state) {:noreply, new_state} end # If the heartbeat gets an error, we need to rebalance. - def handle_info({:EXIT, heartbeat_timer, {:shutdown, :rebalance}}, %State{heartbeat_timer: heartbeat_timer} = state) do + def handle_info( + {:EXIT, heartbeat_timer, {:shutdown, :rebalance}}, + %State{heartbeat_timer: heartbeat_timer} = state + ) do {:ok, state} = rebalance(state) {:noreply, state} end @@ -168,6 +183,7 @@ defmodule KafkaEx.ConsumerGroup.Manager do # the group so that the group can rebalance without waiting for a session # timeout. def terminate(_reason, %State{generation_id: nil, member_id: nil}), do: :ok + def terminate(_reason, %State{} = state) do {:ok, _state} = leave(state) Process.unlink(state.worker_name) @@ -191,40 +207,42 @@ defmodule KafkaEx.ConsumerGroup.Manager do # group members. Once a `JoinGroupResponse` is received, all group members # must send a `SyncGroupRequest` (see sync/2). defp join( - %State{ - worker_name: worker_name, - session_timeout: session_timeout, - group_name: group_name, - topics: topics, - member_id: member_id - } = state - ) do + %State{ + worker_name: worker_name, + session_timeout: session_timeout, + group_name: group_name, + topics: topics, + member_id: member_id + } = state + ) do join_request = %JoinGroupRequest{ group_name: group_name, member_id: member_id || "", topics: topics, - session_timeout: session_timeout, + session_timeout: session_timeout } - join_response = KafkaEx.join_group( - join_request, - worker_name: worker_name, - timeout: session_timeout + @session_timeout_padding - ) + join_response = + KafkaEx.join_group( + join_request, + worker_name: worker_name, + timeout: session_timeout + @session_timeout_padding + ) # crash the worker if we recieve an error, but do it with a meaningful # error message if join_response.error_code != :no_error do - raise "Error joining consumer group #{group_name}: " <> "#{inspect join_response.error_code}" + raise "Error joining consumer group #{group_name}: " <> + "#{inspect(join_response.error_code)}" end Logger.debug(fn -> "Joined consumer group #{group_name}" end) new_state = %State{ - state | - leader_id: join_response.leader_id, - member_id: join_response.member_id, - generation_id: join_response.generation_id + state + | leader_id: join_response.leader_id, + member_id: join_response.member_id, + generation_id: join_response.generation_id } assignments = @@ -294,17 +312,17 @@ defmodule KafkaEx.ConsumerGroup.Manager do # session timeout to expire. Leaving a group triggers a rebalance for the # remaining group members. defp leave( - %State{ - worker_name: worker_name, - group_name: group_name, - member_id: member_id - } = state - ) do + %State{ + worker_name: worker_name, + group_name: group_name, + member_id: member_id + } = state + ) do stop_heartbeat_timer(state) leave_request = %LeaveGroupRequest{ group_name: group_name, - member_id: member_id, + member_id: member_id } leave_group_response = @@ -314,8 +332,8 @@ defmodule KafkaEx.ConsumerGroup.Manager do Logger.debug(fn -> "Left consumer group #{group_name}" end) else Logger.warn(fn -> - "Received error #{inspect leave_group_response.error_code}, " <> - "consumer group manager will exit regardless." + "Received error #{inspect(leave_group_response.error_code)}, " <> + "consumer group manager will exit regardless." end) end @@ -344,13 +362,14 @@ defmodule KafkaEx.ConsumerGroup.Manager do end # Stops the heartbeat process. - defp stop_heartbeat_timer(%State{heartbeat_timer: nil} = state), do: {:ok, state} - defp stop_heartbeat_timer( - %State{heartbeat_timer: heartbeat_timer} = state - ) do + defp stop_heartbeat_timer(%State{heartbeat_timer: nil} = state), + do: {:ok, state} + + defp stop_heartbeat_timer(%State{heartbeat_timer: heartbeat_timer} = state) do if Process.alive?(heartbeat_timer) do :gen_server.stop(heartbeat_timer) end + new_state = %State{state | heartbeat_timer: nil} {:ok, new_state} end @@ -359,27 +378,29 @@ defmodule KafkaEx.ConsumerGroup.Manager do # Starts consuming from the member's assigned partitions. defp start_consumer( - %State{ - consumer_module: consumer_module, - consumer_opts: consumer_opts, - group_name: group_name, - supervisor_pid: pid - } = state, - assignments - ) do - {:ok, consumer_supervisor_pid} = ConsumerGroup.start_consumer( - pid, - consumer_module, - group_name, - assignments, - consumer_opts - ) + %State{ + consumer_module: consumer_module, + consumer_opts: consumer_opts, + group_name: group_name, + supervisor_pid: pid + } = state, + assignments + ) do + {:ok, consumer_supervisor_pid} = + ConsumerGroup.start_consumer( + pid, + consumer_module, + group_name, + assignments, + consumer_opts + ) state = %{ - state | - assignments: assignments, - consumer_supervisor_pid: consumer_supervisor_pid + state + | assignments: assignments, + consumer_supervisor_pid: consumer_supervisor_pid } + {:ok, state} end @@ -396,17 +417,19 @@ defmodule KafkaEx.ConsumerGroup.Manager do # interest to this consumer group. This function returns a list of # topic/partition tuples that can be passed to a GenConsumer's # `assign_partitions` method. - defp assignable_partitions( - %State{worker_name: worker_name, topics: topics, group_name: group_name} - ) do + defp assignable_partitions(%State{ + worker_name: worker_name, + topics: topics, + group_name: group_name + }) do metadata = KafkaEx.metadata(worker_name: worker_name) - Enum.flat_map(topics, fn (topic) -> + Enum.flat_map(topics, fn topic -> partitions = MetadataResponse.partitions_for_topic(metadata, topic) warn_if_no_partitions(partitions, group_name, topic) - Enum.map(partitions, fn (partition) -> + Enum.map(partitions, fn partition -> {topic, partition} end) end) @@ -417,6 +440,7 @@ defmodule KafkaEx.ConsumerGroup.Manager do "Consumer group #{group_name} encountered nonexistent topic #{topic}" end) end + defp warn_if_no_partitions(_partitions, _group_name, _topic), do: :ok # This function is used by the group leader to determine partition @@ -426,22 +450,23 @@ defmodule KafkaEx.ConsumerGroup.Manager do # return value is a complete list of member assignments in the format needed # by `SyncGroupResponse`. defp assign_partitions( - %State{partition_assignment_callback: partition_assignment_callback}, - members, - partitions - ) do + %State{partition_assignment_callback: partition_assignment_callback}, + members, + partitions + ) do # Delegate partition assignment to GenConsumer module. assignments = partition_assignment_callback.(members, partitions) # Convert assignments to format expected by Kafka protocol. packed_assignments = - Enum.map(assignments, fn ({member, topic_partitions}) -> + Enum.map(assignments, fn {member, topic_partitions} -> {member, pack_assignments(topic_partitions)} end) + assignments_map = Enum.into(packed_assignments, %{}) # Fill in empty assignments for missing member IDs. - Enum.map(members, fn (member) -> + Enum.map(members, fn member -> {member, Map.get(assignments_map, member, [])} end) end @@ -452,8 +477,8 @@ defmodule KafkaEx.ConsumerGroup.Manager do # # unpack_assignments([{"foo", [0, 1]}]) #=> [{"foo", 0}, {"foo", 1}] defp unpack_assignments(assignments) do - Enum.flat_map(assignments, fn ({topic, partition_ids}) -> - Enum.map(partition_ids, &({topic, &1})) + Enum.flat_map(assignments, fn {topic, partition_ids} -> + Enum.map(partition_ids, &{topic, &1}) end) end @@ -464,9 +489,9 @@ defmodule KafkaEx.ConsumerGroup.Manager do # pack_assignments([{"foo", 0}, {"foo", 1}]) #=> [{"foo", [0, 1]}] defp pack_assignments(assignments) do assignments - |> Enum.reduce(%{}, fn({topic, partition}, assignments) -> + |> Enum.reduce(%{}, fn {topic, partition}, assignments -> Map.update(assignments, topic, [partition], &(&1 ++ [partition])) end) - |> Map.to_list + |> Map.to_list() end end diff --git a/lib/kafka_ex/consumer_group/partition_assignment.ex b/lib/kafka_ex/consumer_group/partition_assignment.ex index b082710a..1f269bae 100644 --- a/lib/kafka_ex/consumer_group/partition_assignment.ex +++ b/lib/kafka_ex/consumer_group/partition_assignment.ex @@ -64,8 +64,8 @@ defmodule KafkaEx.ConsumerGroup.PartitionAssignment do launch no processes. """ @type callback :: - ((members :: [member_id], partitions :: [partition]) -> - %{member_id => [partition]}) + (members :: [member_id], partitions :: [partition] -> + %{member_id => [partition]}) @doc """ Round robin assignment @@ -81,9 +81,9 @@ defmodule KafkaEx.ConsumerGroup.PartitionAssignment do @spec round_robin([binary], [partition]) :: %{binary => [partition]} def round_robin(members, partitions) do members - |> Stream.cycle + |> Stream.cycle() |> Enum.zip(partitions) - |> Enum.reduce(%{}, fn({member, partition}, assignments) -> + |> Enum.reduce(%{}, fn {member, partition}, assignments -> Map.update(assignments, member, [partition], &(&1 ++ [partition])) end) end diff --git a/lib/kafka_ex/exceptions.ex b/lib/kafka_ex/exceptions.ex index 25ae958e..e6745721 100644 --- a/lib/kafka_ex/exceptions.ex +++ b/lib/kafka_ex/exceptions.ex @@ -2,14 +2,18 @@ defmodule KafkaEx.ConsumerGroupRequiredError do defexception [:message] def exception(%{__struct__: struct}) do - message = "KafkaEx requests of type #{struct} " <> - "require that the worker is configured for a consumer group." + message = + "KafkaEx requests of type #{struct} " <> + "require that the worker is configured for a consumer group." + %__MODULE__{message: message} end def exception(action) when is_binary(action) do - message = "KafkaEx #{action} requires that the worker is configured " <> - "for a consumer group." + message = + "KafkaEx #{action} requires that the worker is configured " <> + "for a consumer group." + %__MODULE__{message: message} end end @@ -18,7 +22,7 @@ defmodule KafkaEx.InvalidConsumerGroupError do defexception [:message] def exception(consumer_group) do - message = "Invalid consumer group: #{inspect consumer_group}" + message = "Invalid consumer group: #{inspect(consumer_group)}" %__MODULE__{message: message} end end diff --git a/lib/kafka_ex/gen_consumer.ex b/lib/kafka_ex/gen_consumer.ex index 4c5d8b0f..6e04a2de 100644 --- a/lib/kafka_ex/gen_consumer.ex +++ b/lib/kafka_ex/gen_consumer.ex @@ -200,14 +200,15 @@ defmodule KafkaEx.GenConsumer do @typedoc """ Option values used when starting a `KafkaEx.GenConsumer`. """ - @type option :: {:commit_interval, non_neg_integer} - | {:commit_threshold, non_neg_integer} - | {:auto_offset_reset, :none | :earliest | :latest} + @type option :: + {:commit_interval, non_neg_integer} + | {:commit_threshold, non_neg_integer} + | {:auto_offset_reset, :none | :earliest | :latest} @typedoc """ Options used when starting a `KafkaEx.GenConsumer`. """ - @type options :: [option | GenServer.option] + @type options :: [option | GenServer.option()] @doc """ Invoked when the server is started. `start_link/5` will block until it @@ -224,7 +225,7 @@ defmodule KafkaEx.GenConsumer do error}` and the process to exit. """ @callback init(topic :: binary, partition :: non_neg_integer) :: - {:ok, state :: term} + {:ok, state :: term} @doc """ Invoked for each message set consumed from a Kafka topic partition. @@ -244,8 +245,9 @@ defmodule KafkaEx.GenConsumer do used sparingly, since committing every message synchronously would impact a consumer's performance and could result in excessive network traffic. """ - @callback handle_message_set(message_set :: [Message.t], state :: term) :: - {:async_commit, new_state :: term} | {:sync_commit, new_state :: term} + @callback handle_message_set(message_set :: [Message.t()], state :: term) :: + {:async_commit, new_state :: term} + | {:sync_commit, new_state :: term} @doc """ Invoked by `KafkaEx.GenConsumer.call/3`. @@ -253,8 +255,8 @@ defmodule KafkaEx.GenConsumer do Note the default implementation will cause a `RuntimeError`. If you want to interact with your consumer, you must implement a handle_call function. """ - @callback handle_call(call :: term, from :: GenServer.from, state :: term) - :: {:reply, reply_value :: term, new_state :: term} + @callback handle_call(call :: term, from :: GenServer.from(), state :: term) :: + {:reply, reply_value :: term, new_state :: term} @doc """ Invoked by `KafkaEx.GenConsumer.cast/2`. @@ -263,7 +265,7 @@ defmodule KafkaEx.GenConsumer do interact with your consumer, you must implement a handle_cast function. """ @callback handle_cast(cast :: term, state :: term) :: - {:noreply, new_state :: term} + {:noreply, new_state :: term} @doc """ Invoked by sending messages to the consumer. @@ -272,7 +274,7 @@ defmodule KafkaEx.GenConsumer do interact with your consumer, you must implement a handle_info function. """ @callback handle_info(info :: term, state :: term) :: - {:noreply, new_state :: term} + {:noreply, new_state :: term} defmacro __using__(_opts) do quote do @@ -401,12 +403,12 @@ defmodule KafkaEx.GenConsumer do This function has the same return values as `GenServer.start_link/3`. """ @spec start_link( - callback_module :: module, - consumer_group_name :: binary, - topic_name :: binary, - partition_id :: non_neg_integer, - options - ) :: GenServer.on_start + callback_module :: module, + consumer_group_name :: binary, + topic_name :: binary, + partition_id :: non_neg_integer, + options + ) :: GenServer.on_start() def start_link(consumer_module, group_name, topic, partition, opts \\ []) do {server_opts, consumer_opts} = Keyword.split(opts, [:debug, :name, :timeout, :spawn_opt]) @@ -421,8 +423,8 @@ defmodule KafkaEx.GenConsumer do @doc """ Returns the topic and partition id for this consumer process """ - @spec partition(GenServer.server) :: - {topic :: binary, partition_id :: non_neg_integer} + @spec partition(GenServer.server()) :: + {topic :: binary, partition_id :: non_neg_integer} def partition(gen_consumer) do GenServer.call(gen_consumer, :partition) end @@ -438,7 +440,7 @@ defmodule KafkaEx.GenConsumer do See the moduledoc for an example. """ - @spec call(GenServer.server, term, timeout) :: term + @spec call(GenServer.server(), term, timeout) :: term def call(gen_consumer, message, timeout \\ 5000) do GenServer.call(gen_consumer, {:consumer_call, message}, timeout) end @@ -451,7 +453,7 @@ defmodule KafkaEx.GenConsumer do form `{:noreply, new_consumer_state}`. The GenConsumer will turn this into an immediate timeout, which drives continued message consumption. """ - @spec cast(GenServer.server, term) :: term + @spec cast(GenServer.server(), term) :: term def cast(gen_consumer, message) do GenServer.cast(gen_consumer, {:consumer_cast, message}) end @@ -459,28 +461,35 @@ defmodule KafkaEx.GenConsumer do # GenServer callbacks def init({consumer_module, group_name, topic, partition, opts}) do - commit_interval = Keyword.get( - opts, - :commit_interval, - Application.get_env(:kafka_ex, :commit_interval, @commit_interval) - ) - commit_threshold = Keyword.get( - opts, - :commit_threshold, - Application.get_env(:kafka_ex, :commit_threshold, @commit_threshold) - ) - auto_offset_reset = Keyword.get( - opts, - :auto_offset_reset, - Application.get_env(:kafka_ex, :auto_offset_reset, @auto_offset_reset) - ) + commit_interval = + Keyword.get( + opts, + :commit_interval, + Application.get_env(:kafka_ex, :commit_interval, @commit_interval) + ) + + commit_threshold = + Keyword.get( + opts, + :commit_threshold, + Application.get_env(:kafka_ex, :commit_threshold, @commit_threshold) + ) + + auto_offset_reset = + Keyword.get( + opts, + :auto_offset_reset, + Application.get_env(:kafka_ex, :auto_offset_reset, @auto_offset_reset) + ) {:ok, consumer_state} = consumer_module.init(topic, partition) worker_opts = Keyword.take(opts, [:uris]) - {:ok, worker_name} = KafkaEx.create_worker( - :no_name, - [consumer_group: group_name] ++ worker_opts - ) + + {:ok, worker_name} = + KafkaEx.create_worker( + :no_name, + [consumer_group: group_name] ++ worker_opts + ) default_fetch_options = [ auto_commit: false, @@ -513,50 +522,54 @@ defmodule KafkaEx.GenConsumer do end def handle_call( - {:consumer_call, message}, - from, - %State{ - consumer_module: consumer_module, - consumer_state: consumer_state - } = state - ) do + {:consumer_call, message}, + from, + %State{ + consumer_module: consumer_module, + consumer_state: consumer_state + } = state + ) do # NOTE we only support the {:reply, _, _} result format here # which we turn into a timeout = 0 clause so that we continue to consume. # any other GenServer flow control could have unintended consequences, # so we leave that for later consideration - {:reply, reply, new_consumer_state} = consumer_module.handle_call( - message, - from, - consumer_state - ) + {:reply, reply, new_consumer_state} = + consumer_module.handle_call( + message, + from, + consumer_state + ) + {:reply, reply, %{state | consumer_state: new_consumer_state}, 0} end def handle_cast( - {:consumer_cast, message}, - %State{ - consumer_module: consumer_module, - consumer_state: consumer_state - } = state - ) do + {:consumer_cast, message}, + %State{ + consumer_module: consumer_module, + consumer_state: consumer_state + } = state + ) do # NOTE we only support the {:noreply, _} result format here # which we turn into a timeout = 0 clause so that we continue to consume. # any other GenServer flow control could have unintended consequences, # so we leave that for later consideration - {:noreply, new_consumer_state} = consumer_module.handle_cast( - message, - consumer_state - ) + {:noreply, new_consumer_state} = + consumer_module.handle_cast( + message, + consumer_state + ) + {:noreply, %{state | consumer_state: new_consumer_state}, 0} end def handle_info( - :timeout, - %State{current_offset: nil, last_commit: nil} = state - ) do + :timeout, + %State{current_offset: nil, last_commit: nil} = state + ) do new_state = %State{ - load_offsets(state) | - last_commit: :erlang.monotonic_time(:milli_seconds) + load_offsets(state) + | last_commit: :erlang.monotonic_time(:milli_seconds) } {:noreply, new_state, 0} @@ -569,20 +582,22 @@ defmodule KafkaEx.GenConsumer do end def handle_info( - message, - %State{ - consumer_module: consumer_module, - consumer_state: consumer_state - } = state - ) do + message, + %State{ + consumer_module: consumer_module, + consumer_state: consumer_state + } = state + ) do # NOTE we only support the {:noreply, _} result format here # which we turn into a timeout = 0 clause so that we continue to consume. # any other GenServer flow control could have unintended consequences, # so we leave that for later consideration - {:noreply, new_consumer_state} = consumer_module.handle_info( - message, - consumer_state - ) + {:noreply, new_consumer_state} = + consumer_module.handle_info( + message, + consumer_state + ) + {:noreply, %{state | consumer_state: new_consumer_state}, 0} end @@ -595,13 +610,13 @@ defmodule KafkaEx.GenConsumer do # Helpers defp consume( - %State{ - topic: topic, - partition: partition, - current_offset: offset, - fetch_options: fetch_options - } = state - ) do + %State{ + topic: topic, + partition: partition, + current_offset: offset, + fetch_options: fetch_options + } = state + ) do [ %FetchResponse{ topic: ^topic, @@ -609,16 +624,18 @@ defmodule KafkaEx.GenConsumer do response = %{error_code: error_code, partition: ^partition} ] } - ] = KafkaEx.fetch( - topic, - partition, - Keyword.merge(fetch_options, [offset: offset]) - ) + ] = + KafkaEx.fetch( + topic, + partition, + Keyword.merge(fetch_options, offset: offset) + ) state = case error_code do :offset_out_of_range -> handle_offset_out_of_range(state) + :no_error -> state end @@ -626,40 +643,42 @@ defmodule KafkaEx.GenConsumer do case response do %{last_offset: nil, message_set: []} -> handle_commit(:async_commit, state) + %{last_offset: _, message_set: message_set} -> handle_message_set(message_set, state) end end defp handle_message_set( - message_set, - %State{ - consumer_module: consumer_module, - consumer_state: consumer_state - } = state - ) do + message_set, + %State{ + consumer_module: consumer_module, + consumer_state: consumer_state + } = state + ) do {sync_status, new_consumer_state} = consumer_module.handle_message_set(message_set, consumer_state) %Message{offset: last_offset} = List.last(message_set) + state_out = %State{ - state | - consumer_state: new_consumer_state, - acked_offset: last_offset + 1, - current_offset: last_offset + 1 + state + | consumer_state: new_consumer_state, + acked_offset: last_offset + 1, + current_offset: last_offset + 1 } handle_commit(sync_status, state_out) end defp handle_offset_out_of_range( - %State{ - worker_name: worker_name, - topic: topic, - partition: partition, - auto_offset_reset: auto_offset_reset - } = state - ) do + %State{ + worker_name: worker_name, + topic: topic, + partition: partition, + auto_offset_reset: auto_offset_reset + } = state + ) do [ %OffsetResponse{ topic: ^topic, @@ -667,39 +686,47 @@ defmodule KafkaEx.GenConsumer do %{partition: ^partition, error_code: :no_error, offset: [offset]} ] } - ] = case auto_offset_reset do - :earliest -> - KafkaEx.earliest_offset(topic, partition, worker_name) - :latest -> - KafkaEx.latest_offset(topic, partition, worker_name) - _ -> - raise "Offset out of range while consuming topic #{topic}, partition #{partition}." - end + ] = + case auto_offset_reset do + :earliest -> + KafkaEx.earliest_offset(topic, partition, worker_name) + + :latest -> + KafkaEx.latest_offset(topic, partition, worker_name) + + _ -> + raise "Offset out of range while consuming topic #{topic}, partition #{ + partition + }." + end %State{ - state | - current_offset: offset, - committed_offset: offset, - acked_offset: offset + state + | current_offset: offset, + committed_offset: offset, + acked_offset: offset } end defp handle_commit(:sync_commit, %State{} = state), do: commit(state) + defp handle_commit( - :async_commit, - %State{ - acked_offset: acked, - committed_offset: committed, - commit_threshold: threshold, - last_commit: last_commit, - commit_interval: interval - } = state - ) do + :async_commit, + %State{ + acked_offset: acked, + committed_offset: committed, + commit_threshold: threshold, + last_commit: last_commit, + commit_interval: interval + } = state + ) do case acked - committed do 0 -> %State{state | last_commit: :erlang.monotonic_time(:milli_seconds)} + n when n >= threshold -> commit(state) + _ -> if :erlang.monotonic_time(:milli_seconds) - last_commit >= interval do commit(state) @@ -709,26 +736,24 @@ defmodule KafkaEx.GenConsumer do end end - defp commit( - %State{acked_offset: offset, committed_offset: offset} = state - ) do + defp commit(%State{acked_offset: offset, committed_offset: offset} = state) do state end defp commit( - %State{ - worker_name: worker_name, - group: group, - topic: topic, - partition: partition, - acked_offset: offset - } = state - ) do + %State{ + worker_name: worker_name, + group: group, + topic: topic, + partition: partition, + acked_offset: offset + } = state + ) do request = %OffsetCommitRequest{ consumer_group: group, topic: topic, partition: partition, - offset: offset, + offset: offset } [%OffsetCommitResponse{topic: ^topic, partitions: [^partition]}] = @@ -739,20 +764,20 @@ defmodule KafkaEx.GenConsumer do end) %State{ - state | - committed_offset: offset, - last_commit: :erlang.monotonic_time(:milli_seconds) + state + | committed_offset: offset, + last_commit: :erlang.monotonic_time(:milli_seconds) } end defp load_offsets( - %State{ - worker_name: worker_name, - group: group, - topic: topic, - partition: partition - } = state - ) do + %State{ + worker_name: worker_name, + group: group, + topic: topic, + partition: partition + } = state + ) do request = %OffsetFetchRequest{ consumer_group: group, topic: topic, @@ -771,11 +796,12 @@ defmodule KafkaEx.GenConsumer do case error_code do :no_error -> %State{ - state | - current_offset: offset, - committed_offset: offset, - acked_offset: offset + state + | current_offset: offset, + committed_offset: offset, + acked_offset: offset } + :unknown_topic_or_partition -> [ %OffsetResponse{ @@ -787,10 +813,10 @@ defmodule KafkaEx.GenConsumer do ] = KafkaEx.earliest_offset(topic, partition, worker_name) %State{ - state | - current_offset: offset, - committed_offset: offset, - acked_offset: offset + state + | current_offset: offset, + committed_offset: offset, + acked_offset: offset } end end diff --git a/lib/kafka_ex/gen_consumer/supervisor.ex b/lib/kafka_ex/gen_consumer/supervisor.ex index d89451d3..b4042d82 100644 --- a/lib/kafka_ex/gen_consumer/supervisor.ex +++ b/lib/kafka_ex/gen_consumer/supervisor.ex @@ -31,18 +31,20 @@ defmodule KafkaEx.GenConsumer.Supervisor do returns `{:ok, pid}`, where `pid` is the PID of the supervisor. """ @spec start_link( - callback_module :: module, - consumer_group_name :: binary, - assigned_partitions :: [ - {topic_name :: binary, partition_id :: non_neg_integer} - ], - KafkaEx.GenConsumer.options - ) :: Elixir.Supervisor.on_start + callback_module :: module, + consumer_group_name :: binary, + assigned_partitions :: [ + {topic_name :: binary, partition_id :: non_neg_integer} + ], + KafkaEx.GenConsumer.options() + ) :: Elixir.Supervisor.on_start() def start_link(consumer_module, group_name, assignments, opts \\ []) do - start_link_result = Elixir.Supervisor.start_link( - __MODULE__, - {consumer_module, group_name, assignments, opts} - ) + start_link_result = + Elixir.Supervisor.start_link( + __MODULE__, + {consumer_module, group_name, assignments, opts} + ) + case start_link_result do {:ok, pid} -> :ok = start_workers(pid, assignments, opts) @@ -61,14 +63,14 @@ defmodule KafkaEx.GenConsumer.Supervisor do @spec child_pids(pid | atom) :: [pid] def child_pids(supervisor_pid) do supervisor_pid - |> Supervisor.which_children - |> Enum.map(fn({_, pid, _, _}) -> pid end) + |> Supervisor.which_children() + |> Enum.map(fn {_, pid, _, _} -> pid end) end @doc """ Returns true if any child pids are alive """ - @spec active?(Supervisor.supervisor) :: boolean + @spec active?(Supervisor.supervisor()) :: boolean def active?(supervisor_pid) do supervisor_pid |> child_pids @@ -84,7 +86,7 @@ defmodule KafkaEx.GenConsumer.Supervisor do end defp start_workers(pid, assignments, opts) do - Enum.each(assignments, fn ({topic, partition}) -> + Enum.each(assignments, fn {topic, partition} -> case Elixir.Supervisor.start_child(pid, [topic, partition, opts]) do {:ok, _child} -> nil {:ok, _child, _info} -> nil diff --git a/lib/kafka_ex/network_client.ex b/lib/kafka_ex/network_client.ex index fa62c7c9..63b4a8e8 100644 --- a/lib/kafka_ex/network_client.ex +++ b/lib/kafka_ex/network_client.ex @@ -4,64 +4,118 @@ defmodule KafkaEx.NetworkClient do alias KafkaEx.Socket @moduledoc false - @spec create_socket(binary, non_neg_integer, KafkaEx.ssl_options, boolean) :: nil | Socket.t + @spec create_socket(binary, non_neg_integer, KafkaEx.ssl_options(), boolean) :: + nil | Socket.t() def create_socket(host, port, ssl_options \\ [], use_ssl \\ false) do - case Socket.create(format_host(host), port, build_socket_options(ssl_options), use_ssl) do + case Socket.create( + format_host(host), + port, + build_socket_options(ssl_options), + use_ssl + ) do {:ok, socket} -> - Logger.log(:debug, "Successfully connected to broker #{inspect(host)}:#{inspect port}") + Logger.log( + :debug, + "Successfully connected to broker #{inspect(host)}:#{inspect(port)}" + ) + socket - err -> - Logger.log(:error, "Could not connect to broker #{inspect(host)}:#{inspect port} because of error #{inspect err}") + + err -> + Logger.log( + :error, + "Could not connect to broker #{inspect(host)}:#{inspect(port)} because of error #{ + inspect(err) + }" + ) + nil end end - @spec close_socket(nil | Socket.t) :: :ok + @spec close_socket(nil | Socket.t()) :: :ok def close_socket(nil), do: :ok def close_socket(socket), do: Socket.close(socket) - @spec send_async_request(Broker.t, iodata) :: :ok | {:error, :closed | :inet.posix} + @spec send_async_request(Broker.t(), iodata) :: + :ok | {:error, :closed | :inet.posix()} def send_async_request(broker, data) do socket = broker.socket + case Socket.send(socket, data) do - :ok -> :ok + :ok -> + :ok + {_, reason} -> - Logger.log(:error, "Asynchronously sending data to broker #{inspect broker.host}:#{inspect broker.port} failed with #{inspect reason}") + Logger.log( + :error, + "Asynchronously sending data to broker #{inspect(broker.host)}:#{ + inspect(broker.port) + } failed with #{inspect(reason)}" + ) + reason end end - @spec send_sync_request(Broker.t, iodata, timeout) :: iodata | {:error, any()} + @spec send_sync_request(Broker.t(), iodata, timeout) :: + iodata | {:error, any()} def send_sync_request(%{:socket => socket} = broker, data, timeout) do :ok = Socket.setopts(socket, [:binary, {:packet, 4}, {:active, false}]) - response = case Socket.send(socket, data) do - :ok -> - case Socket.recv(socket, 0, timeout) do - {:ok, data} -> data - {:error, reason} -> - Logger.log(:error, "Receiving data from broker #{inspect broker.host}:#{inspect broker.port} failed with #{inspect reason}") - {:error, reason} - end - {_, reason} -> - Logger.log(:error, "Sending data to broker #{inspect broker.host}:#{inspect broker.port} failed with #{inspect reason}") - {:error, reason} - end + + response = + case Socket.send(socket, data) do + :ok -> + case Socket.recv(socket, 0, timeout) do + {:ok, data} -> + data + + {:error, reason} -> + Logger.log( + :error, + "Receiving data from broker #{inspect(broker.host)}:#{ + inspect(broker.port) + } failed with #{inspect(reason)}" + ) + + {:error, reason} + end + + {_, reason} -> + Logger.log( + :error, + "Sending data to broker #{inspect(broker.host)}:#{ + inspect(broker.port) + } failed with #{inspect(reason)}" + ) + + {:error, reason} + end :ok = Socket.setopts(socket, [:binary, {:packet, 4}, {:active, true}]) response end - @spec format_host(binary) :: [char] | :inet.ip_address + @spec format_host(binary) :: [char] | :inet.ip_address() def format_host(host) do case Regex.scan(~r/^(\d{1,3})\.(\d{1,3})\.(\d{1,3})\.(\d{1,3})$/, host) do - [match_data] = [[_, _, _, _, _]] -> match_data |> tl |> List.flatten |> Enum.map(&String.to_integer/1) |> List.to_tuple - _ -> apply(String, :to_char_list, [host]) # to_char_list is deprecated from Elixir 1.3 onward + [match_data] = [[_, _, _, _, _]] -> + match_data + |> tl + |> List.flatten() + |> Enum.map(&String.to_integer/1) + |> List.to_tuple() + + # to_char_list is deprecated from Elixir 1.3 onward + _ -> + apply(String, :to_char_list, [host]) end end defp build_socket_options([]) do [:binary, {:packet, 4}] end + defp build_socket_options(ssl_options) do build_socket_options([]) ++ ssl_options end diff --git a/lib/kafka_ex/protocol.ex b/lib/kafka_ex/protocol.ex index 6003d978..82fe1af6 100644 --- a/lib/kafka_ex/protocol.ex +++ b/lib/kafka_ex/protocol.ex @@ -2,47 +2,52 @@ defmodule KafkaEx.Protocol do @moduledoc false @message_type_to_api_key %{ - produce: 0, - fetch: 1, - offset: 2, - metadata: 3, - offset_commit: 8, - offset_fetch: 9, + produce: 0, + fetch: 1, + offset: 2, + metadata: 3, + offset_commit: 8, + offset_fetch: 9, consumer_metadata: 10, - join_group: 11, - heartbeat: 12, - leave_group: 13, - sync_group: 14, - api_versions: 18, - create_topics: 19, + join_group: 11, + heartbeat: 12, + leave_group: 13, + sync_group: 14, + api_versions: 18, + create_topics: 19 } # DescribeConfigs 32 # AlterConfigs 33 Valid resource types are "Topic" and "Broker". - @default_api_version 0 + @default_api_version 0 @spec api_key(atom) :: integer | nil def api_key(type) do Map.get(@message_type_to_api_key, type, nil) end - def create_request(type, correlation_id, client_id, api_version \\ @default_api_version) do - << api_key(type) :: 16, api_version :: 16, correlation_id :: 32, - byte_size(client_id) :: 16, client_id :: binary >> + def create_request( + type, + correlation_id, + client_id, + api_version \\ @default_api_version + ) do + <> end @error_map %{ - 0 => :no_error, - 1 => :offset_out_of_range, - 2 => :invalid_message, - 3 => :unknown_topic_or_partition, - 4 => :invalid_message_size, - 5 => :leader_not_available, - 6 => :not_leader_for_partition, - 7 => :request_timed_out, - 8 => :broker_not_available, - 9 => :replica_not_available, + 0 => :no_error, + 1 => :offset_out_of_range, + 2 => :invalid_message, + 3 => :unknown_topic_or_partition, + 4 => :invalid_message_size, + 5 => :leader_not_available, + 6 => :not_leader_for_partition, + 7 => :request_timed_out, + 8 => :broker_not_available, + 9 => :replica_not_available, 10 => :message_size_too_large, 11 => :stale_controller_epoch, 12 => :offset_metadata_too_large, @@ -75,15 +80,14 @@ defmodule KafkaEx.Protocol do 40 => :invalid_config, 41 => :not_controller, 42 => :invalid_request, - 43 => :unsupported_for_message_format, - + 43 => :unsupported_for_message_format } @spec error(integer) :: atom | integer def error(err_no) do case err_no do -1 -> :unknown_error - _ -> @error_map[err_no] || err_no + _ -> @error_map[err_no] || err_no end end end diff --git a/lib/kafka_ex/protocol/api_versions.ex b/lib/kafka_ex/protocol/api_versions.ex index 1c1cb58d..e03fc86b 100644 --- a/lib/kafka_ex/protocol/api_versions.ex +++ b/lib/kafka_ex/protocol/api_versions.ex @@ -1,4 +1,3 @@ - defmodule KafkaEx.Protocol.ApiVersions do alias KafkaEx.Protocol require Logger @@ -21,55 +20,87 @@ defmodule KafkaEx.Protocol.ApiVersions do defmodule Response do @moduledoc false defstruct error_code: nil, api_versions: nil, throttle_time_ms: nil + @type t :: %Response{ - error_code: atom, - api_versions: [ApiVersion], - throttle_time_ms: integer, - } + error_code: atom, + api_versions: [ApiVersion], + throttle_time_ms: integer + } end defmodule ApiVersion do @moduledoc false defstruct api_key: nil, min_version: nil, max_version: nil + @type t :: %ApiVersion{ - api_key: integer, - min_version: integer, - max_version: integer, - } + api_key: integer, + min_version: integer, + max_version: integer + } end @spec create_request(integer, binary, integer) :: binary - def create_request(correlation_id, client_id, this_api_version \\ @default_this_api_version) + def create_request( + correlation_id, + client_id, + this_api_version \\ @default_this_api_version + ) + + def create_request(correlation_id, client_id, 1), + do: create_request(correlation_id, client_id, 0) - def create_request(correlation_id, client_id, 1), do: create_request(correlation_id, client_id, 0) - def create_request(correlation_id, client_id, 2), do: create_request(correlation_id, client_id, 0) + def create_request(correlation_id, client_id, 2), + do: create_request(correlation_id, client_id, 0) def create_request(correlation_id, client_id, 0) do Protocol.create_request(:api_versions, correlation_id, client_id) end - @spec parse_response(binary, integer) :: Response.t + @spec parse_response(binary, integer) :: Response.t() def parse_response(binary, this_api_version \\ @default_this_api_version) - def parse_response(<< _correlation_id :: 32-signed, - error_code :: 16-signed, - api_versions_count :: 32-signed, - rest :: binary - >>, this_api_version) do - %{parse_rest_of_response(api_versions_count, rest, this_api_version) | error_code: Protocol.error(error_code)} + def parse_response( + <<_correlation_id::32-signed, error_code::16-signed, + api_versions_count::32-signed, rest::binary>>, + this_api_version + ) do + %{ + parse_rest_of_response(api_versions_count, rest, this_api_version) + | error_code: Protocol.error(error_code) + } end defp parse_rest_of_response(api_versions_count, data, this_api_version) do - {api_versions, remaining_data} = Protocol.Common.read_array(api_versions_count, data, &parse_one_api_version/1) + {api_versions, remaining_data} = + Protocol.Common.read_array( + api_versions_count, + data, + &parse_one_api_version/1 + ) case {this_api_version, remaining_data} do - {0, ""} -> %Response{api_versions: api_versions, throttle_time_ms: @default_throttle_time} - {v, << throttle_time_ms :: 32-signed >>} when v in [1, 2] -> %Response{api_versions: api_versions, throttle_time_ms: throttle_time_ms} + {0, ""} -> + %Response{ + api_versions: api_versions, + throttle_time_ms: @default_throttle_time + } + + {v, <>} when v in [1, 2] -> + %Response{ + api_versions: api_versions, + throttle_time_ms: throttle_time_ms + } end end - defp parse_one_api_version(<< api_key :: 16-signed, min_version :: 16-signed, max_version :: 16-signed, rest :: binary >>) do - {%ApiVersion{api_key: api_key, min_version: min_version, max_version: max_version}, rest} + defp parse_one_api_version( + <> + ) do + {%ApiVersion{ + api_key: api_key, + min_version: min_version, + max_version: max_version + }, rest} end - end diff --git a/lib/kafka_ex/protocol/common.ex b/lib/kafka_ex/protocol/common.ex index 0bf4da9d..3bf7b508 100644 --- a/lib/kafka_ex/protocol/common.ex +++ b/lib/kafka_ex/protocol/common.ex @@ -9,19 +9,28 @@ defmodule KafkaEx.Protocol.Common do """ def topic_data([]), do: "" - def topic_data([topic|topics]) do - << byte_size(topic) :: 16-signed, topic :: binary >> <> topic_data(topics) + def topic_data([topic | topics]) do + <> <> topic_data(topics) end def parse_topics(0, _, _), do: [] - def parse_topics(topics_size, << topic_size :: 16-signed, topic :: size(topic_size)-binary, partitions_size :: 32-signed, rest :: binary >>, mod) do + + def parse_topics( + topics_size, + <>, + mod + ) do struct_module = Module.concat(mod, Response) {partitions, topics_data} = mod.parse_partitions(partitions_size, rest, []) + [ %{ __struct__: struct_module, - topic: topic, partitions: partitions - } | parse_topics(topics_size - 1, topics_data, mod) + topic: topic, + partitions: partitions + } + | parse_topics(topics_size - 1, topics_data, mod) ] end @@ -32,7 +41,6 @@ defmodule KafkaEx.Protocol.Common do def read_array(num_items, data, read_one) do {item, rest} = read_one.(data) {items, data_after_array} = read_array(num_items - 1, rest, read_one) - {[item | items], data_after_array} + {[item | items], data_after_array} end - end diff --git a/lib/kafka_ex/protocol/consumer_metadata.ex b/lib/kafka_ex/protocol/consumer_metadata.ex index 9b78407d..346bdf01 100644 --- a/lib/kafka_ex/protocol/consumer_metadata.ex +++ b/lib/kafka_ex/protocol/consumer_metadata.ex @@ -10,17 +10,21 @@ defmodule KafkaEx.Protocol.ConsumerMetadata do alias KafkaEx.Protocol.Metadata.Broker - defstruct coordinator_id: 0, coordinator_host: "", coordinator_port: 0, error_code: 0 + defstruct coordinator_id: 0, + coordinator_host: "", + coordinator_port: 0, + error_code: 0 + @type t :: %Response{ - coordinator_id: integer, - coordinator_host: binary, - coordinator_port: 0..65_535, - error_code: atom - } + coordinator_id: integer, + coordinator_host: binary, + coordinator_port: 0..65_535, + error_code: atom + } - @spec broker_for_consumer_group([Broker.t], t) :: Broker.t | nil + @spec broker_for_consumer_group([Broker.t()], t) :: Broker.t() | nil def broker_for_consumer_group(brokers, consumer_group_metadata) do - Enum.find(brokers, &(connected_coordinator?(&1, consumer_group_metadata))) + Enum.find(brokers, &connected_coordinator?(&1, consumer_group_metadata)) end defp connected_coordinator?(%Broker{} = broker, consumer_group_metadata) do @@ -31,11 +35,24 @@ defmodule KafkaEx.Protocol.ConsumerMetadata do @spec create_request(integer, binary, binary) :: binary def create_request(correlation_id, client_id, consumer_group) do - KafkaEx.Protocol.create_request(:consumer_metadata, correlation_id, client_id) <> << byte_size(consumer_group) :: 16-signed, consumer_group :: binary >> + KafkaEx.Protocol.create_request( + :consumer_metadata, + correlation_id, + client_id + ) <> <> end - @spec parse_response(binary) :: Response.t - def parse_response(<< _corr_id :: 32-signed, error_code :: 16-signed, coord_id :: 32-signed, coord_host_size :: 16-signed, coord_host :: size(coord_host_size)-binary, coord_port :: 32-signed, _ :: binary >>) do - %Response{coordinator_id: coord_id, coordinator_host: coord_host, coordinator_port: coord_port, error_code: Protocol.error(error_code)} + @spec parse_response(binary) :: Response.t() + def parse_response( + <<_corr_id::32-signed, error_code::16-signed, coord_id::32-signed, + coord_host_size::16-signed, coord_host::size(coord_host_size)-binary, + coord_port::32-signed, _::binary>> + ) do + %Response{ + coordinator_id: coord_id, + coordinator_host: coord_host, + coordinator_port: coord_port, + error_code: Protocol.error(error_code) + } end end diff --git a/lib/kafka_ex/protocol/create_topics.ex b/lib/kafka_ex/protocol/create_topics.ex index 315703cb..c48d8ea3 100644 --- a/lib/kafka_ex/protocol/create_topics.ex +++ b/lib/kafka_ex/protocol/create_topics.ex @@ -40,13 +40,14 @@ defmodule KafkaEx.Protocol.CreateTopics do replication_factor: -1, replica_assignment: [], config_entries: [] + @type t :: %TopicRequest{ - topic: binary, - num_partitions: integer, - replication_factor: integer, - replica_assignment: [ReplicaAssignment], - config_entries: [ConfigEntry], - } + topic: binary, + num_partitions: integer, + replication_factor: integer, + replica_assignment: [ReplicaAssignment], + config_entries: [ConfigEntry] + } end defmodule Request do @@ -68,88 +69,112 @@ defmodule KafkaEx.Protocol.CreateTopics do end def api_version(api_versions) do - KafkaEx.ApiVersions.find_api_version(api_versions, :create_topics, @supported_versions_range) + KafkaEx.ApiVersions.find_api_version( + api_versions, + :create_topics, + @supported_versions_range + ) end - @spec create_request(integer, binary, Request.t, integer) :: binary - def create_request(correlation_id, client_id, create_topics_request, api_version) + @spec create_request(integer, binary, Request.t(), integer) :: binary + def create_request( + correlation_id, + client_id, + create_topics_request, + api_version + ) def create_request(correlation_id, client_id, create_topics_request, 0) do Protocol.create_request(:create_topics, correlation_id, client_id) <> encode_topic_requests(create_topics_request.create_topic_requests) <> - << create_topics_request.timeout :: 32-signed >> + <> end - @spec encode_topic_requests([TopicRequest.t]) :: binary + @spec encode_topic_requests([TopicRequest.t()]) :: binary defp encode_topic_requests(requests) do requests |> map_encode(&encode_topic_request/1) end - @spec encode_topic_request(TopicRequest.t) :: binary + @spec encode_topic_request(TopicRequest.t()) :: binary defp encode_topic_request(request) do encode_string(request.topic) <> - << request.num_partitions :: 32-signed, request.replication_factor :: 16-signed >> <> + <> <> encode_replica_assignments(request.replica_assignment) <> encode_config_entries(request.config_entries) end - - @spec encode_replica_assignments([ReplicaAssignment.t]) :: binary + @spec encode_replica_assignments([ReplicaAssignment.t()]) :: binary defp encode_replica_assignments(replica_assignments) do replica_assignments |> map_encode(&encode_replica_assignment/1) end defp encode_replica_assignment(replica_assignment) do - << replica_assignment.partition :: 32-signed >> <> - replica_assignment.replicas |> map_encode(&(<< &1 :: 32-signed >>)) + (<> <> replica_assignment.replicas) + |> map_encode(&<<&1::32-signed>>) end - @spec encode_config_entries([ConfigEntry.t]) :: binary + @spec encode_config_entries([ConfigEntry.t()]) :: binary defp encode_config_entries(config_entries) do config_entries |> map_encode(&encode_config_entry/1) end - @spec encode_config_entry(ConfigEntry.t) :: binary + @spec encode_config_entry(ConfigEntry.t()) :: binary defp encode_config_entry(config_entry) do - encode_string(config_entry.config_name) <> encode_nullable_string(config_entry.config_value) + encode_string(config_entry.config_name) <> + encode_nullable_string(config_entry.config_value) end - @spec encode_nullable_string(String.t) :: binary + @spec encode_nullable_string(String.t()) :: binary defp encode_nullable_string(text) do case text do - nil -> << -1 :: 16-signed >> + nil -> <<-1::16-signed>> _ -> encode_string(text) end end - @spec encode_string(String.t) :: binary + @spec encode_string(String.t()) :: binary defp encode_string(text) do - << byte_size(text) :: 16-signed, text :: binary, >> + <> end defp map_encode(elems, function) do if nil == elems or [] == elems do - << 0 :: 32-signed >> + <<0::32-signed>> else - << length(elems) :: 32-signed >> <> - (elems - |> Enum.map(function) - |> Enum.reduce(&(&1 <> &2))) + <> <> + (elems + |> Enum.map(function) + |> Enum.reduce(&(&1 <> &2))) end end - @spec parse_response(binary, integer) :: [] | Response.t - def parse_response(<< _correlation_id :: 32-signed, topic_errors_count :: 32-signed, topic_errors :: binary >>, 0) do - %Response{topic_errors: parse_topic_errors(topic_errors_count, topic_errors)} + @spec parse_response(binary, integer) :: [] | Response.t() + def parse_response( + <<_correlation_id::32-signed, topic_errors_count::32-signed, + topic_errors::binary>>, + 0 + ) do + %Response{ + topic_errors: parse_topic_errors(topic_errors_count, topic_errors) + } end - @spec parse_topic_errors(integer, binary) :: [TopicError.t] + @spec parse_topic_errors(integer, binary) :: [TopicError.t()] defp parse_topic_errors(0, _), do: [] - defp parse_topic_errors(topic_errors_count, - << topic_name_size :: 16-signed, topic_name :: size(topic_name_size)-binary, error_code :: 16-signed, rest :: binary >>) do - [%TopicError{topic_name: topic_name, error_code: Protocol.error(error_code)} | parse_topic_errors(topic_errors_count - 1, rest)] + defp parse_topic_errors( + topic_errors_count, + <> + ) do + [ + %TopicError{ + topic_name: topic_name, + error_code: Protocol.error(error_code) + } + | parse_topic_errors(topic_errors_count - 1, rest) + ] end - end diff --git a/lib/kafka_ex/protocol/fetch.ex b/lib/kafka_ex/protocol/fetch.ex index 0d662c26..e6290ef9 100644 --- a/lib/kafka_ex/protocol/fetch.ex +++ b/lib/kafka_ex/protocol/fetch.ex @@ -10,15 +10,25 @@ defmodule KafkaEx.Protocol.Fetch do defmodule Request do @moduledoc false defstruct correlation_id: nil, - client_id: nil, topic: nil, partition: nil, - offset: nil, wait_time: nil, min_bytes: nil, - max_bytes: nil, auto_commit: nil + client_id: nil, + topic: nil, + partition: nil, + offset: nil, + wait_time: nil, + min_bytes: nil, + max_bytes: nil, + auto_commit: nil + @type t :: %Request{ - correlation_id: integer, client_id: binary, - topic: binary, partition: integer, - offset: integer, wait_time: integer, - min_bytes: integer, max_bytes: integer - } + correlation_id: integer, + client_id: binary, + topic: binary, + partition: integer, + offset: integer, + wait_time: integer, + min_bytes: integer, + max_bytes: integer + } end defmodule Response do @@ -36,83 +46,144 @@ defmodule KafkaEx.Protocol.Fetch do defmodule Message do @moduledoc false defstruct attributes: 0, crc: nil, offset: nil, key: nil, value: nil - @type t :: %Message{attributes: integer, crc: integer, offset: integer, key: binary, value: binary} + + @type t :: %Message{ + attributes: integer, + crc: integer, + offset: integer, + key: binary, + value: binary + } end - @spec create_request(Request.t) :: binary + @spec create_request(Request.t()) :: binary def create_request(fetch_request) do KafkaEx.Protocol.create_request( - :fetch, fetch_request.correlation_id, fetch_request.client_id + :fetch, + fetch_request.correlation_id, + fetch_request.client_id ) <> - << - -1 :: 32-signed, fetch_request.wait_time :: 32-signed, fetch_request.min_bytes :: 32-signed, - 1 :: 32-signed, byte_size(fetch_request.topic) :: 16-signed, fetch_request.topic :: binary, - 1 :: 32-signed, fetch_request.partition :: 32-signed, fetch_request.offset :: 64, - fetch_request.max_bytes :: 32 - >> + << + -1::32-signed, + fetch_request.wait_time::32-signed, + fetch_request.min_bytes::32-signed, + 1::32-signed, + byte_size(fetch_request.topic)::16-signed, + fetch_request.topic::binary, + 1::32-signed, + fetch_request.partition::32-signed, + fetch_request.offset::64, + fetch_request.max_bytes::32 + >> end - def parse_response(<< _correlation_id :: 32-signed, topics_size :: 32-signed, rest :: binary>>) do + def parse_response( + <<_correlation_id::32-signed, topics_size::32-signed, rest::binary>> + ) do parse_topics(topics_size, rest, __MODULE__) end def parse_partitions(0, rest, partitions), do: {partitions, rest} - def parse_partitions(partitions_size, << partition :: 32-signed, error_code :: 16-signed, hw_mark_offset :: 64-signed, - msg_set_size :: 32-signed, msg_set_data :: size(msg_set_size)-binary, rest :: binary >>, partitions) do + + def parse_partitions( + partitions_size, + <>, + partitions + ) do {:ok, message_set, last_offset} = parse_message_set([], msg_set_data) - parse_partitions(partitions_size - 1, rest, [%{partition: partition, error_code: Protocol.error(error_code), hw_mark_offset: hw_mark_offset, message_set: message_set, last_offset: last_offset} | partitions]) + + parse_partitions(partitions_size - 1, rest, [ + %{ + partition: partition, + error_code: Protocol.error(error_code), + hw_mark_offset: hw_mark_offset, + message_set: message_set, + last_offset: last_offset + } + | partitions + ]) end - defp parse_message_set([], << >>) do + defp parse_message_set([], <<>>) do {:ok, [], nil} end - defp parse_message_set(list, << offset :: 64, msg_size :: 32, msg_data :: size(msg_size)-binary, rest :: binary >>) do + + defp parse_message_set( + list, + <> + ) do {:ok, message} = parse_message(%Message{offset: offset}, msg_data) - parse_message_set(append_messages(message, list), rest) + parse_message_set(append_messages(message, list), rest) end - defp parse_message_set([last|_] = list, _) do + + defp parse_message_set([last | _] = list, _) do {:ok, Enum.reverse(list), last.offset} end - defp parse_message_set(_, << offset :: 64, msg_size :: 32, partial_message_data :: binary >>) when byte_size(partial_message_data) < msg_size do - raise RuntimeError, "Insufficient data fetched at offset #{offset}. Message size is #{msg_size} but only received #{byte_size(partial_message_data)} bytes. Try increasing max_bytes." + + defp parse_message_set( + _, + <> + ) + when byte_size(partial_message_data) < msg_size do + raise RuntimeError, + "Insufficient data fetched at offset #{offset}. Message size is #{ + msg_size + } but only received #{byte_size(partial_message_data)} bytes. Try increasing max_bytes." end # handles the single message case and the batch (compression) case defp append_messages([], list) do list end + defp append_messages([message | messages], list) do append_messages(messages, [message | list]) end + defp append_messages(message, list) do [message | list] end - defp parse_message(%Message{} = message, << crc :: 32, _magic :: 8, attributes :: 8, rest :: binary>>) do + defp parse_message( + %Message{} = message, + <> + ) do maybe_decompress(%{message | crc: crc, attributes: attributes}, rest) end defp maybe_decompress(%Message{attributes: 0} = message, rest) do parse_key(message, rest) end + defp maybe_decompress(%Message{attributes: attributes}, rest) do - << -1 :: 32-signed, value_size :: 32, value :: size(value_size)-binary >> = rest + <<-1::32-signed, value_size::32, value::size(value_size)-binary>> = rest decompressed = Compression.decompress(attributes, value) {:ok, msg_set, _offset} = parse_message_set([], decompressed) {:ok, msg_set} end - defp parse_key(%Message{} = message, << -1 :: 32-signed, rest :: binary >>) do + defp parse_key(%Message{} = message, <<-1::32-signed, rest::binary>>) do parse_value(%{message | key: nil}, rest) end - defp parse_key(%Message{} = message, << key_size :: 32, key :: size(key_size)-binary, rest :: binary >>) do + + defp parse_key( + %Message{} = message, + <> + ) do parse_value(%{message | key: key}, rest) end - defp parse_value(%Message{} = message, << -1 :: 32-signed >>) do + defp parse_value(%Message{} = message, <<-1::32-signed>>) do {:ok, %{message | value: nil}} end - defp parse_value(%Message{} = message, << value_size :: 32, value :: size(value_size)-binary >>) do + + defp parse_value( + %Message{} = message, + <> + ) do {:ok, %{message | value: value}} end end diff --git a/lib/kafka_ex/protocol/heartbeat.ex b/lib/kafka_ex/protocol/heartbeat.ex index 6fb81590..5284b4ba 100644 --- a/lib/kafka_ex/protocol/heartbeat.ex +++ b/lib/kafka_ex/protocol/heartbeat.ex @@ -8,10 +8,10 @@ defmodule KafkaEx.Protocol.Heartbeat do defstruct group_name: nil, member_id: nil, generation_id: nil @type t :: %Request{ - group_name: binary, - member_id: binary, - generation_id: integer, - } + group_name: binary, + member_id: binary, + generation_id: integer + } end defmodule Response do @@ -22,16 +22,16 @@ defmodule KafkaEx.Protocol.Heartbeat do @type t :: %Response{error_code: atom | integer} end - @spec create_request(integer, binary, Request.t) :: binary + @spec create_request(integer, binary, Request.t()) :: binary def create_request(correlation_id, client_id, request) do KafkaEx.Protocol.create_request(:heartbeat, correlation_id, client_id) <> - << byte_size(request.group_name) :: 16-signed, request.group_name :: binary, - request.generation_id :: 32-signed, - byte_size(request.member_id) :: 16-signed, request.member_id :: binary >> + <> end - @spec parse_response(binary) :: Response.t - def parse_response(<< _correlation_id :: 32-signed, error_code :: 16-signed >>) do + @spec parse_response(binary) :: Response.t() + def parse_response(<<_correlation_id::32-signed, error_code::16-signed>>) do %Response{error_code: KafkaEx.Protocol.error(error_code)} end end diff --git a/lib/kafka_ex/protocol/join_group.ex b/lib/kafka_ex/protocol/join_group.ex index 222307d0..c5bc5a34 100644 --- a/lib/kafka_ex/protocol/join_group.ex +++ b/lib/kafka_ex/protocol/join_group.ex @@ -11,62 +11,90 @@ defmodule KafkaEx.Protocol.JoinGroup do defmodule Request do @moduledoc false - defstruct member_id: nil, - group_name: nil, topics: nil, - session_timeout: nil + defstruct member_id: nil, group_name: nil, topics: nil, session_timeout: nil + @type t :: %Request{ - member_id: binary, - group_name: binary, - topics: [binary], - session_timeout: integer, - } + member_id: binary, + group_name: binary, + topics: [binary], + session_timeout: integer + } end defmodule Response do @moduledoc false - defstruct error_code: nil, generation_id: 0, leader_id: nil, member_id: nil, members: [] - @type t :: %Response{error_code: atom | integer, generation_id: integer, - leader_id: binary, member_id: binary, members: [binary]} + defstruct error_code: nil, + generation_id: 0, + leader_id: nil, + member_id: nil, + members: [] + + @type t :: %Response{ + error_code: atom | integer, + generation_id: integer, + leader_id: binary, + member_id: binary, + members: [binary] + } def leader?(%__MODULE__{member_id: member_id, leader_id: leader_id}) do member_id == leader_id end end - @spec create_request(integer, binary, Request.t) :: binary + @spec create_request(integer, binary, Request.t()) :: binary def create_request(correlation_id, client_id, %Request{} = join_group_req) do metadata = - << @metadata_version :: 16-signed, - length(join_group_req.topics) :: 32-signed, topic_data(join_group_req.topics) :: binary, - 0 :: 32-signed - >> + <<@metadata_version::16-signed, length(join_group_req.topics)::32-signed, + topic_data(join_group_req.topics)::binary, 0::32-signed>> KafkaEx.Protocol.create_request(:join_group, correlation_id, client_id) <> - << byte_size(join_group_req.group_name) :: 16-signed, join_group_req.group_name :: binary, - join_group_req.session_timeout :: 32-signed, - byte_size(join_group_req.member_id) :: 16-signed, join_group_req.member_id :: binary, - byte_size(@protocol_type) :: 16-signed, @protocol_type :: binary, - 1 :: 32-signed, # We always have just one GroupProtocl - byte_size(@strategy_name) :: 16-signed, @strategy_name :: binary, - byte_size(metadata) :: 32-signed, metadata :: binary - >> + << + byte_size(join_group_req.group_name)::16-signed, + join_group_req.group_name::binary, + join_group_req.session_timeout::32-signed, + byte_size(join_group_req.member_id)::16-signed, + join_group_req.member_id::binary, + byte_size(@protocol_type)::16-signed, + @protocol_type::binary, + # We always have just one GroupProtocl + 1::32-signed, + byte_size(@strategy_name)::16-signed, + @strategy_name::binary, + byte_size(metadata)::32-signed, + metadata::binary + >> end - @spec parse_response(binary) :: Response.t - def parse_response(<< _correlation_id :: 32-signed, error_code :: 16-signed, generation_id :: 32-signed, - protocol_len :: 16-signed, _protocol :: size(protocol_len)-binary, - leader_len :: 16-signed, leader :: size(leader_len)-binary, - member_id_len :: 16-signed, member_id :: size(member_id_len)-binary, - members_size :: 32-signed, rest :: binary >>) do + @spec parse_response(binary) :: Response.t() + def parse_response( + <<_correlation_id::32-signed, error_code::16-signed, + generation_id::32-signed, protocol_len::16-signed, + _protocol::size(protocol_len)-binary, leader_len::16-signed, + leader::size(leader_len)-binary, member_id_len::16-signed, + member_id::size(member_id_len)-binary, members_size::32-signed, + rest::binary>> + ) do members = parse_members(members_size, rest, []) - %Response{error_code: KafkaEx.Protocol.error(error_code), generation_id: generation_id, - leader_id: leader, member_id: member_id, members: members} + + %Response{ + error_code: KafkaEx.Protocol.error(error_code), + generation_id: generation_id, + leader_id: leader, + member_id: member_id, + members: members + } end defp parse_members(0, <<>>, members), do: members - defp parse_members(size, << member_len :: 16-signed, member :: size(member_len)-binary, - meta_len :: 32-signed, _metadata :: size(meta_len)-binary, - rest :: binary >>, members) do - parse_members(size - 1, rest, [member|members]) + + defp parse_members( + size, + <>, + members + ) do + parse_members(size - 1, rest, [member | members]) end end diff --git a/lib/kafka_ex/protocol/leave_group.ex b/lib/kafka_ex/protocol/leave_group.ex index 097cf684..a4549efa 100644 --- a/lib/kafka_ex/protocol/leave_group.ex +++ b/lib/kafka_ex/protocol/leave_group.ex @@ -4,9 +4,9 @@ defmodule KafkaEx.Protocol.LeaveGroup do defstruct group_name: nil, member_id: nil @type t :: %Request{ - group_name: binary, - member_id: binary, - } + group_name: binary, + member_id: binary + } end defmodule Response do @@ -15,18 +15,18 @@ defmodule KafkaEx.Protocol.LeaveGroup do defstruct error_code: nil @type t :: %Response{ - error_code: atom | integer, - } + error_code: atom | integer + } end - @spec create_request(integer, binary, Request.t) :: binary + @spec create_request(integer, binary, Request.t()) :: binary def create_request(correlation_id, client_id, request) do KafkaEx.Protocol.create_request(:leave_group, correlation_id, client_id) <> - << byte_size(request.group_name) :: 16-signed, request.group_name :: binary, - byte_size(request.member_id) :: 16-signed, request.member_id :: binary >> + <> end - def parse_response(<< _correlation_id :: 32-signed, error_code :: 16-signed >>) do + def parse_response(<<_correlation_id::32-signed, error_code::16-signed>>) do %Response{error_code: KafkaEx.Protocol.error(error_code)} end end diff --git a/lib/kafka_ex/protocol/metadata.ex b/lib/kafka_ex/protocol/metadata.ex index c51a4f61..a11dfd1f 100644 --- a/lib/kafka_ex/protocol/metadata.ex +++ b/lib/kafka_ex/protocol/metadata.ex @@ -20,7 +20,14 @@ defmodule KafkaEx.Protocol.Metadata do alias KafkaEx.Socket defstruct node_id: -1, host: "", port: 0, socket: nil, is_controller: nil - @type t :: %__MODULE__{node_id: integer, host: binary, port: integer, socket: KafkaEx.Socket.t, is_controller: boolean} + + @type t :: %__MODULE__{ + node_id: integer, + host: binary, + port: integer, + socket: KafkaEx.Socket.t(), + is_controller: boolean + } def connected?(%Broker{} = broker) do broker.socket != nil && Socket.open?(broker.socket) @@ -32,30 +39,39 @@ defmodule KafkaEx.Protocol.Metadata do alias KafkaEx.Protocol.Metadata.Broker alias KafkaEx.Protocol.Metadata.TopicMetadata defstruct brokers: [], topic_metadatas: [], controller_id: nil + @type t :: %Response{ - brokers: [Broker.t], - topic_metadatas: [TopicMetadata.t], - controller_id: integer - } + brokers: [Broker.t()], + topic_metadatas: [TopicMetadata.t()], + controller_id: integer + } def broker_for_topic(metadata, brokers, topic, partition) do case Enum.find(metadata.topic_metadatas, &(topic == &1.topic)) do - nil -> nil - topic_metadata -> find_lead_broker(metadata.brokers, topic_metadata, brokers, partition) + nil -> + nil + + topic_metadata -> + find_lead_broker(metadata.brokers, topic_metadata, brokers, partition) end end def partitions_for_topic(metadata, topic) do case Enum.find(metadata.topic_metadatas, &(&1.topic == topic)) do nil -> - [] # topic doesn't exist yet, no partitions + # topic doesn't exist yet, no partitions + [] + topic_metadata -> - Enum.map(topic_metadata.partition_metadatas, &(&1.partition_id)) + Enum.map(topic_metadata.partition_metadatas, & &1.partition_id) end end defp find_lead_broker(metadata_brokers, topic_metadata, brokers, partition) do - case Enum.find(topic_metadata.partition_metadatas, &(partition == &1.partition_id)) do + case Enum.find( + topic_metadata.partition_metadatas, + &(partition == &1.partition_id) + ) do nil -> nil lead_broker -> find_broker(lead_broker, metadata_brokers, brokers) end @@ -63,7 +79,9 @@ defmodule KafkaEx.Protocol.Metadata do defp find_broker(lead_broker, metadata_brokers, brokers) do case Enum.find(metadata_brokers, &(lead_broker.leader == &1.node_id)) do - nil -> nil + nil -> + nil + broker -> Enum.find(brokers, &broker_for_host?(&1, broker.host, broker.port)) end @@ -77,40 +95,62 @@ defmodule KafkaEx.Protocol.Metadata do defmodule TopicMetadata do @moduledoc false alias KafkaEx.Protocol.Metadata.PartitionMetadata - defstruct error_code: 0, topic: nil, is_internal: nil, partition_metadatas: [] + + defstruct error_code: 0, + topic: nil, + is_internal: nil, + partition_metadatas: [] + @type t :: %TopicMetadata{ - error_code: integer | :no_error, - topic: nil | binary, - is_internal: nil | boolean, - partition_metadatas: [PartitionMetadata.t] - } + error_code: integer | :no_error, + topic: nil | binary, + is_internal: nil | boolean, + partition_metadatas: [PartitionMetadata.t()] + } end defmodule PartitionMetadata do @moduledoc false - defstruct error_code: 0, partition_id: nil, leader: -1, replicas: [], isrs: [] + defstruct error_code: 0, + partition_id: nil, + leader: -1, + replicas: [], + isrs: [] + @type t :: %PartitionMetadata{ - error_code: integer, - partition_id: nil | integer, - leader: integer, - replicas: [integer], - isrs: [integer] - } + error_code: integer, + partition_id: nil | integer, + leader: integer, + replicas: [integer], + isrs: [integer] + } end def api_version(api_versions) do - case KafkaEx.ApiVersions.find_api_version(api_versions, :metadata, @supported_versions_range) do - {:ok, version} -> version + case KafkaEx.ApiVersions.find_api_version( + api_versions, + :metadata, + @supported_versions_range + ) do + {:ok, version} -> + version + # those three should never happen since :metadata is part of the protocol since the beginning. # they are left here as this will server as reference implementation # :unknown_message_for_server -> # :unknown_message_for_client -> # :no_version_supported -> - _ -> @default_api_version + _ -> + @default_api_version end end - def create_request(correlation_id, client_id, topics, api_version \\ @default_api_version) + def create_request( + correlation_id, + client_id, + topics, + api_version \\ @default_api_version + ) def create_request(correlation_id, client_id, nil, api_version) do create_request(correlation_id, client_id, "", api_version) @@ -118,102 +158,196 @@ defmodule KafkaEx.Protocol.Metadata do def create_request(correlation_id, client_id, "", api_version) do topic_count = if 0 == api_version, do: 0, else: -1 - KafkaEx.Protocol.create_request(:metadata, correlation_id, client_id, api_version) <> << topic_count :: 32-signed >> + + KafkaEx.Protocol.create_request( + :metadata, + correlation_id, + client_id, + api_version + ) <> <> end - def create_request(correlation_id, client_id, topic, api_version) when is_binary(topic) do + def create_request(correlation_id, client_id, topic, api_version) + when is_binary(topic) do create_request(correlation_id, client_id, [topic], api_version) end - def create_request(correlation_id, client_id, topics, api_version) when is_list(topics) do - KafkaEx.Protocol.create_request(:metadata, correlation_id, client_id, api_version) <> << length(topics) :: 32-signed, topic_data(topics) :: binary >> + def create_request(correlation_id, client_id, topics, api_version) + when is_list(topics) do + KafkaEx.Protocol.create_request( + :metadata, + correlation_id, + client_id, + api_version + ) <> <> end def parse_response(data), do: parse_response(data, @default_api_version) def parse_response(data, nil), do: parse_response(data, @default_api_version) - def parse_response(<< _correlation_id :: 32-signed, brokers_size :: 32-signed, rest :: binary >>, api_version) do + def parse_response( + <<_correlation_id::32-signed, brokers_size::32-signed, rest::binary>>, + api_version + ) do case api_version do 1 -> {brokers, rest} = parse_brokers(1, brokers_size, rest, []) - << controller_id :: 32-signed, rest :: binary >> = rest - << topic_metadatas_size :: 32-signed, rest :: binary >> = rest - %Response{brokers: brokers, controller_id: controller_id, topic_metadatas: parse_topic_metadatas_v1(topic_metadatas_size, rest)} + <> = rest + <> = rest + + %Response{ + brokers: brokers, + controller_id: controller_id, + topic_metadatas: parse_topic_metadatas_v1(topic_metadatas_size, rest) + } + 0 -> {brokers, rest} = parse_brokers(0, brokers_size, rest, []) - << topic_metadatas_size :: 32-signed, rest :: binary >> = rest - %Response{brokers: brokers, topic_metadatas: parse_topic_metadatas(topic_metadatas_size, rest)} + <> = rest + + %Response{ + brokers: brokers, + topic_metadatas: parse_topic_metadatas(topic_metadatas_size, rest) + } end end defp parse_brokers(_api_version, 0, rest, brokers), do: {brokers, rest} - defp parse_brokers(0, brokers_size, << node_id :: 32-signed, host_len :: 16-signed, host :: size(host_len)-binary, port :: 32-signed, rest :: binary >>, brokers) do - parse_brokers(0, brokers_size - 1, rest, [%Broker{node_id: node_id, host: host, port: port} | brokers]) + defp parse_brokers( + 0, + brokers_size, + <>, + brokers + ) do + parse_brokers(0, brokers_size - 1, rest, [ + %Broker{node_id: node_id, host: host, port: port} | brokers + ]) end # defp parse_brokers_v1(0, rest, brokers), do: {brokers, rest} - defp parse_brokers(1, brokers_size, << - node_id :: 32-signed, - host_len :: 16-signed, - host :: size(host_len)-binary, - port :: 32-signed, - # rack is nullable - -1 :: 16-signed, - rest :: binary - >>, brokers) do - parse_brokers(1, brokers_size - 1, rest, [%Broker{node_id: node_id, host: host, port: port} | brokers]) - end - - defp parse_brokers(1, brokers_size, << - node_id :: 32-signed, - host_len :: 16-signed, - host :: size(host_len)-binary, - port :: 32-signed, - rack_len :: 16-signed, - _rack :: size(rack_len)-binary, - rest :: binary - >>, brokers) do - parse_brokers(1, brokers_size - 1, rest, [%Broker{node_id: node_id, host: host, port: port} | brokers]) + defp parse_brokers( + 1, + brokers_size, + << + node_id::32-signed, + host_len::16-signed, + host::size(host_len)-binary, + port::32-signed, + # rack is nullable + -1::16-signed, + rest::binary + >>, + brokers + ) do + parse_brokers(1, brokers_size - 1, rest, [ + %Broker{node_id: node_id, host: host, port: port} | brokers + ]) + end + + defp parse_brokers( + 1, + brokers_size, + << + node_id::32-signed, + host_len::16-signed, + host::size(host_len)-binary, + port::32-signed, + rack_len::16-signed, + _rack::size(rack_len)-binary, + rest::binary + >>, + brokers + ) do + parse_brokers(1, brokers_size - 1, rest, [ + %Broker{node_id: node_id, host: host, port: port} | brokers + ]) end defp parse_topic_metadatas(0, _), do: [] - defp parse_topic_metadatas(topic_metadatas_size, << error_code :: 16-signed, topic_len :: 16-signed, topic :: size(topic_len)-binary, partition_metadatas_size :: 32-signed, rest :: binary >>) do - {partition_metadatas, rest} = parse_partition_metadatas(partition_metadatas_size, [], rest) - [%TopicMetadata{error_code: Protocol.error(error_code), topic: topic, partition_metadatas: partition_metadatas} | parse_topic_metadatas(topic_metadatas_size - 1, rest)] + defp parse_topic_metadatas( + topic_metadatas_size, + <> + ) do + {partition_metadatas, rest} = + parse_partition_metadatas(partition_metadatas_size, [], rest) + + [ + %TopicMetadata{ + error_code: Protocol.error(error_code), + topic: topic, + partition_metadatas: partition_metadatas + } + | parse_topic_metadatas(topic_metadatas_size - 1, rest) + ] end defp parse_topic_metadatas_v1(0, _), do: [] defp parse_topic_metadatas_v1( - topic_metadatas_size, - << error_code :: 16-signed, - topic_len :: 16-signed, - topic :: size(topic_len)-binary, - # booleans are actually 8-signed - is_internal :: 8-signed, - partition_metadatas_size :: 32-signed, - rest :: binary >>) do - {partition_metadatas, rest} = parse_partition_metadatas(partition_metadatas_size, [], rest) - [%TopicMetadata{error_code: Protocol.error(error_code), topic: topic, partition_metadatas: partition_metadatas, is_internal: is_internal == 1} | - parse_topic_metadatas_v1(topic_metadatas_size - 1, rest)] + topic_metadatas_size, + << + error_code::16-signed, + topic_len::16-signed, + topic::size(topic_len)-binary, + # booleans are actually 8-signed + is_internal::8-signed, + partition_metadatas_size::32-signed, + rest::binary + >> + ) do + {partition_metadatas, rest} = + parse_partition_metadatas(partition_metadatas_size, [], rest) + + [ + %TopicMetadata{ + error_code: Protocol.error(error_code), + topic: topic, + partition_metadatas: partition_metadatas, + is_internal: is_internal == 1 + } + | parse_topic_metadatas_v1(topic_metadatas_size - 1, rest) + ] end - defp parse_partition_metadatas(0, partition_metadatas, rest), do: {partition_metadatas, rest} - - defp parse_partition_metadatas(partition_metadatas_size, partition_metadatas, << error_code :: 16-signed, partition_id :: 32-signed, leader :: 32-signed, rest :: binary >>) do - {replicas, rest} = parse_replicas(rest) - {isrs, rest} = parse_isrs(rest) - parse_partition_metadatas(partition_metadatas_size - 1, [%PartitionMetadata{error_code: Protocol.error(error_code), partition_id: partition_id, leader: leader, replicas: replicas, isrs: isrs} | partition_metadatas], rest) + defp parse_partition_metadatas(0, partition_metadatas, rest), + do: {partition_metadatas, rest} + + defp parse_partition_metadatas( + partition_metadatas_size, + partition_metadatas, + <> + ) do + {replicas, rest} = parse_replicas(rest) + {isrs, rest} = parse_isrs(rest) + + parse_partition_metadatas( + partition_metadatas_size - 1, + [ + %PartitionMetadata{ + error_code: Protocol.error(error_code), + partition_id: partition_id, + leader: leader, + replicas: replicas, + isrs: isrs + } + | partition_metadatas + ], + rest + ) end - defp parse_replicas(<< num_replicas :: 32-signed, rest :: binary >>) do + defp parse_replicas(<>) do parse_int32_array(num_replicas, rest) end - defp parse_isrs(<< num_isrs :: 32-signed, rest ::binary >>) do + defp parse_isrs(<>) do parse_int32_array([], num_isrs, rest) end @@ -223,7 +357,7 @@ defmodule KafkaEx.Protocol.Metadata do {Enum.reverse(array), rest} end - defp parse_int32_array(array, num, << value :: 32-signed, rest :: binary >>) do - parse_int32_array([value|array], num - 1, rest) + defp parse_int32_array(array, num, <>) do + parse_int32_array([value | array], num - 1, rest) end end diff --git a/lib/kafka_ex/protocol/offset.ex b/lib/kafka_ex/protocol/offset.ex index b0b013b8..c551a9d4 100644 --- a/lib/kafka_ex/protocol/offset.ex +++ b/lib/kafka_ex/protocol/offset.ex @@ -7,8 +7,19 @@ defmodule KafkaEx.Protocol.Offset do defmodule Request do @moduledoc false - defstruct replica_id: -1, topic_name: nil, partition: nil, time: -1, max_number_of_offsets: 1 - @type t :: %Request{replica_id: integer, topic_name: binary, partition: integer, time: integer, max_number_of_offsets: integer} + defstruct replica_id: -1, + topic_name: nil, + partition: nil, + time: -1, + max_number_of_offsets: 1 + + @type t :: %Request{ + replica_id: integer, + topic_name: binary, + partition: integer, + time: integer, + max_number_of_offsets: integer + } end defmodule Response do @@ -16,50 +27,79 @@ defmodule KafkaEx.Protocol.Offset do defstruct topic: nil, partition_offsets: [] @type t :: %Response{topic: binary, partition_offsets: list} - def extract_offset([%__MODULE__{partition_offsets: [%{offset: [offset]}]}]), do: offset + def extract_offset([%__MODULE__{partition_offsets: [%{offset: [offset]}]}]), + do: offset + def extract_offset([%__MODULE__{partition_offsets: [%{offset: []}]}]), do: 0 end def create_request(correlation_id, client_id, topic, partition, time) do KafkaEx.Protocol.create_request(:offset, correlation_id, client_id) <> - << -1 :: 32-signed, 1 :: 32-signed, byte_size(topic) :: 16-signed, topic :: binary, 1 :: 32-signed, partition :: 32-signed, parse_time(time) :: 64, 1 :: 32>> + <<-1::32-signed, 1::32-signed, byte_size(topic)::16-signed, topic::binary, + 1::32-signed, partition::32-signed, parse_time(time)::64, 1::32>> end - def parse_response(<< _correlation_id :: 32-signed, num_topics :: 32-signed, rest :: binary >>), do: parse_topics(num_topics, rest) + def parse_response( + <<_correlation_id::32-signed, num_topics::32-signed, rest::binary>> + ), + do: parse_topics(num_topics, rest) defp parse_time(:latest), do: -1 defp parse_time(:earliest), do: -2 - @spec parse_time(:calendar.datetime) :: integer + @spec parse_time(:calendar.datetime()) :: integer defp parse_time(time) do current_time_in_seconds = :calendar.datetime_to_gregorian_seconds(time) + unix_epoch_in_seconds = :calendar.datetime_to_gregorian_seconds({{1970, 1, 1}, {0, 0, 0}}) + (current_time_in_seconds - unix_epoch_in_seconds) * 1000 end defp parse_topics(0, _), do: [] - defp parse_topics(topics_size, << topic_size :: 16-signed, topic :: size(topic_size)-binary, partitions_size :: 32-signed, rest :: binary >>) do + defp parse_topics( + topics_size, + <> + ) do {partitions, topics_data} = parse_partitions(partitions_size, rest) - [%Response{topic: topic, partition_offsets: partitions} | parse_topics(topics_size - 1, topics_data)] + + [ + %Response{topic: topic, partition_offsets: partitions} + | parse_topics(topics_size - 1, topics_data) + ] end defp parse_partitions(partitions_size, rest, partitions \\ []) defp parse_partitions(0, rest, partitions), do: {partitions, rest} - defp parse_partitions(partitions_size, << partition :: 32-signed, error_code :: 16-signed, offsets_size :: 32-signed, rest :: binary >>, partitions) do + defp parse_partitions( + partitions_size, + <>, + partitions + ) do {offsets, rest} = parse_offsets(offsets_size, rest) - parse_partitions(partitions_size - 1, rest, [%{partition: partition, error_code: Protocol.error(error_code), offset: offsets} | partitions]) + + parse_partitions(partitions_size - 1, rest, [ + %{ + partition: partition, + error_code: Protocol.error(error_code), + offset: offsets + } + | partitions + ]) end defp parse_offsets(offsets_size, rest, offsets \\ []) defp parse_offsets(0, rest, offsets), do: {Enum.reverse(offsets), rest} - defp parse_offsets(offsets_size, << offset :: 64-signed, rest :: binary >>, offsets) do - parse_offsets(offsets_size - 1, rest, [offset|offsets]) + defp parse_offsets(offsets_size, <>, offsets) do + parse_offsets(offsets_size - 1, rest, [offset | offsets]) end end diff --git a/lib/kafka_ex/protocol/offset_commit.ex b/lib/kafka_ex/protocol/offset_commit.ex index 3b3ead34..c72ba53f 100644 --- a/lib/kafka_ex/protocol/offset_commit.ex +++ b/lib/kafka_ex/protocol/offset_commit.ex @@ -7,8 +7,18 @@ defmodule KafkaEx.Protocol.OffsetCommit do defmodule Request do @moduledoc false - defstruct consumer_group: nil, topic: nil, partition: nil, offset: nil, metadata: "" - @type t :: %Request{consumer_group: binary, topic: binary, partition: integer, offset: integer} + defstruct consumer_group: nil, + topic: nil, + partition: nil, + offset: nil, + metadata: "" + + @type t :: %Request{ + consumer_group: binary, + topic: binary, + partition: integer, + offset: integer + } end defmodule Response do @@ -17,29 +27,52 @@ defmodule KafkaEx.Protocol.OffsetCommit do @type t :: %Response{partitions: [] | [integer], topic: binary} end - @spec create_request(integer, binary, Request.t) :: binary + @spec create_request(integer, binary, Request.t()) :: binary def create_request(correlation_id, client_id, offset_commit_request) do - Protocol.create_request(:offset_commit, correlation_id, client_id) <> << byte_size(offset_commit_request.consumer_group) :: 16-signed, offset_commit_request.consumer_group :: binary, 1 :: 32-signed, byte_size(offset_commit_request.topic) :: 16-signed, offset_commit_request.topic :: binary, 1 :: 32-signed, offset_commit_request.partition :: 32-signed, offset_commit_request.offset :: 64, byte_size(offset_commit_request.metadata) :: 16-signed, offset_commit_request.metadata :: binary >> + Protocol.create_request(:offset_commit, correlation_id, client_id) <> + <> end - @spec parse_response(binary) :: [] | [Response.t] - def parse_response(<< _correlation_id :: 32-signed, topics_count :: 32-signed, topics_data :: binary >>) do + @spec parse_response(binary) :: [] | [Response.t()] + def parse_response( + <<_correlation_id::32-signed, topics_count::32-signed, + topics_data::binary>> + ) do parse_topics(topics_count, topics_data) end defp parse_topics(0, _), do: [] - defp parse_topics(topic_count, << topic_size :: 16-signed, topic :: size(topic_size)-binary, partitions_count :: 32-signed, rest :: binary >>) do + defp parse_topics( + topic_count, + <> + ) do {partitions, topics_data} = parse_partitions(partitions_count, rest, []) - [%Response{topic: topic, partitions: partitions} | parse_topics(topic_count - 1, topics_data)] + + [ + %Response{topic: topic, partitions: partitions} + | parse_topics(topic_count - 1, topics_data) + ] end defp parse_topics(_, _), do: [] defp parse_partitions(0, rest, partitions), do: {partitions, rest} - defp parse_partitions(partitions_count, << partition :: 32-signed, _error_code :: 16-signed, rest :: binary >>, partitions) do - #do something with error_code + defp parse_partitions( + partitions_count, + <>, + partitions + ) do + # do something with error_code parse_partitions(partitions_count - 1, rest, [partition | partitions]) end end diff --git a/lib/kafka_ex/protocol/offset_fetch.ex b/lib/kafka_ex/protocol/offset_fetch.ex index 2bb58d8c..aef44f60 100644 --- a/lib/kafka_ex/protocol/offset_fetch.ex +++ b/lib/kafka_ex/protocol/offset_fetch.ex @@ -9,11 +9,12 @@ defmodule KafkaEx.Protocol.OffsetFetch do defmodule Request do @moduledoc false defstruct consumer_group: nil, topic: nil, partition: nil + @type t :: %Request{ - consumer_group: nil | binary, - topic: binary, - partition: integer - } + consumer_group: nil | binary, + topic: binary, + partition: integer + } end defmodule Response do @@ -27,26 +28,53 @@ defmodule KafkaEx.Protocol.OffsetFetch do def last_offset(offset_fetch_data) do case offset_fetch_data do - [] -> 0 - _ -> partitions = offset_fetch_data |> hd |> Map.get(:partitions, []) + [] -> + 0 + + _ -> + partitions = offset_fetch_data |> hd |> Map.get(:partitions, []) + case partitions do [] -> 0 - _ -> partitions |> hd |> Map.get(:offset, 0) + _ -> partitions |> hd |> Map.get(:offset, 0) end end end end def create_request(correlation_id, client_id, offset_fetch_request) do - KafkaEx.Protocol.create_request(:offset_fetch, correlation_id, client_id) <> << byte_size(offset_fetch_request.consumer_group) :: 16-signed, offset_fetch_request.consumer_group :: binary, 1 :: 32-signed, byte_size(offset_fetch_request.topic) :: 16-signed, offset_fetch_request.topic :: binary, 1 :: 32-signed, offset_fetch_request.partition :: 32 >> + KafkaEx.Protocol.create_request(:offset_fetch, correlation_id, client_id) <> + <> end - def parse_response(<< _correlation_id :: 32-signed, topics_size :: 32-signed, topics_data :: binary >>) do + def parse_response( + <<_correlation_id::32-signed, topics_size::32-signed, + topics_data::binary>> + ) do parse_topics(topics_size, topics_data, __MODULE__) end def parse_partitions(0, rest, partitions), do: {partitions, rest} - def parse_partitions(partitions_size, << partition :: 32-signed, offset :: 64-signed, metadata_size :: 16-signed, metadata :: size(metadata_size)-binary, error_code :: 16-signed, rest :: binary >>, partitions) do - parse_partitions(partitions_size - 1, rest, [%{partition: partition, offset: offset, metadata: metadata, error_code: Protocol.error(error_code)} | partitions]) + + def parse_partitions( + partitions_size, + <>, + partitions + ) do + parse_partitions(partitions_size - 1, rest, [ + %{ + partition: partition, + offset: offset, + metadata: metadata, + error_code: Protocol.error(error_code) + } + | partitions + ]) end end diff --git a/lib/kafka_ex/protocol/produce.ex b/lib/kafka_ex/protocol/produce.ex index 734fb971..a5020a9f 100644 --- a/lib/kafka_ex/protocol/produce.ex +++ b/lib/kafka_ex/protocol/produce.ex @@ -89,16 +89,20 @@ defmodule KafkaEx.Protocol.Produce do defp create_message_set(messages, compression_type) do {message_set, _} = create_message_set(messages, :none) + {compressed_message_set, attribute} = Compression.compress(compression_type, message_set) + {message, msize} = create_message(compressed_message_set, nil, attribute) - {[<< 0 :: 64-signed >>, << msize :: 32-signed >>, message], 8 + 4 + msize} + {[<<0::64-signed>>, <>, message], 8 + 4 + msize} end - defp create_message_set_uncompressed([%Message{key: key, value: value}|messages]) do + defp create_message_set_uncompressed([ + %Message{key: key, value: value} | messages + ]) do {message, msize} = create_message(value, key) - message_set = [<< 0 :: 64-signed >>, << msize :: 32-signed >>, message] + message_set = [<<0::64-signed>>, <>, message] {message_set2, ms2size} = create_message_set(messages, :none) {[message_set, message_set2], 8 + 4 + msize + ms2size} end @@ -106,16 +110,17 @@ defmodule KafkaEx.Protocol.Produce do defp create_message(value, key, attributes \\ 0) do {bkey, skey} = bytes(key) {bvalue, svalue} = bytes(value) - sub = [<< 0 :: 8, attributes :: 8-signed >>, bkey, bvalue] + sub = [<<0::8, attributes::8-signed>>, bkey, bvalue] crc = :erlang.crc32(sub) - {[<< crc :: 32 >>, sub], 4 + 2 + skey + svalue} + {[<>, sub], 4 + 2 + skey + svalue} end - defp bytes(nil), do: {<< -1 :: 32-signed >>, 4} + defp bytes(nil), do: {<<-1::32-signed>>, 4} + defp bytes(data) do case :erlang.iolist_size(data) do - 0 -> {<< 0 :: 32 >>, 4} - size -> {[<< size :: 32>> , data], 4 + size} + 0 -> {<<0::32>>, 4} + size -> {[<>, data], 4 + size} end end diff --git a/lib/kafka_ex/protocol/sync_group.ex b/lib/kafka_ex/protocol/sync_group.ex index a94be427..6f2781c2 100644 --- a/lib/kafka_ex/protocol/sync_group.ex +++ b/lib/kafka_ex/protocol/sync_group.ex @@ -6,14 +6,19 @@ defmodule KafkaEx.Protocol.SyncGroup do defmodule Request do @moduledoc false - defstruct member_id: nil, group_name: nil, generation_id: nil, assignments: [] + defstruct member_id: nil, + group_name: nil, + generation_id: nil, + assignments: [] @type t :: %Request{ - member_id: binary, - group_name: binary, - generation_id: integer, - assignments: [{member :: binary, [{topic :: binary, partitions :: [integer]}]}], - } + member_id: binary, + group_name: binary, + generation_id: integer, + assignments: [ + {member :: binary, [{topic :: binary, partitions :: [integer]}]} + ] + } end defmodule Assignment do @@ -25,71 +30,102 @@ defmodule KafkaEx.Protocol.SyncGroup do defmodule Response do @moduledoc false defstruct error_code: nil, assignments: [] - @type t :: %Response{error_code: atom | integer, assignments: [Assignment.t]} + + @type t :: %Response{ + error_code: atom | integer, + assignments: [Assignment.t()] + } end - @spec create_request(integer, binary, Request.t) :: binary + @spec create_request(integer, binary, Request.t()) :: binary def create_request(correlation_id, client_id, %Request{} = request) do KafkaEx.Protocol.create_request(:sync_group, correlation_id, client_id) <> - << byte_size(request.group_name) :: 16-signed, request.group_name :: binary, - request.generation_id :: 32-signed, - byte_size(request.member_id) :: 16-signed, request.member_id :: binary, - length(request.assignments) :: 32-signed, group_assignment_data(request.assignments, "") :: binary - >> + <> end - @spec parse_response(binary) :: Response.t - def parse_response(<< _correlation_id :: 32-signed, error_code :: 16-signed, - member_assignment_len :: 32-signed, - member_assignment :: size(member_assignment_len)-binary >>) do - %Response{error_code: KafkaEx.Protocol.error(error_code), assignments: parse_member_assignment(member_assignment)} + @spec parse_response(binary) :: Response.t() + def parse_response( + <<_correlation_id::32-signed, error_code::16-signed, + member_assignment_len::32-signed, + member_assignment::size(member_assignment_len)-binary>> + ) do + %Response{ + error_code: KafkaEx.Protocol.error(error_code), + assignments: parse_member_assignment(member_assignment) + } end # Helper functions to create assignment data structure defp group_assignment_data([], acc), do: acc - defp group_assignment_data([h|t], acc), do: group_assignment_data(t, acc <> member_assignment_data(h)) + + defp group_assignment_data([h | t], acc), + do: group_assignment_data(t, acc <> member_assignment_data(h)) defp member_assignment_data({member_id, member_assignment}) do assignment_bytes_for_member = << - @member_assignment_version :: 16-signed, - length(member_assignment) :: 32-signed, - topic_assignment_data(member_assignment, "") :: binary, - 0 :: 32-signed # UserData - >> - << byte_size(member_id) :: 16-signed, member_id :: binary, - byte_size(assignment_bytes_for_member) :: 32-signed, - assignment_bytes_for_member :: binary >> + @member_assignment_version::16-signed, + length(member_assignment)::32-signed, + topic_assignment_data(member_assignment, "")::binary, + # UserData + 0::32-signed + >> + + <> end defp topic_assignment_data([], acc), do: acc - defp topic_assignment_data([h|t], acc), do: topic_assignment_data(t, acc <> partition_assignment_data(h)) + + defp topic_assignment_data([h | t], acc), + do: topic_assignment_data(t, acc <> partition_assignment_data(h)) defp partition_assignment_data({topic_name, partition_ids}) do - << byte_size(topic_name) :: 16-signed, topic_name :: binary, - length(partition_ids) :: 32-signed, partition_id_data(partition_ids, "") :: binary >> + <> end defp partition_id_data([], acc), do: acc - defp partition_id_data([h|t], acc), do: partition_id_data(t, acc <> << h :: 32-signed >>) + + defp partition_id_data([h | t], acc), + do: partition_id_data(t, acc <> <>) # Helper functions to parse assignments defp parse_member_assignment(<<>>), do: [] - defp parse_member_assignment(<< @member_assignment_version :: 16-signed, assignments_size :: 32-signed, rest :: binary >>) do + + defp parse_member_assignment( + <<@member_assignment_version::16-signed, assignments_size::32-signed, + rest::binary>> + ) do parse_assignments(assignments_size, rest, []) end defp parse_assignments(0, _rest, assignments), do: assignments - defp parse_assignments(size, << topic_len :: 16-signed, topic :: size(topic_len)-binary, - partition_len :: 32-signed, - rest :: binary >>, assignments) do + + defp parse_assignments( + size, + <>, + assignments + ) do {partitions, rest} = parse_partitions(partition_len, rest, []) - parse_assignments(size - 1, rest, [{topic, partitions}|assignments]) + parse_assignments(size - 1, rest, [{topic, partitions} | assignments]) end defp parse_partitions(0, rest, partitions), do: {partitions, rest} - defp parse_partitions(size, << partition :: 32-signed, rest :: binary >>, partitions) do - parse_partitions(size - 1, rest, [partition|partitions]) + + defp parse_partitions( + size, + <>, + partitions + ) do + parse_partitions(size - 1, rest, [partition | partitions]) end end diff --git a/lib/kafka_ex/server.ex b/lib/kafka_ex/server.ex index ba629cc2..642f0530 100644 --- a/lib/kafka_ex/server.ex +++ b/lib/kafka_ex/server.ex @@ -42,25 +42,25 @@ defmodule KafkaEx.Server do ) @type t :: %State{ - metadata: Metadata.Response.t, - brokers: [Broker.t], - event_pid: nil | pid, - consumer_metadata: ConsumerMetadata.Response.t, - correlation_id: integer, - metadata_update_interval: nil | integer, - consumer_group_update_interval: nil | integer, - worker_name: atom, - ssl_options: KafkaEx.ssl_options, - use_ssl: boolean, - api_versions: [KafkaEx.Protocol.ApiVersions.ApiVersion], - } + metadata: Metadata.Response.t(), + brokers: [Broker.t()], + event_pid: nil | pid, + consumer_metadata: ConsumerMetadata.Response.t(), + correlation_id: integer, + metadata_update_interval: nil | integer, + consumer_group_update_interval: nil | integer, + worker_name: atom, + ssl_options: KafkaEx.ssl_options(), + use_ssl: boolean, + api_versions: [KafkaEx.Protocol.ApiVersions.ApiVersion] + } @spec increment_correlation_id(t) :: t def increment_correlation_id(%State{correlation_id: cid} = state) do %{state | correlation_id: cid + 1} end - @spec broker_for_partition(t, binary, integer) :: Broker.t | nil + @spec broker_for_partition(t, binary, integer) :: Broker.t() | nil def broker_for_partition(state, topic, partition) do MetadataResponse.broker_for_topic( state.metadata, @@ -72,112 +72,170 @@ defmodule KafkaEx.Server do end @callback kafka_server_init(args :: [term]) :: - {:ok, state} | - {:ok, state, timeout | :hibernate} | - :ignore | - {:stop, reason :: any} when state: any - @callback kafka_server_produce(request :: ProduceRequest.t, state :: State.t) :: - {:reply, reply, new_state} | - {:reply, reply, new_state, timeout | :hibernate} | - {:noreply, new_state} | - {:noreply, new_state, timeout | :hibernate} | - {:stop, reason, reply, new_state} | - {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_consumer_group(state :: State.t) :: - {:reply, reply, new_state} | - {:reply, reply, new_state, timeout | :hibernate} | - {:noreply, new_state} | - {:noreply, new_state, timeout | :hibernate} | - {:stop, reason, reply, new_state} | - {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_fetch(fetch_request :: FetchRequest.t, state :: State.t) :: - {:reply, reply, new_state} | - {:reply, reply, new_state, timeout | :hibernate} | - {:noreply, new_state} | - {:noreply, new_state, timeout | :hibernate} | - {:stop, reason, reply, new_state} | - {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_offset(topic :: binary, parition :: integer, time :: :calendar.datetime | :latest | :earliest, state :: State.t) :: - {:reply, reply, new_state} | - {:reply, reply, new_state, timeout | :hibernate} | - {:noreply, new_state} | - {:noreply, new_state, timeout | :hibernate} | - {:stop, reason, reply, new_state} | - {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_offset_fetch(request :: OffsetFetchRequest.t, state :: State.t) :: - {:reply, reply, new_state} | - {:reply, reply, new_state, timeout | :hibernate} | - {:noreply, new_state} | - {:noreply, new_state, timeout | :hibernate} | - {:stop, reason, reply, new_state} | - {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_offset_commit(request :: OffsetCommitRequest.t, state :: State.t) :: - {:reply, reply, new_state} | - {:reply, reply, new_state, timeout | :hibernate} | - {:noreply, new_state} | - {:noreply, new_state, timeout | :hibernate} | - {:stop, reason, reply, new_state} | - {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_consumer_group_metadata(state :: State.t) :: - {:reply, reply, new_state} | - {:reply, reply, new_state, timeout | :hibernate} | - {:noreply, new_state} | - {:noreply, new_state, timeout | :hibernate} | - {:stop, reason, reply, new_state} | - {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_metadata(topic :: binary, state :: State.t) :: - {:reply, reply, new_state} | - {:reply, reply, new_state, timeout | :hibernate} | - {:noreply, new_state} | - {:noreply, new_state, timeout | :hibernate} | - {:stop, reason, reply, new_state} | - {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_join_group(JoinGroupRequest.t, network_timeout :: integer, state :: State.t) :: - {:reply, reply, new_state} | - {:reply, reply, new_state, timeout | :hibernate} | - {:noreply, new_state} | - {:noreply, new_state, timeout | :hibernate} | - {:stop, reason, reply, new_state} | - {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_sync_group(SyncGroupRequest.t, network_timeout :: integer, state :: State.t) :: - {:reply, reply, new_state} | - {:reply, reply, new_state, timeout | :hibernate} | - {:noreply, new_state} | - {:noreply, new_state, timeout | :hibernate} | - {:stop, reason, reply, new_state} | - {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_leave_group(LeaveGroupRequest.t, network_timeout :: integer, state :: State.t) :: - {:reply, reply, new_state} | - {:reply, reply, new_state, timeout | :hibernate} | - {:noreply, new_state} | - {:noreply, new_state, timeout | :hibernate} | - {:stop, reason, reply, new_state} | - {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_server_heartbeat(HeartbeatRequest.t, network_timeout :: integer, state :: State.t) :: - {:reply, reply, new_state} | - {:reply, reply, new_state, timeout | :hibernate} | - {:noreply, new_state} | - {:noreply, new_state, timeout | :hibernate} | - {:stop, reason, reply, new_state} | - {:stop, reason, new_state} when reply: term, new_state: term, reason: term - @callback kafka_create_topics([CreateTopicsRequest.t], network_timeout :: integer, state :: State.t) :: - {:reply, reply, new_state} when reply: term, new_state: term - @callback kafka_api_versions(state :: State.t) :: - {:reply, reply, new_state} when reply: term, new_state: term - @callback kafka_server_update_metadata(state :: State.t) :: - {:noreply, new_state} | - {:noreply, new_state, timeout | :hibernate} | - {:stop, reason :: term, new_state} when new_state: term - @callback kafka_server_update_consumer_metadata(state :: State.t) :: - {:noreply, new_state} | - {:noreply, new_state, timeout | :hibernate} | - {:stop, reason :: term, new_state} when new_state: term - - @default_call_timeout 5_000 # Default from GenServer + {:ok, state} + | {:ok, state, timeout | :hibernate} + | :ignore + | {:stop, reason :: any} + when state: any + @callback kafka_server_produce( + request :: ProduceRequest.t(), + state :: State.t() + ) :: + {:reply, reply, new_state} + | {:reply, reply, new_state, timeout | :hibernate} + | {:noreply, new_state} + | {:noreply, new_state, timeout | :hibernate} + | {:stop, reason, reply, new_state} + | {:stop, reason, new_state} + when reply: term, new_state: term, reason: term + @callback kafka_server_consumer_group(state :: State.t()) :: + {:reply, reply, new_state} + | {:reply, reply, new_state, timeout | :hibernate} + | {:noreply, new_state} + | {:noreply, new_state, timeout | :hibernate} + | {:stop, reason, reply, new_state} + | {:stop, reason, new_state} + when reply: term, new_state: term, reason: term + @callback kafka_server_fetch( + fetch_request :: FetchRequest.t(), + state :: State.t() + ) :: + {:reply, reply, new_state} + | {:reply, reply, new_state, timeout | :hibernate} + | {:noreply, new_state} + | {:noreply, new_state, timeout | :hibernate} + | {:stop, reason, reply, new_state} + | {:stop, reason, new_state} + when reply: term, new_state: term, reason: term + @callback kafka_server_offset( + topic :: binary, + parition :: integer, + time :: :calendar.datetime() | :latest | :earliest, + state :: State.t() + ) :: + {:reply, reply, new_state} + | {:reply, reply, new_state, timeout | :hibernate} + | {:noreply, new_state} + | {:noreply, new_state, timeout | :hibernate} + | {:stop, reason, reply, new_state} + | {:stop, reason, new_state} + when reply: term, new_state: term, reason: term + @callback kafka_server_offset_fetch( + request :: OffsetFetchRequest.t(), + state :: State.t() + ) :: + {:reply, reply, new_state} + | {:reply, reply, new_state, timeout | :hibernate} + | {:noreply, new_state} + | {:noreply, new_state, timeout | :hibernate} + | {:stop, reason, reply, new_state} + | {:stop, reason, new_state} + when reply: term, new_state: term, reason: term + @callback kafka_server_offset_commit( + request :: OffsetCommitRequest.t(), + state :: State.t() + ) :: + {:reply, reply, new_state} + | {:reply, reply, new_state, timeout | :hibernate} + | {:noreply, new_state} + | {:noreply, new_state, timeout | :hibernate} + | {:stop, reason, reply, new_state} + | {:stop, reason, new_state} + when reply: term, new_state: term, reason: term + @callback kafka_server_consumer_group_metadata(state :: State.t()) :: + {:reply, reply, new_state} + | {:reply, reply, new_state, timeout | :hibernate} + | {:noreply, new_state} + | {:noreply, new_state, timeout | :hibernate} + | {:stop, reason, reply, new_state} + | {:stop, reason, new_state} + when reply: term, new_state: term, reason: term + @callback kafka_server_metadata(topic :: binary, state :: State.t()) :: + {:reply, reply, new_state} + | {:reply, reply, new_state, timeout | :hibernate} + | {:noreply, new_state} + | {:noreply, new_state, timeout | :hibernate} + | {:stop, reason, reply, new_state} + | {:stop, reason, new_state} + when reply: term, new_state: term, reason: term + @callback kafka_server_join_group( + JoinGroupRequest.t(), + network_timeout :: integer, + state :: State.t() + ) :: + {:reply, reply, new_state} + | {:reply, reply, new_state, timeout | :hibernate} + | {:noreply, new_state} + | {:noreply, new_state, timeout | :hibernate} + | {:stop, reason, reply, new_state} + | {:stop, reason, new_state} + when reply: term, new_state: term, reason: term + @callback kafka_server_sync_group( + SyncGroupRequest.t(), + network_timeout :: integer, + state :: State.t() + ) :: + {:reply, reply, new_state} + | {:reply, reply, new_state, timeout | :hibernate} + | {:noreply, new_state} + | {:noreply, new_state, timeout | :hibernate} + | {:stop, reason, reply, new_state} + | {:stop, reason, new_state} + when reply: term, new_state: term, reason: term + @callback kafka_server_leave_group( + LeaveGroupRequest.t(), + network_timeout :: integer, + state :: State.t() + ) :: + {:reply, reply, new_state} + | {:reply, reply, new_state, timeout | :hibernate} + | {:noreply, new_state} + | {:noreply, new_state, timeout | :hibernate} + | {:stop, reason, reply, new_state} + | {:stop, reason, new_state} + when reply: term, new_state: term, reason: term + @callback kafka_server_heartbeat( + HeartbeatRequest.t(), + network_timeout :: integer, + state :: State.t() + ) :: + {:reply, reply, new_state} + | {:reply, reply, new_state, timeout | :hibernate} + | {:noreply, new_state} + | {:noreply, new_state, timeout | :hibernate} + | {:stop, reason, reply, new_state} + | {:stop, reason, new_state} + when reply: term, new_state: term, reason: term + @callback kafka_create_topics( + [CreateTopicsRequest.t()], + network_timeout :: integer, + state :: State.t() + ) :: {:reply, reply, new_state} + when reply: term, new_state: term + @callback kafka_api_versions(state :: State.t()) :: {:reply, reply, new_state} + when reply: term, new_state: term + @callback kafka_server_update_metadata(state :: State.t()) :: + {:noreply, new_state} + | {:noreply, new_state, timeout | :hibernate} + | {:stop, reason :: term, new_state} + when new_state: term + @callback kafka_server_update_consumer_metadata(state :: State.t()) :: + {:noreply, new_state} + | {:noreply, new_state, timeout | :hibernate} + | {:stop, reason :: term, new_state} + when new_state: term + + # Default from GenServer + @default_call_timeout 5_000 @doc false - @spec call(GenServer.server(), atom | tuple, nil | number | opts :: Keyword.t) :: term + @spec call( + GenServer.server(), + atom | tuple, + nil | number | opts :: Keyword.t() + ) :: term def call(server, request, opts \\ []) + def call(server, request, opts) when is_list(opts) do call(server, request, opts[:timeout]) end @@ -186,7 +244,12 @@ defmodule KafkaEx.Server do # If using the configured sync_timeout that is less than the default # GenServer.call timeout, use the larger value unless explicitly set # using opts[:timeout]. - timeout = max(@default_call_timeout, Application.get_env(:kafka_ex, :sync_timeout, @default_call_timeout)) + timeout = + max( + @default_call_timeout, + Application.get_env(:kafka_ex, :sync_timeout, @default_call_timeout) + ) + call(server, request, timeout) end @@ -207,8 +270,8 @@ defmodule KafkaEx.Server do @wait_time 10 @min_bytes 1 @max_bytes 1_000_000 - @metadata_update_interval 30_000 - @sync_timeout 1_000 + @metadata_update_interval 30_000 + @sync_timeout 1_000 @ssl_options [] def init([args]) do @@ -288,11 +351,20 @@ defmodule KafkaEx.Server do end def terminate(reason, state) do - Logger.log(:debug, "Shutting down worker #{inspect state.worker_name}, reason: #{inspect reason}") + Logger.log( + :debug, + "Shutting down worker #{inspect(state.worker_name)}, reason: #{ + inspect(reason) + }" + ) + if state.event_pid do :gen_event.stop(state.event_pid) end - Enum.each(state.brokers, fn(broker) -> NetworkClient.close_socket(broker.socket) end) + + Enum.each(state.brokers, fn broker -> + NetworkClient.close_socket(broker.socket) + end) end # KakfaEx.Server behavior default implementations @@ -300,94 +372,196 @@ defmodule KafkaEx.Server do # credo:disable-for-next-line Credo.Check.Refactor.CyclomaticComplexity def kafka_server_produce(produce_request, state) do correlation_id = state.correlation_id + 1 - produce_request_data = try do - Produce.create_request(correlation_id, @client_id, produce_request) - rescue - e in FunctionClauseError -> nil - end + + produce_request_data = + try do + Produce.create_request(correlation_id, @client_id, produce_request) + rescue + e in FunctionClauseError -> nil + end case produce_request_data do - nil -> {:reply, {:error, "Invalid produce request"}, state} - _ -> kafka_server_produce_send_request(correlation_id, produce_request, produce_request_data, state) + nil -> + {:reply, {:error, "Invalid produce request"}, state} + + _ -> + kafka_server_produce_send_request( + correlation_id, + produce_request, + produce_request_data, + state + ) end end # credo:disable-for-next-line Credo.Check.Refactor.CyclomaticComplexity - def kafka_server_produce_send_request(correlation_id, produce_request, produce_request_data, state) do - {broker, state, corr_id} = case MetadataResponse.broker_for_topic(state.metadata, state.brokers, produce_request.topic, produce_request.partition) do - nil -> - {retrieved_corr_id, _} = retrieve_metadata(state.brokers, state.correlation_id, config_sync_timeout(), produce_request.topic, state.api_versions) - state = update_metadata(%{state | correlation_id: retrieved_corr_id}) - { - MetadataResponse.broker_for_topic(state.metadata, state.brokers, produce_request.topic, produce_request.partition), - state, - retrieved_corr_id - } - broker -> {broker, state, correlation_id} - end + def kafka_server_produce_send_request( + correlation_id, + produce_request, + produce_request_data, + state + ) do + {broker, state, corr_id} = + case MetadataResponse.broker_for_topic( + state.metadata, + state.brokers, + produce_request.topic, + produce_request.partition + ) do + nil -> + {retrieved_corr_id, _} = + retrieve_metadata( + state.brokers, + state.correlation_id, + config_sync_timeout(), + produce_request.topic, + state.api_versions + ) + + state = + update_metadata(%{state | correlation_id: retrieved_corr_id}) + + { + MetadataResponse.broker_for_topic( + state.metadata, + state.brokers, + produce_request.topic, + produce_request.partition + ), + state, + retrieved_corr_id + } + + broker -> + {broker, state, correlation_id} + end + + response = + case broker do + nil -> + Logger.log( + :error, + "kafka_server_produce_send_request: leader for topic #{ + produce_request.topic + }/#{produce_request.partition} is not available" + ) + + :leader_not_available + + broker -> + case produce_request.required_acks do + 0 -> + NetworkClient.send_async_request(broker, produce_request_data) - response = case broker do - nil -> - Logger.log(:error, "kafka_server_produce_send_request: leader for topic #{produce_request.topic}/#{produce_request.partition} is not available") - :leader_not_available - broker -> case produce_request.required_acks do - 0 -> NetworkClient.send_async_request(broker, produce_request_data) - _ -> - response = broker - |> NetworkClient.send_sync_request( - produce_request_data, - config_sync_timeout()) - |> case do - {:error, reason} -> reason - response -> Produce.parse_response(response) - end - - # credo:disable-for-next-line Credo.Check.Refactor.Nesting - case response do - [%KafkaEx.Protocol.Produce.Response{partitions: [%{error_code: :no_error, offset: offset}], topic: topic}] when offset != nil -> - {:ok, offset} _ -> - {:error, response} + response = + broker + |> NetworkClient.send_sync_request( + produce_request_data, + config_sync_timeout() + ) + |> case do + {:error, reason} -> reason + response -> Produce.parse_response(response) + end + + # credo:disable-for-next-line Credo.Check.Refactor.Nesting + case response do + [ + %KafkaEx.Protocol.Produce.Response{ + partitions: [%{error_code: :no_error, offset: offset}], + topic: topic + } + ] + when offset != nil -> + {:ok, offset} + + _ -> + {:error, response} + end end end - end + state = %{state | correlation_id: corr_id + 1} {:reply, response, state} end def kafka_server_offset(topic, partition, time, state) do - offset_request = Offset.create_request(state.correlation_id, @client_id, topic, partition, time) - {broker, state} = case MetadataResponse.broker_for_topic(state.metadata, state.brokers, topic, partition) do - nil -> - state = update_metadata(state) - {MetadataResponse.broker_for_topic(state.metadata, state.brokers, topic, partition), state} - broker -> {broker, state} - end + offset_request = + Offset.create_request( + state.correlation_id, + @client_id, + topic, + partition, + time + ) + + {broker, state} = + case MetadataResponse.broker_for_topic( + state.metadata, + state.brokers, + topic, + partition + ) do + nil -> + state = update_metadata(state) + + {MetadataResponse.broker_for_topic( + state.metadata, + state.brokers, + topic, + partition + ), state} + + broker -> + {broker, state} + end - {response, state} = case broker do - nil -> - Logger.log(:error, "kafka_server_offset: leader for topic #{topic}/#{partition} is not available") - {:topic_not_found, state} - _ -> - response = broker - |> NetworkClient.send_sync_request( - offset_request, - config_sync_timeout()) - |> case do - {:error, reason} -> {:error, reason} - response -> Offset.parse_response(response) - end + {response, state} = + case broker do + nil -> + Logger.log( + :error, + "kafka_server_offset: leader for topic #{topic}/#{partition} is not available" + ) - state = %{state | correlation_id: state.correlation_id + 1} - {response, state} - end + {:topic_not_found, state} + + _ -> + response = + broker + |> NetworkClient.send_sync_request( + offset_request, + config_sync_timeout() + ) + |> case do + {:error, reason} -> {:error, reason} + response -> Offset.parse_response(response) + end + + state = %{state | correlation_id: state.correlation_id + 1} + {response, state} + end {:reply, response, state} end def kafka_server_metadata(topic, state) do - {correlation_id, metadata} = retrieve_metadata(state.brokers, state.correlation_id, config_sync_timeout(), topic, state.api_versions) - updated_state = %{state | metadata: metadata, correlation_id: correlation_id} + {correlation_id, metadata} = + retrieve_metadata( + state.brokers, + state.correlation_id, + config_sync_timeout(), + topic, + state.api_versions + ) + + updated_state = %{ + state + | metadata: metadata, + correlation_id: correlation_id + } + {:reply, metadata, updated_state} end @@ -396,77 +570,180 @@ defmodule KafkaEx.Server do end def update_metadata(state) do - {correlation_id, metadata} = retrieve_metadata(state.brokers, state.correlation_id, config_sync_timeout(), nil, state.api_versions) - metadata_brokers = metadata.brokers |> Enum.map(&(%{&1 | is_controller: &1.node_id == metadata.controller_id})) - brokers = state.brokers + {correlation_id, metadata} = + retrieve_metadata( + state.brokers, + state.correlation_id, + config_sync_timeout(), + nil, + state.api_versions + ) + + metadata_brokers = + metadata.brokers + |> Enum.map( + &%{&1 | is_controller: &1.node_id == metadata.controller_id} + ) + + brokers = + state.brokers |> remove_stale_brokers(metadata_brokers) |> add_new_brokers(metadata_brokers, state.ssl_options, state.use_ssl) - %{state | metadata: metadata, brokers: brokers, correlation_id: correlation_id + 1} + + %{ + state + | metadata: metadata, + brokers: brokers, + correlation_id: correlation_id + 1 + } end # credo:disable-for-next-line Credo.Check.Refactor.FunctionArity - def retrieve_metadata(brokers, correlation_id, sync_timeout, topic \\ [], server_api_versions \\ [:unsupported]) do - retrieve_metadata(brokers, correlation_id, sync_timeout, topic, @retry_count, 0, server_api_versions) + def retrieve_metadata( + brokers, + correlation_id, + sync_timeout, + topic \\ [], + server_api_versions \\ [:unsupported] + ) do + retrieve_metadata( + brokers, + correlation_id, + sync_timeout, + topic, + @retry_count, + 0, + server_api_versions + ) end # credo:disable-for-next-line Credo.Check.Refactor.FunctionArity - def retrieve_metadata(brokers, correlation_id, sync_timeout, topic, retry, error_code, server_api_versions \\ [:unsupported]) do + def retrieve_metadata( + brokers, + correlation_id, + sync_timeout, + topic, + retry, + error_code, + server_api_versions \\ [:unsupported] + ) do api_version = Metadata.api_version(server_api_versions) - retrieve_metadata_with_version(brokers, correlation_id, sync_timeout, topic, retry, error_code, api_version) + + retrieve_metadata_with_version( + brokers, + correlation_id, + sync_timeout, + topic, + retry, + error_code, + api_version + ) end # credo:disable-for-next-line Credo.Check.Refactor.FunctionArity - def retrieve_metadata_with_version(_, correlation_id, _sync_timeout, topic, 0, error_code, server_api_versions) do - Logger.log(:error, "Metadata request for topic #{inspect topic} failed with error_code #{inspect error_code}") + def retrieve_metadata_with_version( + _, + correlation_id, + _sync_timeout, + topic, + 0, + error_code, + server_api_versions + ) do + Logger.log( + :error, + "Metadata request for topic #{inspect(topic)} failed with error_code #{ + inspect(error_code) + }" + ) + {correlation_id, %Metadata.Response{}} end # credo:disable-for-next-line Credo.Check.Refactor.FunctionArity - def retrieve_metadata_with_version(brokers, correlation_id, sync_timeout, topic, retry, _error_code, api_version) do - metadata_request = Metadata.create_request(correlation_id, @client_id, topic, api_version) + def retrieve_metadata_with_version( + brokers, + correlation_id, + sync_timeout, + topic, + retry, + _error_code, + api_version + ) do + metadata_request = + Metadata.create_request( + correlation_id, + @client_id, + topic, + api_version + ) + data = first_broker_response(metadata_request, brokers, sync_timeout) + if data do response = Metadata.parse_response(data, api_version) - case Enum.find(response.topic_metadatas, &(&1.error_code == :leader_not_available)) do - nil -> {correlation_id + 1, response} + + case Enum.find( + response.topic_metadatas, + &(&1.error_code == :leader_not_available) + ) do + nil -> + {correlation_id + 1, response} + topic_metadata -> :timer.sleep(300) - retrieve_metadata_with_version(brokers, correlation_id + 1, sync_timeout, topic, retry - 1, topic_metadata.error_code, api_version) + + retrieve_metadata_with_version( + brokers, + correlation_id + 1, + sync_timeout, + topic, + retry - 1, + topic_metadata.error_code, + api_version + ) end else - message = "Unable to fetch metadata from any brokers. Timeout is #{sync_timeout}." + message = + "Unable to fetch metadata from any brokers. Timeout is #{ + sync_timeout + }." + Logger.log(:error, message) raise message :no_metadata_available end end - - defoverridable [ - kafka_server_produce: 2, kafka_server_offset: 4, - kafka_server_metadata: 2, kafka_server_update_metadata: 1, - ] + defoverridable kafka_server_produce: 2, + kafka_server_offset: 4, + kafka_server_metadata: 2, + kafka_server_update_metadata: 1 defp kafka_common_init(args, name) do use_ssl = Keyword.get(args, :use_ssl, false) ssl_options = Keyword.get(args, :ssl_options, []) uris = Keyword.get(args, :uris, []) - metadata_update_interval = Keyword.get( - args, - :metadata_update_interval, - @metadata_update_interval - ) - brokers = for {host, port} <- uris do - connect_broker(host, port, ssl_options, use_ssl) - end + metadata_update_interval = + Keyword.get( + args, + :metadata_update_interval, + @metadata_update_interval + ) - {correlation_id, metadata} = retrieve_metadata( - brokers, - 0, - config_sync_timeout() - ) + brokers = + for {host, port} <- uris do + connect_broker(host, port, ssl_options, use_ssl) + end + + {correlation_id, metadata} = + retrieve_metadata( + brokers, + 0, + config_sync_timeout() + ) state = %State{ metadata: metadata, @@ -480,10 +757,12 @@ defmodule KafkaEx.Server do } state = update_metadata(state) - {:ok, _} = :timer.send_interval( - state.metadata_update_interval, - :update_metadata - ) + + {:ok, _} = + :timer.send_interval( + state.metadata_update_interval, + :update_metadata + ) state end @@ -498,9 +777,9 @@ defmodule KafkaEx.Server do defp client_request(request, state) do %{ - request | - client_id: @client_id, - correlation_id: state.correlation_id + request + | client_id: @client_id, + correlation_id: state.correlation_id } end @@ -510,10 +789,12 @@ defmodule KafkaEx.Server do case State.broker_for_partition(state, topic, partition) do nil -> updated_state = update_metadata(state) + { State.broker_for_partition(updated_state, topic, partition), updated_state } + broker -> {broker, state} end @@ -522,31 +803,39 @@ defmodule KafkaEx.Server do # assumes module.create_request(request) and module.parse_response # both work defp network_request(request, module, state) do - {broker, updated_state} = broker_for_partition_with_update( - state, - request.topic, - request.partition - ) + {broker, updated_state} = + broker_for_partition_with_update( + state, + request.topic, + request.partition + ) case broker do nil -> Logger.error(fn -> - "network_request: leader for topic #{request.topic}/#{request.partition} is not available" + "network_request: leader for topic #{request.topic}/#{ + request.partition + } is not available" end) + {{:error, :topic_not_found}, updated_state} + _ -> - wire_request = request - |> client_request(updated_state) - |> module.create_request + wire_request = + request + |> client_request(updated_state) + |> module.create_request - response = broker + response = + broker |> NetworkClient.send_sync_request( - wire_request, - config_sync_timeout()) + wire_request, + config_sync_timeout() + ) |> case do - {:error, reason} -> {:error, reason} - response -> module.parse_response(response) - end + {:error, reason} -> {:error, reason} + response -> module.parse_response(response) + end state_out = State.increment_correlation_id(updated_state) {response, state_out} @@ -554,30 +843,82 @@ defmodule KafkaEx.Server do end defp remove_stale_brokers(brokers, metadata_brokers) do - {brokers_to_keep, brokers_to_remove} = apply(Enum, :partition, [brokers, fn(broker) -> - Enum.find_value(metadata_brokers, &(broker.node_id == -1 || (broker.node_id == &1.node_id) && broker.socket && Socket.info(broker.socket))) - end]) + {brokers_to_keep, brokers_to_remove} = + apply(Enum, :partition, [ + brokers, + fn broker -> + Enum.find_value( + metadata_brokers, + &(broker.node_id == -1 || + (broker.node_id == &1.node_id && broker.socket && + Socket.info(broker.socket))) + ) + end + ]) + case length(brokers_to_keep) do - 0 -> brokers_to_remove - _ -> Enum.each(brokers_to_remove, fn(broker) -> - Logger.log(:debug, "Closing connection to broker #{broker.node_id}: #{inspect broker.host} on port #{inspect broker.port}") - NetworkClient.close_socket(broker.socket) - end) + 0 -> + brokers_to_remove + + _ -> + Enum.each(brokers_to_remove, fn broker -> + Logger.log( + :debug, + "Closing connection to broker #{broker.node_id}: #{ + inspect(broker.host) + } on port #{inspect(broker.port)}" + ) + + NetworkClient.close_socket(broker.socket) + end) + brokers_to_keep end end defp add_new_brokers(brokers, [], _, _), do: brokers - defp add_new_brokers(brokers, [metadata_broker|metadata_brokers], ssl_options, use_ssl) do + + defp add_new_brokers( + brokers, + [metadata_broker | metadata_brokers], + ssl_options, + use_ssl + ) do case Enum.find(brokers, &(metadata_broker.node_id == &1.node_id)) do - nil -> Logger.log(:debug, "Establishing connection to broker #{metadata_broker.node_id}: #{inspect metadata_broker.host} on port #{inspect metadata_broker.port}") - add_new_brokers([%{metadata_broker | socket: NetworkClient.create_socket(metadata_broker.host, metadata_broker.port, ssl_options, use_ssl)} | brokers], metadata_brokers, ssl_options, use_ssl) - _ -> add_new_brokers(brokers, metadata_brokers, ssl_options, use_ssl) + nil -> + Logger.log( + :debug, + "Establishing connection to broker #{metadata_broker.node_id}: #{ + inspect(metadata_broker.host) + } on port #{inspect(metadata_broker.port)}" + ) + + add_new_brokers( + [ + %{ + metadata_broker + | socket: + NetworkClient.create_socket( + metadata_broker.host, + metadata_broker.port, + ssl_options, + use_ssl + ) + } + | brokers + ], + metadata_brokers, + ssl_options, + use_ssl + ) + + _ -> + add_new_brokers(brokers, metadata_brokers, ssl_options, use_ssl) end end defp first_broker_response(request, brokers, timeout) do - Enum.find_value(brokers, fn(broker) -> + Enum.find_value(brokers, fn broker -> if Broker.connected?(broker) do # credo:disable-for-next-line Credo.Check.Refactor.Nesting case NetworkClient.send_sync_request(broker, request, timeout) do diff --git a/lib/kafka_ex/server_0_p_10_and_later.ex b/lib/kafka_ex/server_0_p_10_and_later.ex index 61f96879..8f1d47b2 100644 --- a/lib/kafka_ex/server_0_p_10_and_later.ex +++ b/lib/kafka_ex/server_0_p_10_and_later.ex @@ -18,7 +18,6 @@ defmodule KafkaEx.Server0P10AndLater do @consumer_group_update_interval 30_000 - def start_link(args, name \\ __MODULE__) def start_link(args, :no_name) do @@ -26,22 +25,33 @@ defmodule KafkaEx.Server0P10AndLater do end def start_link(args, name) do - GenServer.start_link(__MODULE__, [args, name], [name: name]) + GenServer.start_link(__MODULE__, [args, name], name: name) end # The functions below are all defined in KafkaEx.Server0P8P2 defdelegate kafka_server_consumer_group(state), to: Server0P8P2 defdelegate kafka_server_fetch(fetch_request, state), to: Server0P8P2 defdelegate kafka_server_offset_fetch(offset_fetch, state), to: Server0P8P2 - defdelegate kafka_server_offset_commit(offset_commit_request, state), to: Server0P8P2 + + defdelegate kafka_server_offset_commit(offset_commit_request, state), + to: Server0P8P2 + defdelegate kafka_server_consumer_group_metadata(state), to: Server0P8P2 defdelegate kafka_server_update_consumer_metadata(state), to: Server0P8P2 # The functions below are all defined in KafkaEx.Server0P9P0 - defdelegate kafka_server_join_group(request, network_timeout, state_in), to: Server0P9P0 - defdelegate kafka_server_sync_group(request, network_timeout, state_in), to: Server0P9P0 - defdelegate kafka_server_leave_group(request, network_timeout, state_in), to: Server0P9P0 - defdelegate kafka_server_heartbeat(request, network_timeout, state_in), to: Server0P9P0 + defdelegate kafka_server_join_group(request, network_timeout, state_in), + to: Server0P9P0 + + defdelegate kafka_server_sync_group(request, network_timeout, state_in), + to: Server0P9P0 + + defdelegate kafka_server_leave_group(request, network_timeout, state_in), + to: Server0P9P0 + + defdelegate kafka_server_heartbeat(request, network_timeout, state_in), + to: Server0P9P0 + defdelegate consumer_group?(state), to: Server0P9P0 def kafka_server_init([args]) do @@ -50,12 +60,21 @@ defmodule KafkaEx.Server0P10AndLater do def kafka_server_init([args, name]) do uris = Keyword.get(args, :uris, []) - metadata_update_interval = Keyword.get(args, :metadata_update_interval, @metadata_update_interval) - consumer_group_update_interval = Keyword.get(args, :consumer_group_update_interval, @consumer_group_update_interval) + + metadata_update_interval = + Keyword.get(args, :metadata_update_interval, @metadata_update_interval) + + consumer_group_update_interval = + Keyword.get( + args, + :consumer_group_update_interval, + @consumer_group_update_interval + ) # this should have already been validated, but it's possible someone could # try to short-circuit the start call consumer_group = Keyword.get(args, :consumer_group) + unless KafkaEx.valid_consumer_group?(consumer_group) do raise InvalidConsumerGroupError, consumer_group end @@ -63,12 +82,31 @@ defmodule KafkaEx.Server0P10AndLater do use_ssl = Keyword.get(args, :use_ssl, false) ssl_options = Keyword.get(args, :ssl_options, []) - brokers = Enum.map(uris, fn({host, port}) -> %Broker{host: host, port: port, socket: NetworkClient.create_socket(host, port, ssl_options, use_ssl)} end) + brokers = + Enum.map(uris, fn {host, port} -> + %Broker{ + host: host, + port: port, + socket: NetworkClient.create_socket(host, port, ssl_options, use_ssl) + } + end) + + {_, + %KafkaEx.Protocol.ApiVersions.Response{ + api_versions: api_versions, + error_code: :no_error + }, state} = kafka_api_versions(%State{brokers: brokers}) - {_, %KafkaEx.Protocol.ApiVersions.Response{api_versions: api_versions, error_code: :no_error}, state} = kafka_api_versions(%State{brokers: brokers}) api_versions = KafkaEx.ApiVersions.api_versions_map(api_versions) - {correlation_id, metadata} = retrieve_metadata(brokers, state.correlation_id, config_sync_timeout(), [], api_versions) + {correlation_id, metadata} = + retrieve_metadata( + brokers, + state.correlation_id, + config_sync_timeout(), + [], + api_versions + ) state = %State{ metadata: metadata, @@ -80,18 +118,26 @@ defmodule KafkaEx.Server0P10AndLater do worker_name: name, ssl_options: ssl_options, use_ssl: use_ssl, - api_versions: api_versions, + api_versions: api_versions } # Get the initial "real" broker list and start a regular refresh cycle. state = update_metadata(state) - {:ok, _} = :timer.send_interval(state.metadata_update_interval, :update_metadata) + + {:ok, _} = + :timer.send_interval(state.metadata_update_interval, :update_metadata) state = if consumer_group?(state) do # If we are using consumer groups then initialize the state and start the update cycle {_, updated_state} = update_consumer_metadata(state) - {:ok, _} = :timer.send_interval(state.consumer_group_update_interval, :update_consumer_metadata) + + {:ok, _} = + :timer.send_interval( + state.consumer_group_update_interval, + :update_consumer_metadata + ) + updated_state else state @@ -101,8 +147,21 @@ defmodule KafkaEx.Server0P10AndLater do end def kafka_server_metadata(topic, state) do - {correlation_id, metadata} = retrieve_metadata(state.brokers, state.correlation_id, config_sync_timeout(), topic, state.api_versions) - updated_state = %{state | metadata: metadata, correlation_id: correlation_id} + {correlation_id, metadata} = + retrieve_metadata( + state.brokers, + state.correlation_id, + config_sync_timeout(), + topic, + state.api_versions + ) + + updated_state = %{ + state + | metadata: metadata, + correlation_id: correlation_id + } + {:reply, metadata, updated_state} end @@ -111,70 +170,108 @@ defmodule KafkaEx.Server0P10AndLater do end def kafka_api_versions(state) do - response = state.correlation_id - |> ApiVersions.create_request(@client_id) - |> first_broker_response(state) - |> ApiVersions.parse_response + response = + state.correlation_id + |> ApiVersions.create_request(@client_id) + |> first_broker_response(state) + |> ApiVersions.parse_response() {:reply, response, %{state | correlation_id: state.correlation_id + 1}} end def kafka_create_topics(requests, network_timeout, state) do - api_version = case CreateTopics.api_version(state.api_versions) do - {:ok, api_version} -> api_version - _ -> raise "CreateTopic is not supported in this version of Kafka, or the versions supported by the client do not match the ones supported by the server." - end + api_version = + case CreateTopics.api_version(state.api_versions) do + {:ok, api_version} -> + api_version + + _ -> + raise "CreateTopic is not supported in this version of Kafka, or the versions supported by the client do not match the ones supported by the server." + end create_topics_request = %CreateTopics.Request{ create_topic_requests: requests, timeout: network_timeout } - main_request = CreateTopics.create_request(state.correlation_id, @client_id, create_topics_request, api_version) - - broker = state.brokers |> Enum.find(&(&1.is_controller)) - - {response, state} = case broker do - nil -> - Logger.log(:error, "Coordinator for topic is not available") - {:topic_not_found, state} - _ -> - response = broker - |> NetworkClient.send_sync_request(main_request, config_sync_timeout()) - |> case do - {:error, reason} -> {:error, reason} - response -> CreateTopics.parse_response(response, api_version) - end - {response, %{state | correlation_id: state.correlation_id + 1}} - end + main_request = + CreateTopics.create_request( + state.correlation_id, + @client_id, + create_topics_request, + api_version + ) + + broker = state.brokers |> Enum.find(& &1.is_controller) + + {response, state} = + case broker do + nil -> + Logger.log(:error, "Coordinator for topic is not available") + {:topic_not_found, state} + + _ -> + response = + broker + |> NetworkClient.send_sync_request( + main_request, + config_sync_timeout() + ) + |> case do + {:error, reason} -> {:error, reason} + response -> CreateTopics.parse_response(response, api_version) + end + + {response, %{state | correlation_id: state.correlation_id + 1}} + end {:reply, response, state} end - defp update_consumer_metadata(state), do: update_consumer_metadata(state, @retry_count, 0) + defp update_consumer_metadata(state), + do: update_consumer_metadata(state, @retry_count, 0) + + defp update_consumer_metadata( + %State{consumer_group: consumer_group} = state, + 0, + error_code + ) do + Logger.log( + :error, + "Fetching consumer_group #{consumer_group} metadata failed with error_code #{ + inspect(error_code) + }" + ) - defp update_consumer_metadata(%State{consumer_group: consumer_group} = state, 0, error_code) do - Logger.log(:error, "Fetching consumer_group #{consumer_group} metadata failed with error_code #{inspect error_code}") {%ConsumerMetadataResponse{error_code: error_code}, state} end - defp update_consumer_metadata(%State{consumer_group: consumer_group, correlation_id: correlation_id} = state, retry, _error_code) do - response = correlation_id + defp update_consumer_metadata( + %State{consumer_group: consumer_group, correlation_id: correlation_id} = + state, + retry, + _error_code + ) do + response = + correlation_id |> ConsumerMetadata.create_request(@client_id, consumer_group) |> first_broker_response(state) - |> ConsumerMetadata.parse_response + |> ConsumerMetadata.parse_response() case response.error_code do :no_error -> { response, %{ - state | - consumer_metadata: response, - correlation_id: state.correlation_id + 1 + state + | consumer_metadata: response, + correlation_id: state.correlation_id + 1 } } - _ -> :timer.sleep(400) + + _ -> + :timer.sleep(400) + update_consumer_metadata( %{state | correlation_id: state.correlation_id + 1}, retry - 1, @@ -184,18 +281,29 @@ defmodule KafkaEx.Server0P10AndLater do end defp broker_for_consumer_group(state) do - ConsumerMetadataResponse.broker_for_consumer_group(state.brokers, state.consumer_metadata) + ConsumerMetadataResponse.broker_for_consumer_group( + state.brokers, + state.consumer_metadata + ) end # refactored from two versions, one that used the first broker as valid answer, hence # the optional extra flag to do that. Wraps broker_for_consumer_group with an update # call if no broker was found. - def broker_for_consumer_group_with_update(state, use_first_as_default \\ false) do + def broker_for_consumer_group_with_update( + state, + use_first_as_default \\ false + ) do case broker_for_consumer_group(state) do nil -> {_, updated_state} = update_consumer_metadata(state) - default_broker = if use_first_as_default, do: hd(state.brokers), else: nil - {broker_for_consumer_group(updated_state) || default_broker, updated_state} + + default_broker = + if use_first_as_default, do: hd(state.brokers), else: nil + + {broker_for_consumer_group(updated_state) || default_broker, + updated_state} + broker -> {broker, state} end @@ -204,5 +312,4 @@ defmodule KafkaEx.Server0P10AndLater do defp first_broker_response(request, state) do first_broker_response(request, state.brokers, config_sync_timeout()) end - end diff --git a/lib/kafka_ex/server_0_p_8_p_0.ex b/lib/kafka_ex/server_0_p_8_p_0.ex index fc36edbe..a3659672 100644 --- a/lib/kafka_ex/server_0_p_8_p_0.ex +++ b/lib/kafka_ex/server_0_p_8_p_0.ex @@ -46,7 +46,7 @@ defmodule KafkaEx.Server0P8P0 do end def start_link(args, name) do - GenServer.start_link(__MODULE__, [args, name], [name: name]) + GenServer.start_link(__MODULE__, [args, name], name: name) end def kafka_server_fetch(fetch_request, state) do @@ -55,17 +55,44 @@ defmodule KafkaEx.Server0P8P0 do {:reply, response, state} end - def kafka_server_offset_fetch(_, _state), do: raise "Offset Fetch is not supported in 0.8.0 version of kafka" - def kafka_server_offset_commit(_, _state), do: raise "Offset Commit is not supported in 0.8.0 version of kafka" - def kafka_server_consumer_group(_state), do: raise "Consumer Group is not supported in 0.8.0 version of kafka" - def kafka_server_consumer_group_metadata(_state), do: raise "Consumer Group Metadata is not supported in 0.8.0 version of kafka" - def kafka_server_join_group(_, _, _state), do: raise "Join Group is not supported in 0.8.0 version of kafka" - def kafka_server_sync_group(_, _, _state), do: raise "Sync Group is not supported in 0.8.0 version of kafka" - def kafka_server_leave_group(_, _, _state), do: raise "Leave Group is not supported in 0.8.0 version of Kafka" - def kafka_server_heartbeat(_, _, _state), do: raise "Heartbeat is not supported in 0.8.0 version of kafka" - def kafka_server_update_consumer_metadata(_state), do: raise "Consumer Group Metadata is not supported in 0.8.0 version of kafka" - def kafka_api_versions(_state), do: raise "ApiVersions is not supported in 0.8.0 version of kafka" - def kafka_create_topics(_, _, _state), do: raise "CreateTopic is not supported in 0.8.0 version of kafka" + def kafka_server_offset_fetch(_, _state), + do: raise("Offset Fetch is not supported in 0.8.0 version of kafka") + + def kafka_server_offset_commit(_, _state), + do: raise("Offset Commit is not supported in 0.8.0 version of kafka") + + def kafka_server_consumer_group(_state), + do: raise("Consumer Group is not supported in 0.8.0 version of kafka") + + def kafka_server_consumer_group_metadata(_state), + do: + raise( + "Consumer Group Metadata is not supported in 0.8.0 version of kafka" + ) + + def kafka_server_join_group(_, _, _state), + do: raise("Join Group is not supported in 0.8.0 version of kafka") + + def kafka_server_sync_group(_, _, _state), + do: raise("Sync Group is not supported in 0.8.0 version of kafka") + + def kafka_server_leave_group(_, _, _state), + do: raise("Leave Group is not supported in 0.8.0 version of Kafka") + + def kafka_server_heartbeat(_, _, _state), + do: raise("Heartbeat is not supported in 0.8.0 version of kafka") + + def kafka_server_update_consumer_metadata(_state), + do: + raise( + "Consumer Group Metadata is not supported in 0.8.0 version of kafka" + ) + + def kafka_api_versions(_state), + do: raise("ApiVersions is not supported in 0.8.0 version of kafka") + + def kafka_create_topics(_, _, _state), + do: raise("CreateTopic is not supported in 0.8.0 version of kafka") defp fetch(request, state) do case network_request(request, Fetch, state) do diff --git a/lib/kafka_ex/server_0_p_8_p_2.ex b/lib/kafka_ex/server_0_p_8_p_2.ex index ae000e5c..6b8cc2ac 100644 --- a/lib/kafka_ex/server_0_p_8_p_2.ex +++ b/lib/kafka_ex/server_0_p_8_p_2.ex @@ -32,8 +32,9 @@ defmodule KafkaEx.Server0P8P2 do def start_link(args, :no_name) do GenServer.start_link(__MODULE__, [args]) end + def start_link(args, name) do - GenServer.start_link(__MODULE__, [args, name], [name: name]) + GenServer.start_link(__MODULE__, [args, name], name: name) end def kafka_server_init([args]) do @@ -42,18 +43,37 @@ defmodule KafkaEx.Server0P8P2 do def kafka_server_init([args, name]) do uris = Keyword.get(args, :uris, []) - metadata_update_interval = Keyword.get(args, :metadata_update_interval, @metadata_update_interval) - consumer_group_update_interval = Keyword.get(args, :consumer_group_update_interval, @consumer_group_update_interval) + + metadata_update_interval = + Keyword.get(args, :metadata_update_interval, @metadata_update_interval) + + consumer_group_update_interval = + Keyword.get( + args, + :consumer_group_update_interval, + @consumer_group_update_interval + ) # this should have already been validated, but it's possible someone could # try to short-circuit the start call consumer_group = Keyword.get(args, :consumer_group) + unless KafkaEx.valid_consumer_group?(consumer_group) do raise InvalidConsumerGroupError, consumer_group end - brokers = Enum.map(uris, fn({host, port}) -> %Broker{host: host, port: port, socket: NetworkClient.create_socket(host, port)} end) - {correlation_id, metadata} = retrieve_metadata(brokers, 0, config_sync_timeout()) + brokers = + Enum.map(uris, fn {host, port} -> + %Broker{ + host: host, + port: port, + socket: NetworkClient.create_socket(host, port) + } + end) + + {correlation_id, metadata} = + retrieve_metadata(brokers, 0, config_sync_timeout()) + state = %State{ metadata: metadata, brokers: brokers, @@ -64,19 +84,27 @@ defmodule KafkaEx.Server0P8P2 do worker_name: name, api_versions: [:unsupported] } + # Get the initial "real" broker list and start a regular refresh cycle. state = update_metadata(state) - {:ok, _} = :timer.send_interval(state.metadata_update_interval, :update_metadata) + + {:ok, _} = + :timer.send_interval(state.metadata_update_interval, :update_metadata) if consumer_group?(state) do # If we are using consumer groups then initialize the state and start the update cycle {_, updated_state} = update_consumer_metadata(state) - {:ok, _} = :timer.send_interval(state.consumer_group_update_interval, :update_consumer_metadata) + + {:ok, _} = + :timer.send_interval( + state.consumer_group_update_interval, + :update_consumer_metadata + ) + {:ok, updated_state} else {:ok, state} end - end def kafka_server_consumer_group(state) do @@ -102,24 +130,33 @@ defmodule KafkaEx.Server0P8P2 do consumer_group = offset_fetch.consumer_group || state.consumer_group offset_fetch = %{offset_fetch | consumer_group: consumer_group} - offset_fetch_request = OffsetFetch.create_request(state.correlation_id, @client_id, offset_fetch) - - {response, state} = case broker do - nil -> - Logger.log(:error, "Coordinator for topic #{offset_fetch.topic} is not available") - {:topic_not_found, state} - _ -> - response = broker - |> NetworkClient.send_sync_request( - offset_fetch_request, - config_sync_timeout()) - |> case do - {:error, reason} -> {:error, reason} - response -> OffsetFetch.parse_response(response) - end - - {response, %{state | correlation_id: state.correlation_id + 1}} - end + offset_fetch_request = + OffsetFetch.create_request(state.correlation_id, @client_id, offset_fetch) + + {response, state} = + case broker do + nil -> + Logger.log( + :error, + "Coordinator for topic #{offset_fetch.topic} is not available" + ) + + {:topic_not_found, state} + + _ -> + response = + broker + |> NetworkClient.send_sync_request( + offset_fetch_request, + config_sync_timeout() + ) + |> case do + {:error, reason} -> {:error, reason} + response -> OffsetFetch.parse_response(response) + end + + {response, %{state | correlation_id: state.correlation_id + 1}} + end {:reply, response, state} end @@ -144,45 +181,93 @@ defmodule KafkaEx.Server0P8P2 do {:noreply, state} end - def kafka_server_join_group(_, _, _state), do: raise "Join Group is not supported in 0.8.2 version of kafka" - def kafka_server_sync_group(_, _, _state), do: raise "Sync Group is not supported in 0.8.2 version of kafka" - def kafka_server_leave_group(_, _, _state), do: raise "Leave Group is not supported in 0.8.2 version of Kafka" - def kafka_server_heartbeat(_, _, _state), do: raise "Heartbeat is not supported in 0.8.2 version of kafka" - def kafka_api_versions(_state), do: raise "ApiVersions is not supported in 0.8.2 version of kafka" - def kafka_create_topics(_, _, _state), do: raise "CreateTopic is not supported in 0.8.2 version of kafka" + def kafka_server_join_group(_, _, _state), + do: raise("Join Group is not supported in 0.8.2 version of kafka") + + def kafka_server_sync_group(_, _, _state), + do: raise("Sync Group is not supported in 0.8.2 version of kafka") + + def kafka_server_leave_group(_, _, _state), + do: raise("Leave Group is not supported in 0.8.2 version of Kafka") - defp update_consumer_metadata(state), do: update_consumer_metadata(state, @retry_count, 0) + def kafka_server_heartbeat(_, _, _state), + do: raise("Heartbeat is not supported in 0.8.2 version of kafka") + + def kafka_api_versions(_state), + do: raise("ApiVersions is not supported in 0.8.2 version of kafka") + + def kafka_create_topics(_, _, _state), + do: raise("CreateTopic is not supported in 0.8.2 version of kafka") + + defp update_consumer_metadata(state), + do: update_consumer_metadata(state, @retry_count, 0) + + defp update_consumer_metadata( + %State{consumer_group: consumer_group} = state, + 0, + error_code + ) do + Logger.log( + :error, + "Fetching consumer_group #{consumer_group} metadata failed with error_code #{ + inspect(error_code) + }" + ) - defp update_consumer_metadata(%State{consumer_group: consumer_group} = state, 0, error_code) do - Logger.log(:error, "Fetching consumer_group #{consumer_group} metadata failed with error_code #{inspect error_code}") {%ConsumerMetadataResponse{error_code: error_code}, state} end - defp update_consumer_metadata(%State{consumer_group: consumer_group, correlation_id: correlation_id} = state, retry, _error_code) do - response = correlation_id + defp update_consumer_metadata( + %State{consumer_group: consumer_group, correlation_id: correlation_id} = + state, + retry, + _error_code + ) do + response = + correlation_id |> ConsumerMetadata.create_request(@client_id, consumer_group) |> first_broker_response(state) - |> ConsumerMetadata.parse_response + |> ConsumerMetadata.parse_response() case response.error_code do - :no_error -> {response, %{state | consumer_metadata: response, correlation_id: state.correlation_id + 1}} - _ -> :timer.sleep(400) - update_consumer_metadata(%{state | correlation_id: state.correlation_id + 1}, retry - 1, response.error_code) + :no_error -> + {response, + %{ + state + | consumer_metadata: response, + correlation_id: state.correlation_id + 1 + }} + + _ -> + :timer.sleep(400) + + update_consumer_metadata( + %{state | correlation_id: state.correlation_id + 1}, + retry - 1, + response.error_code + ) end end defp fetch(request, state) do true = consumer_group_if_auto_commit?(request.auto_commit, state) + case network_request(request, Fetch, state) do - {{:error, error}, state_out} -> {error, state_out} + {{:error, error}, state_out} -> + {error, state_out} + {response, state_out} -> - last_offset = response |> hd |> Map.get(:partitions) |> hd |> Map.get(:last_offset) + last_offset = + response |> hd |> Map.get(:partitions) |> hd |> Map.get(:last_offset) + if last_offset != nil && request.auto_commit do offset_commit_request = %OffsetCommit.Request{ topic: request.topic, offset: last_offset, partition: request.partition, - consumer_group: state_out.consumer_group} + consumer_group: state_out.consumer_group + } + {_, state} = offset_commit(state_out, offset_commit_request) {response, state} else @@ -196,36 +281,59 @@ defmodule KafkaEx.Server0P8P2 do # if the request has a specific consumer group, use that # otherwise use the worker's consumer group - consumer_group = offset_commit_request.consumer_group || state.consumer_group - offset_commit_request = %{offset_commit_request | consumer_group: consumer_group} + consumer_group = + offset_commit_request.consumer_group || state.consumer_group + + offset_commit_request = %{ + offset_commit_request + | consumer_group: consumer_group + } - offset_commit_request_payload = OffsetCommit.create_request(state.correlation_id, @client_id, offset_commit_request) + offset_commit_request_payload = + OffsetCommit.create_request( + state.correlation_id, + @client_id, + offset_commit_request + ) - response = broker + response = + broker |> NetworkClient.send_sync_request( - offset_commit_request_payload, - config_sync_timeout()) + offset_commit_request_payload, + config_sync_timeout() + ) |> case do - {:error, reason} -> {:error, reason} - response -> OffsetCommit.parse_response(response) - end + {:error, reason} -> {:error, reason} + response -> OffsetCommit.parse_response(response) + end {response, %{state | correlation_id: state.correlation_id + 1}} end defp broker_for_consumer_group(state) do - ConsumerMetadataResponse.broker_for_consumer_group(state.brokers, state.consumer_metadata) + ConsumerMetadataResponse.broker_for_consumer_group( + state.brokers, + state.consumer_metadata + ) end # refactored from two versions, one that used the first broker as valid answer, hence # the optional extra flag to do that. Wraps broker_for_consumer_group with an update # call if no broker was found. - defp broker_for_consumer_group_with_update(state, use_first_as_default \\ false) do + defp broker_for_consumer_group_with_update( + state, + use_first_as_default \\ false + ) do case broker_for_consumer_group(state) do nil -> {_, updated_state} = update_consumer_metadata(state) - default_broker = if use_first_as_default, do: hd(state.brokers), else: nil - {broker_for_consumer_group(updated_state) || default_broker, updated_state} + + default_broker = + if use_first_as_default, do: hd(state.brokers), else: nil + + {broker_for_consumer_group(updated_state) || default_broker, + updated_state} + broker -> {broker, state} end diff --git a/lib/kafka_ex/server_0_p_9_p_0.ex b/lib/kafka_ex/server_0_p_9_p_0.ex index 87f6336a..6aabca42 100644 --- a/lib/kafka_ex/server_0_p_9_p_0.ex +++ b/lib/kafka_ex/server_0_p_9_p_0.ex @@ -33,8 +33,9 @@ defmodule KafkaEx.Server0P9P0 do def start_link(args, :no_name) do GenServer.start_link(__MODULE__, [args]) end + def start_link(args, name) do - GenServer.start_link(__MODULE__, [args, name], [name: name]) + GenServer.start_link(__MODULE__, [args, name], name: name) end # The functions below are all defined in KafkaEx.Server0P8P2 and their @@ -43,11 +44,16 @@ defmodule KafkaEx.Server0P9P0 do defdelegate kafka_server_consumer_group(state), to: Server0P8P2 defdelegate kafka_server_fetch(fetch_request, state), to: Server0P8P2 defdelegate kafka_server_offset_fetch(offset_fetch, state), to: Server0P8P2 - defdelegate kafka_server_offset_commit(offset_commit_request, state), to: Server0P8P2 + + defdelegate kafka_server_offset_commit(offset_commit_request, state), + to: Server0P8P2 + defdelegate kafka_server_consumer_group_metadata(state), to: Server0P8P2 defdelegate kafka_server_update_consumer_metadata(state), to: Server0P8P2 defdelegate kafka_api_versions(state), to: Server0P8P2 - defdelegate kafka_create_topics(requests, network_timeout, state), to: Server0P8P2 + + defdelegate kafka_create_topics(requests, network_timeout, state), + to: Server0P8P2 def kafka_server_init([args]) do kafka_server_init([args, self()]) @@ -55,12 +61,21 @@ defmodule KafkaEx.Server0P9P0 do def kafka_server_init([args, name]) do uris = Keyword.get(args, :uris, []) - metadata_update_interval = Keyword.get(args, :metadata_update_interval, @metadata_update_interval) - consumer_group_update_interval = Keyword.get(args, :consumer_group_update_interval, @consumer_group_update_interval) + + metadata_update_interval = + Keyword.get(args, :metadata_update_interval, @metadata_update_interval) + + consumer_group_update_interval = + Keyword.get( + args, + :consumer_group_update_interval, + @consumer_group_update_interval + ) # this should have already been validated, but it's possible someone could # try to short-circuit the start call consumer_group = Keyword.get(args, :consumer_group) + unless KafkaEx.valid_consumer_group?(consumer_group) do raise InvalidConsumerGroupError, consumer_group end @@ -68,8 +83,18 @@ defmodule KafkaEx.Server0P9P0 do use_ssl = Keyword.get(args, :use_ssl, false) ssl_options = Keyword.get(args, :ssl_options, []) - brokers = Enum.map(uris, fn({host, port}) -> %Broker{host: host, port: port, socket: NetworkClient.create_socket(host, port, ssl_options, use_ssl)} end) - {correlation_id, metadata} = retrieve_metadata(brokers, 0, config_sync_timeout()) + brokers = + Enum.map(uris, fn {host, port} -> + %Broker{ + host: host, + port: port, + socket: NetworkClient.create_socket(host, port, ssl_options, use_ssl) + } + end) + + {correlation_id, metadata} = + retrieve_metadata(brokers, 0, config_sync_timeout()) + state = %State{ metadata: metadata, brokers: brokers, @@ -82,15 +107,24 @@ defmodule KafkaEx.Server0P9P0 do use_ssl: use_ssl, api_versions: [:unsupported] } + # Get the initial "real" broker list and start a regular refresh cycle. state = update_metadata(state) - {:ok, _} = :timer.send_interval(state.metadata_update_interval, :update_metadata) + + {:ok, _} = + :timer.send_interval(state.metadata_update_interval, :update_metadata) state = if consumer_group?(state) do # If we are using consumer groups then initialize the state and start the update cycle {_, updated_state} = update_consumer_metadata(state) - {:ok, _} = :timer.send_interval(state.consumer_group_update_interval, :update_consumer_metadata) + + {:ok, _} = + :timer.send_interval( + state.consumer_group_update_interval, + :update_consumer_metadata + ) + updated_state else state @@ -100,55 +134,59 @@ defmodule KafkaEx.Server0P9P0 do end def kafka_server_join_group(request, network_timeout, state_in) do - {response, state_out} = consumer_group_sync_request( - request, - JoinGroup, - network_timeout, - state_in - ) + {response, state_out} = + consumer_group_sync_request( + request, + JoinGroup, + network_timeout, + state_in + ) {:reply, response, state_out} end def kafka_server_sync_group(request, network_timeout, state_in) do - {response, state_out} = consumer_group_sync_request( - request, - SyncGroup, - network_timeout, - state_in - ) + {response, state_out} = + consumer_group_sync_request( + request, + SyncGroup, + network_timeout, + state_in + ) {:reply, response, state_out} end def kafka_server_leave_group(request, network_timeout, state_in) do - {response, state_out} = consumer_group_sync_request( - request, - LeaveGroup, - network_timeout, - state_in - ) + {response, state_out} = + consumer_group_sync_request( + request, + LeaveGroup, + network_timeout, + state_in + ) {:reply, response, state_out} end def kafka_server_heartbeat(request, network_timeout, state_in) do - {response, state_out} = consumer_group_sync_request( - request, - Heartbeat, - network_timeout, - state_in - ) + {response, state_out} = + consumer_group_sync_request( + request, + Heartbeat, + network_timeout, + state_in + ) {:reply, response, state_out} end defp consumer_group_sync_request( - request, - protocol_module, - network_timeout, - state - ) do + request, + protocol_module, + network_timeout, + state + ) do unless consumer_group?(state) do raise ConsumerGroupRequiredError, request end @@ -159,24 +197,30 @@ defmodule KafkaEx.Server0P9P0 do sync_timeout = config_sync_timeout(network_timeout) - wire_request = protocol_module.create_request( - state.correlation_id, - @client_id, - request - ) - wire_response = NetworkClient.send_sync_request( - broker, - wire_request, - sync_timeout - ) + wire_request = + protocol_module.create_request( + state.correlation_id, + @client_id, + request + ) + + wire_response = + NetworkClient.send_sync_request( + broker, + wire_request, + sync_timeout + ) case wire_response do - {:error, reason} -> {{:error, reason}, state_out} + {:error, reason} -> + {{:error, reason}, state_out} + _ -> response = protocol_module.parse_response(wire_response) if response.error_code == :not_coordinator_for_consumer do {_, updated_state_out} = update_consumer_metadata(state_out) + consumer_group_sync_request( request, protocol_module, @@ -189,30 +233,50 @@ defmodule KafkaEx.Server0P9P0 do end end - defp update_consumer_metadata(state), do: update_consumer_metadata(state, @retry_count, 0) + defp update_consumer_metadata(state), + do: update_consumer_metadata(state, @retry_count, 0) + + defp update_consumer_metadata( + %State{consumer_group: consumer_group} = state, + 0, + error_code + ) do + Logger.log( + :error, + "Fetching consumer_group #{consumer_group} metadata failed with error_code #{ + inspect(error_code) + }" + ) - defp update_consumer_metadata(%State{consumer_group: consumer_group} = state, 0, error_code) do - Logger.log(:error, "Fetching consumer_group #{consumer_group} metadata failed with error_code #{inspect error_code}") {%ConsumerMetadataResponse{error_code: error_code}, state} end - defp update_consumer_metadata(%State{consumer_group: consumer_group, correlation_id: correlation_id} = state, retry, _error_code) do - response = correlation_id + defp update_consumer_metadata( + %State{consumer_group: consumer_group, correlation_id: correlation_id} = + state, + retry, + _error_code + ) do + response = + correlation_id |> ConsumerMetadata.create_request(@client_id, consumer_group) |> first_broker_response(state) - |> ConsumerMetadata.parse_response + |> ConsumerMetadata.parse_response() case response.error_code do :no_error -> { response, %{ - state | - consumer_metadata: response, - correlation_id: state.correlation_id + 1 + state + | consumer_metadata: response, + correlation_id: state.correlation_id + 1 } } - _ -> :timer.sleep(400) + + _ -> + :timer.sleep(400) + update_consumer_metadata( %{state | correlation_id: state.correlation_id + 1}, retry - 1, @@ -222,18 +286,29 @@ defmodule KafkaEx.Server0P9P0 do end defp broker_for_consumer_group(state) do - ConsumerMetadataResponse.broker_for_consumer_group(state.brokers, state.consumer_metadata) + ConsumerMetadataResponse.broker_for_consumer_group( + state.brokers, + state.consumer_metadata + ) end # refactored from two versions, one that used the first broker as valid answer, hence # the optional extra flag to do that. Wraps broker_for_consumer_group with an update # call if no broker was found. - def broker_for_consumer_group_with_update(state, use_first_as_default \\ false) do + def broker_for_consumer_group_with_update( + state, + use_first_as_default \\ false + ) do case broker_for_consumer_group(state) do nil -> {_, updated_state} = update_consumer_metadata(state) - default_broker = if use_first_as_default, do: hd(state.brokers), else: nil - {broker_for_consumer_group(updated_state) || default_broker, updated_state} + + default_broker = + if use_first_as_default, do: hd(state.brokers), else: nil + + {broker_for_consumer_group(updated_state) || default_broker, + updated_state} + broker -> {broker, state} end @@ -245,7 +320,6 @@ defmodule KafkaEx.Server0P9P0 do def consumer_group?(%State{consumer_group: :no_consumer_group}), do: false def consumer_group?(_), do: true - defp first_broker_response(request, state) do first_broker_response(request, state.brokers, config_sync_timeout()) end diff --git a/lib/kafka_ex/socket.ex b/lib/kafka_ex/socket.ex index 273a3f9e..9d9bc77f 100644 --- a/lib/kafka_ex/socket.ex +++ b/lib/kafka_ex/socket.ex @@ -4,7 +4,11 @@ defmodule KafkaEx.Socket do """ defstruct socket: nil, ssl: false - @type t :: %KafkaEx.Socket{socket: :gen_tcp.socket | :ssl.sslsocket, ssl: boolean} + + @type t :: %KafkaEx.Socket{ + socket: :gen_tcp.socket() | :ssl.sslsocket(), + ssl: boolean + } @doc """ Creates a socket. @@ -12,7 +16,8 @@ defmodule KafkaEx.Socket do For more information about the available options, see `:ssl.connect/3` for ssl or `:gen_tcp.connect/3` for non ssl. """ - @spec create(:inet.ip_address, non_neg_integer, [] | [...]) :: {:ok, KafkaEx.Socket.t} | {:error, any} + @spec create(:inet.ip_address(), non_neg_integer, [] | [...]) :: + {:ok, KafkaEx.Socket.t()} | {:error, any} def create(host, port, socket_options \\ [], is_ssl \\ false) do case create_socket(host, port, is_ssl, socket_options) do {:ok, socket} -> {:ok, %KafkaEx.Socket{socket: socket, ssl: is_ssl}} @@ -25,7 +30,7 @@ defmodule KafkaEx.Socket do For more information, see `:ssl.close/1` for ssl or `:gen_tcp.send/1` for non ssl. """ - @spec close(KafkaEx.Socket.t) :: :ok + @spec close(KafkaEx.Socket.t()) :: :ok def close(%KafkaEx.Socket{ssl: true} = socket), do: :ssl.close(socket.socket) def close(socket), do: :gen_tcp.close(socket.socket) @@ -36,10 +41,11 @@ defmodule KafkaEx.Socket do For more information, see `:ssl.send/2` for ssl or `:gen_tcp.send/2` for non ssl. """ - @spec send(KafkaEx.Socket.t, iodata) :: :ok | {:error, any} + @spec send(KafkaEx.Socket.t(), iodata) :: :ok | {:error, any} def send(%KafkaEx.Socket{ssl: true} = socket, data) do :ssl.send(socket.socket, data) end + def send(socket, data) do :gen_tcp.send(socket.socket, data) end @@ -49,10 +55,11 @@ defmodule KafkaEx.Socket do For more information, see `:ssl.setopts/2` for ssl or `:inet.setopts/2` for non ssl. """ - @spec setopts(KafkaEx.Socket.t, list) :: :ok | {:error, any} + @spec setopts(KafkaEx.Socket.t(), list) :: :ok | {:error, any} def setopts(%KafkaEx.Socket{ssl: true} = socket, options) do :ssl.setopts(socket.socket, options) end + def setopts(socket, options) do :inet.setopts(socket.socket, options) end @@ -62,18 +69,22 @@ defmodule KafkaEx.Socket do For more information, see `:ssl.recv/2` for ssl or `:gen_tcp.recv/2` for non ssl. """ - @spec recv(KafkaEx.Socket.t, non_neg_integer) :: {:ok, String.t | binary | term} | {:error, any} + @spec recv(KafkaEx.Socket.t(), non_neg_integer) :: + {:ok, String.t() | binary | term} | {:error, any} def recv(%KafkaEx.Socket{ssl: true} = socket, length) do :ssl.recv(socket.socket, length) end + def recv(socket, length) do :gen_tcp.recv(socket.socket, length) end - @spec recv(KafkaEx.Socket.t, non_neg_integer, timeout) :: {:ok, String.t | binary | term} | {:error, any} + @spec recv(KafkaEx.Socket.t(), non_neg_integer, timeout) :: + {:ok, String.t() | binary | term} | {:error, any} def recv(%KafkaEx.Socket{ssl: true} = socket, length, timeout) do :ssl.recv(socket.socket, length, timeout) end + def recv(socket, length, timeout) do :gen_tcp.recv(socket.socket, length, timeout) end @@ -81,7 +92,7 @@ defmodule KafkaEx.Socket do @doc """ Returns true if the socket is open """ - @spec open?(KafkaEx.Socket.t) :: boolean + @spec open?(KafkaEx.Socket.t()) :: boolean def open?(%KafkaEx.Socket{} = socket) do info(socket) != nil end @@ -91,22 +102,24 @@ defmodule KafkaEx.Socket do For more information, see `Port.info` """ - @spec info(KafkaEx.Socket.t) :: list | nil + @spec info(KafkaEx.Socket.t()) :: list | nil def info(socket) do socket |> extract_port - |> Port.info + |> Port.info() end defp extract_port(%KafkaEx.Socket{ssl: true} = socket) do {:sslsocket, {:gen_tcp, port, _, _}, _} = socket.socket port end + defp extract_port(socket), do: socket.socket defp create_socket(host, port, true, socket_options) do :ssl.connect(host, port, socket_options) end + defp create_socket(host, port, false, socket_options) do :gen_tcp.connect(host, port, socket_options) end diff --git a/lib/kafka_ex/stream.ex b/lib/kafka_ex/stream.ex index cdf1cf57..3b22591a 100644 --- a/lib/kafka_ex/stream.ex +++ b/lib/kafka_ex/stream.ex @@ -6,9 +6,9 @@ defmodule KafkaEx.Stream do alias KafkaEx.Protocol.Fetch.Response, as: FetchResponse defstruct worker_name: nil, - fetch_request: %FetchRequest{}, - consumer_group: nil, - no_wait_at_logend: false + fetch_request: %FetchRequest{}, + consumer_group: nil, + no_wait_at_logend: false @type t :: %__MODULE__{} @@ -31,7 +31,7 @@ defmodule KafkaEx.Stream do # there isn't really any cleanup, so we don't need to do anything with # the after_fun callback - after_fun = fn(_last_offset) -> :ok end + after_fun = fn _last_offset -> :ok end Stream.resource(start_fun, next_fun, after_fun).(acc, fun) end @@ -54,23 +54,24 @@ defmodule KafkaEx.Stream do # if we get a response, we return the message set and point at the next # offset after the last message defp stream_control( - %{ - error_code: :no_error, - last_offset: last_offset, - message_set: message_set - }, - _stream_data, - _offset - ) when is_integer(last_offset) do + %{ + error_code: :no_error, + last_offset: last_offset, + message_set: message_set + }, + _stream_data, + _offset + ) + when is_integer(last_offset) do {message_set, last_offset + 1} end # if we don't get any messages and no_wait_at_logend is true, we halt defp stream_control( - %{}, - %KafkaEx.Stream{no_wait_at_logend: true}, - offset - ) do + %{}, + %KafkaEx.Stream{no_wait_at_logend: true}, + offset + ) do {:halt, offset} end @@ -78,6 +79,7 @@ defmodule KafkaEx.Stream do defp stream_control(%{}, %KafkaEx.Stream{}, offset) do {[], offset} end + ###################################################################### ###################################################################### @@ -85,12 +87,12 @@ defmodule KafkaEx.Stream do # first, determine if we even need to commit an offset defp maybe_commit_offset( - fetch_response, - %KafkaEx.Stream{ - fetch_request: %FetchRequest{auto_commit: auto_commit} - } = stream_data, - acc - ) do + fetch_response, + %KafkaEx.Stream{ + fetch_request: %FetchRequest{auto_commit: auto_commit} + } = stream_data, + acc + ) do if need_commit?(fetch_response, auto_commit) do offset_to_commit = last_offset(acc, fetch_response.message_set) commit_offset(stream_data, offset_to_commit) @@ -101,7 +103,9 @@ defmodule KafkaEx.Stream do # no response -> no commit defp need_commit?(fetch_response, _auto_commit) - when fetch_response == %{}, do: false + when fetch_response == %{}, + do: false + # no messages in response -> no commit defp need_commit?(%{message_set: []}, _auto_commit), do: false # otherwise, use the auto_commit setting @@ -110,7 +114,7 @@ defmodule KafkaEx.Stream do # if we have requested fewer messages than we fetched, commit the offset # of the last one we will actually consume defp last_offset({:cont, {_, n}}, message_set) - when n <= length(message_set) do + when n <= length(message_set) do message = Enum.at(message_set, n - 1) message.offset end @@ -128,17 +132,20 @@ defmodule KafkaEx.Stream do consumer_group: stream_data.consumer_group, topic: stream_data.fetch_request.topic, partition: stream_data.fetch_request.partition, - offset: offset, metadata: "" + offset: offset, + metadata: "" } }) end + ###################################################################### # make the actual fetch request defp fetch_response(data, offset) do req = data.fetch_request + data.worker_name - |> GenServer.call({:fetch, %{req| offset: offset}}) + |> GenServer.call({:fetch, %{req | offset: offset}}) |> FetchResponse.partition_messages(req.topic, req.partition) end end diff --git a/lib/kafka_ex/supervisor.ex b/lib/kafka_ex/supervisor.ex index 285aca1d..ce80ec9a 100644 --- a/lib/kafka_ex/supervisor.ex +++ b/lib/kafka_ex/supervisor.ex @@ -4,7 +4,13 @@ defmodule KafkaEx.Supervisor do use Supervisor def start_link(server, max_restarts, max_seconds) do - {:ok, pid} = Supervisor.start_link(__MODULE__, [server, max_restarts, max_seconds], [name: __MODULE__]) + {:ok, pid} = + Supervisor.start_link( + __MODULE__, + [server, max_restarts, max_seconds], + name: __MODULE__ + ) + {:ok, pid} end @@ -20,6 +26,12 @@ defmodule KafkaEx.Supervisor do children = [ worker(server, []) ] - supervise(children, [strategy: :simple_one_for_one, max_restarts: max_restarts, max_seconds: max_seconds]) + + supervise( + children, + strategy: :simple_one_for_one, + max_restarts: max_restarts, + max_seconds: max_seconds + ) end end diff --git a/mix.exs b/mix.exs index ec52d813..c7d13189 100644 --- a/mix.exs +++ b/mix.exs @@ -40,8 +40,7 @@ defmodule KafkaEx.Mixfile do {:excoveralls, "~> 0.7", only: :test}, {:ex_doc, "~> 0.18", only: :dev}, {:snappy, - git: "https://github.com/fdmanana/snappy-erlang-nif", - only: [:dev, :test]} + git: "https://github.com/fdmanana/snappy-erlang-nif", only: [:dev, :test]} ] end @@ -50,9 +49,11 @@ defmodule KafkaEx.Mixfile do end defp package do - [maintainers: ["Abejide Ayodele", "Dan Swain", "Jack Lund"], - files: ["lib", "config/config.exs", "mix.exs", "README.md"], - licenses: ["MIT"], - links: %{"Github" => "https://github.com/kafkaex/kafka_ex"}] + [ + maintainers: ["Abejide Ayodele", "Dan Swain", "Jack Lund"], + files: ["lib", "config/config.exs", "mix.exs", "README.md"], + licenses: ["MIT"], + links: %{"Github" => "https://github.com/kafkaex/kafka_ex"} + ] end end diff --git a/test/compression_test.exs b/test/compression_test.exs index f196d426..ecc5a373 100644 --- a/test/compression_test.exs +++ b/test/compression_test.exs @@ -2,10 +2,18 @@ defmodule CompressionTest do use ExUnit.Case test "snappy decompression works with chunked messages" do - data = <<130, 83, 78, 65, 80, 80, 89, 0, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 0, 14, 12, 44, 0, 0, 0, 0, 0, 0, 0, 37, 0, 0, 3, 246, 0, 0, 0, 75, 246, 7, 92, 10, 44, 16, 236, 0, 0, 255, 255, 255, 255, 0, 0, 3, 232, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 118, 10, 0>> - - expected = <<0, 0, 0, 0, 0, 0, 0, 37, 0, 0, 3, 246, 10, 44, 16, 236, 0, 0, 255, 255, 255, 255, 0, 0, 3, 232 >> <> - String.duplicate("ABCDEFGHIJ", 100) + data = + <<130, 83, 78, 65, 80, 80, 89, 0, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 0, 14, 12, + 44, 0, 0, 0, 0, 0, 0, 0, 37, 0, 0, 3, 246, 0, 0, 0, 75, 246, 7, 92, 10, + 44, 16, 236, 0, 0, 255, 255, 255, 255, 0, 0, 3, 232, 65, 66, 67, 68, 69, + 70, 71, 72, 73, 74, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, + 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, + 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 254, 10, 0, 118, 10, 0>> + + expected = + <<0, 0, 0, 0, 0, 0, 0, 37, 0, 0, 3, 246, 10, 44, 16, 236, 0, 0, 255, 255, + 255, 255, 0, 0, 3, 232>> <> String.duplicate("ABCDEFGHIJ", 100) + assert expected == KafkaEx.Compression.decompress(2, data) end end diff --git a/test/integration/consumer_group_implementation_test.exs b/test/integration/consumer_group_implementation_test.exs index 23de6be3..f5faed63 100644 --- a/test/integration/consumer_group_implementation_test.exs +++ b/test/integration/consumer_group_implementation_test.exs @@ -25,13 +25,13 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do end def calls do - Agent.get(__MODULE__, &(&1)) + Agent.get(__MODULE__, & &1) end def assign_partitions(members, partitions) do Logger.debug(fn -> - "Consumer #{inspect self()} got " <> - "partition assignment: #{inspect members} #{inspect partitions}" + "Consumer #{inspect(self())} got " <> + "partition assignment: #{inspect(members)} #{inspect(partitions)}" end) Agent.update(__MODULE__, &(&1 + 1)) @@ -61,8 +61,9 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do def init(topic, partition) do Logger.debug(fn -> - "Initialized consumer #{inspect self()} for #{topic}:#{partition}" + "Initialized consumer #{inspect(self())} for #{topic}:#{partition}" end) + {:ok, %{message_sets: []}} end @@ -84,8 +85,9 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do def handle_message_set(message_set, state) do Logger.debug(fn -> - "Consumer #{inspect self()} handled message set #{inspect message_set}" + "Consumer #{inspect(self())} handled message set #{inspect(message_set)}" end) + { :async_commit, %{state | message_sets: state.message_sets ++ [message_set]} @@ -100,11 +102,13 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do def correct_last_message?(nil, _, _), do: false def correct_last_message?([], _, _), do: false + def correct_last_message?(message_set, expected_message, expected_offset) do Logger.debug(fn -> - "Got message set: #{inspect message_set} " <> + "Got message set: #{inspect(message_set)} " <> "expecting '#{expected_message}' @ offset #{expected_offset}" end) + message = List.last(message_set) message.value == expected_message && message.offset == expected_offset end @@ -114,34 +118,39 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do if Process.alive?(pid) do Process.exit(pid, :normal) end + !Process.alive?(pid) end) end def num_open_ports() do - :erlang.ports - |> Enum.map(&(:erlang.port_info(&1, :name))) + :erlang.ports() + |> Enum.map(&:erlang.port_info(&1, :name)) |> Enum.filter(&(&1 == {:name, 'tcp_inet'})) |> length end setup do ports_before = num_open_ports() - {:ok, _} = TestPartitioner.start_link - {:ok, consumer_group_pid1} = ConsumerGroup.start_link( - TestConsumer, - @consumer_group_name, - [@topic_name], - heartbeat_interval: 100, - partition_assignment_callback: &TestPartitioner.assign_partitions/2 - ) - {:ok, consumer_group_pid2} = ConsumerGroup.start_link( - TestConsumer, - @consumer_group_name, - [@topic_name], - heartbeat_interval: 100, - partition_assignment_callback: &TestPartitioner.assign_partitions/2 - ) + {:ok, _} = TestPartitioner.start_link() + + {:ok, consumer_group_pid1} = + ConsumerGroup.start_link( + TestConsumer, + @consumer_group_name, + [@topic_name], + heartbeat_interval: 100, + partition_assignment_callback: &TestPartitioner.assign_partitions/2 + ) + + {:ok, consumer_group_pid2} = + ConsumerGroup.start_link( + TestConsumer, + @consumer_group_name, + [@topic_name], + heartbeat_interval: 100, + partition_assignment_callback: &TestPartitioner.assign_partitions/2 + ) # wait for both consumer groups to join wait_for(fn -> @@ -149,10 +158,10 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do ConsumerGroup.active?(consumer_group_pid2) end) - on_exit fn -> + on_exit(fn -> sync_stop(consumer_group_pid1) sync_stop(consumer_group_pid2) - end + end) { :ok, @@ -165,14 +174,14 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do test "basic startup, consume, and shutdown test", context do assert num_open_ports() > context[:ports_before] - assert TestPartitioner.calls > 0 + assert TestPartitioner.calls() > 0 generation_id1 = ConsumerGroup.generation_id(context[:consumer_group_pid1]) generation_id2 = ConsumerGroup.generation_id(context[:consumer_group_pid2]) assert generation_id1 == generation_id2 assert @consumer_group_name == - ConsumerGroup.group_name(context[:consumer_group_pid1]) + ConsumerGroup.group_name(context[:consumer_group_pid1]) member1 = ConsumerGroup.member_id(context[:consumer_group_pid1]) member2 = ConsumerGroup.member_id(context[:consumer_group_pid2]) @@ -186,9 +195,11 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do leader1 == member1 -> assert ConsumerGroup.leader?(context[:consumer_group_pid1]) refute ConsumerGroup.leader?(context[:consumer_group_pid2]) + leader1 == member2 -> refute ConsumerGroup.leader?(context[:consumer_group_pid1]) assert ConsumerGroup.leader?(context[:consumer_group_pid2]) + true -> raise "Neither member is the leader" end @@ -197,26 +208,31 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do assignments2 = ConsumerGroup.assignments(context[:consumer_group_pid2]) assert 2 == length(assignments1) assert 2 == length(assignments2) + assert MapSet.disjoint?( - Enum.into(assignments1, MapSet.new), - Enum.into(assignments2, MapSet.new) - ) + Enum.into(assignments1, MapSet.new()), + Enum.into(assignments2, MapSet.new()) + ) consumer1_pid = ConsumerGroup.consumer_supervisor_pid(context[:consumer_group_pid1]) - consumer1_assignments = consumer1_pid - |> GenConsumer.Supervisor.child_pids - |> Enum.map(&GenConsumer.partition/1) - |> Enum.sort + + consumer1_assignments = + consumer1_pid + |> GenConsumer.Supervisor.child_pids() + |> Enum.map(&GenConsumer.partition/1) + |> Enum.sort() assert consumer1_assignments == Enum.sort(assignments1) consumer2_pid = ConsumerGroup.consumer_supervisor_pid(context[:consumer_group_pid2]) - consumer2_assignments = consumer2_pid - |> GenConsumer.Supervisor.child_pids - |> Enum.map(&GenConsumer.partition/1) - |> Enum.sort + + consumer2_assignments = + consumer2_pid + |> GenConsumer.Supervisor.child_pids() + |> Enum.map(&GenConsumer.partition/1) + |> Enum.sort() assert consumer2_assignments == Enum.sort(assignments2) @@ -225,34 +241,40 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do partition_range = 0..(@partition_count - 1) - starting_offsets = partition_range - |> Enum.map(fn(px) -> {px, latest_offset_number(@topic_name, px)} end) - |> Enum.into(%{}) + starting_offsets = + partition_range + |> Enum.map(fn px -> {px, latest_offset_number(@topic_name, px)} end) + |> Enum.into(%{}) - messages = partition_range - |> Enum.map(fn(px) -> - offset = Map.get(starting_offsets, px) - {px, produce("M #{px} #{offset}", px)} - end) - |> Enum.into(%{}) + messages = + partition_range + |> Enum.map(fn px -> + offset = Map.get(starting_offsets, px) + {px, produce("M #{px} #{offset}", px)} + end) + |> Enum.into(%{}) - consumers = Map.merge( - ConsumerGroup.partition_consumer_map(context[:consumer_group_pid1]), - ConsumerGroup.partition_consumer_map(context[:consumer_group_pid2]) - ) + consumers = + Map.merge( + ConsumerGroup.partition_consumer_map(context[:consumer_group_pid1]), + ConsumerGroup.partition_consumer_map(context[:consumer_group_pid2]) + ) # we actually consume the messages - last_offsets = partition_range - |> Enum.map(fn(px) -> - consumer_pid = Map.get(consumers, {@topic_name, px}) - wait_for(fn -> - message_set = TestConsumer.last_message_set(consumer_pid) - correct_last_message?(message_set, messages[px], starting_offsets[px]) + last_offsets = + partition_range + |> Enum.map(fn px -> + consumer_pid = Map.get(consumers, {@topic_name, px}) + + wait_for(fn -> + message_set = TestConsumer.last_message_set(consumer_pid) + correct_last_message?(message_set, messages[px], starting_offsets[px]) + end) + + last_message = List.last(TestConsumer.last_message_set(consumer_pid)) + {px, last_message.offset} end) - last_message = List.last(TestConsumer.last_message_set(consumer_pid)) - {px, last_message.offset} - end) - |> Enum.into(%{}) + |> Enum.into(%{}) # stop the supervisors Process.unlink(context[:consumer_group_pid1]) @@ -265,6 +287,7 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do wait_for(fn -> ending_offset = latest_consumer_offset_number(@topic_name, px, @consumer_group_name) + last_offset = Map.get(last_offsets, px) ending_offset == last_offset + 1 end) @@ -289,13 +312,14 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do # and become the leader assert ConsumerGroup.leader?(context[:consumer_group_pid2]) - {:ok, consumer_group_pid3} = ConsumerGroup.start_link( - TestConsumer, - @consumer_group_name, - [@topic_name], - heartbeat_interval: 100, - partition_assignment_callback: &TestPartitioner.assign_partitions/2 - ) + {:ok, consumer_group_pid3} = + ConsumerGroup.start_link( + TestConsumer, + @consumer_group_name, + [@topic_name], + heartbeat_interval: 100, + partition_assignment_callback: &TestPartitioner.assign_partitions/2 + ) # the new worker should get assigned some partitions wait_for(fn -> @@ -322,6 +346,7 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do test "handle_cast and handle_info calls", context do consumer_group_pid = ConsumerGroup.consumer_supervisor_pid(context[:consumer_group_pid1]) + consumer_pids = GenConsumer.Supervisor.child_pids(consumer_group_pid) # Send a cast and info message to each consumer @@ -335,10 +360,13 @@ defmodule KafkaEx.ConsumerGroupImplementationTest do wait_for(fn -> TestConsumer.get(consumer_pid, :test_cast) != nil end) + assert :value == TestConsumer.get(consumer_pid, :test_cast) + wait_for(fn -> TestConsumer.get(consumer_pid, :test_info) != nil end) + assert :value == TestConsumer.get(consumer_pid, :test_info) end end diff --git a/test/integration/consumer_group_test.exs b/test/integration/consumer_group_test.exs index 359587f4..e608857c 100644 --- a/test/integration/consumer_group_test.exs +++ b/test/integration/consumer_group_test.exs @@ -8,37 +8,50 @@ defmodule KafkaEx.ConsumerGroup.Test do test "fetching the consumer group from the default worker" do assert Application.get_env(:kafka_ex, :consumer_group) == - KafkaEx.consumer_group() + KafkaEx.consumer_group() end test "create_worker returns an error when an invalid consumer group is provided" do - assert {:error, :invalid_consumer_group} == KafkaEx.create_worker(:francine, consumer_group: 0) + assert {:error, :invalid_consumer_group} == + KafkaEx.create_worker(:francine, consumer_group: 0) end test "create_worker allows us to disable the consumer group" do - {:ok, pid} = KafkaEx.create_worker(:barney, consumer_group: :no_consumer_group) + {:ok, pid} = + KafkaEx.create_worker(:barney, consumer_group: :no_consumer_group) consumer_group = :sys.get_state(pid).consumer_group assert consumer_group == :no_consumer_group end test "create_worker allows us to provide a consumer group" do - {:ok, pid} = KafkaEx.create_worker(:bah, consumer_group: "my_consumer_group") + {:ok, pid} = + KafkaEx.create_worker(:bah, consumer_group: "my_consumer_group") + consumer_group = :sys.get_state(pid).consumer_group assert consumer_group == "my_consumer_group" end test "create_worker allows custom consumer_group_update_interval" do - {:ok, pid} = KafkaEx.create_worker(:consumer_group_update_interval_custom, uris: uris(), consumer_group_update_interval: 10) - consumer_group_update_interval = :sys.get_state(pid).consumer_group_update_interval + {:ok, pid} = + KafkaEx.create_worker( + :consumer_group_update_interval_custom, + uris: uris(), + consumer_group_update_interval: 10 + ) + + consumer_group_update_interval = + :sys.get_state(pid).consumer_group_update_interval assert consumer_group_update_interval == 10 end test "create_worker provides a default consumer_group_update_interval of '30000'" do {:ok, pid} = KafkaEx.create_worker(:de, uris: uris()) - consumer_group_update_interval = :sys.get_state(pid).consumer_group_update_interval + + consumer_group_update_interval = + :sys.get_state(pid).consumer_group_update_interval assert consumer_group_update_interval == 30000 end @@ -51,7 +64,9 @@ defmodule KafkaEx.ConsumerGroup.Test do end test "create_worker takes a consumer_group option and sets that as the consumer_group of the worker" do - {:ok, pid} = KafkaEx.create_worker(:joe, [uris: uris(), consumer_group: "foo"]) + {:ok, pid} = + KafkaEx.create_worker(:joe, uris: uris(), consumer_group: "foo") + consumer_group = :sys.get_state(pid).consumer_group assert consumer_group == "foo" @@ -60,19 +75,42 @@ defmodule KafkaEx.ConsumerGroup.Test do test "asking the worker for the name of its consumer group" do consumer_group = "this_is_my_consumer_group" worker_name = :consumer_group_reader_test - {:ok, _pid} = KafkaEx.create_worker(worker_name, - consumer_group: consumer_group) + + {:ok, _pid} = + KafkaEx.create_worker( + worker_name, + consumer_group: consumer_group + ) assert consumer_group == KafkaEx.consumer_group(worker_name) end test "consumer_group_metadata works" do random_string = generate_random_string() - KafkaEx.produce(%Proto.Produce.Request{topic: "food", partition: 0, required_acks: 1, messages: [%Proto.Produce.Message{value: "hey"}]}) + + KafkaEx.produce(%Proto.Produce.Request{ + topic: "food", + partition: 0, + required_acks: 1, + messages: [%Proto.Produce.Message{value: "hey"}] + }) + KafkaEx.fetch("food", 0, offset: 0) - KafkaEx.create_worker(:consumer_group_metadata_worker, consumer_group: random_string, uris: Application.get_env(:kafka_ex, :brokers)) + + KafkaEx.create_worker( + :consumer_group_metadata_worker, + consumer_group: random_string, + uris: Application.get_env(:kafka_ex, :brokers) + ) + pid = Process.whereis(:consumer_group_metadata_worker) - metadata = KafkaEx.consumer_group_metadata(:consumer_group_metadata_worker, random_string) + + metadata = + KafkaEx.consumer_group_metadata( + :consumer_group_metadata_worker, + random_string + ) + consumer_group_metadata = :sys.get_state(pid).consumer_metadata assert metadata != %Proto.ConsumerMetadata.Response{} @@ -81,13 +119,22 @@ defmodule KafkaEx.ConsumerGroup.Test do assert metadata == consumer_group_metadata end - #update_consumer_metadata + # update_consumer_metadata test "worker updates metadata after specified interval" do - {:ok, pid} = KafkaEx.create_worker(:update_consumer_metadata, [uris: uris(), consumer_group: "kafka_ex", consumer_group_update_interval: 100]) + {:ok, pid} = + KafkaEx.create_worker( + :update_consumer_metadata, + uris: uris(), + consumer_group: "kafka_ex", + consumer_group_update_interval: 100 + ) + consumer_metadata = %KafkaEx.Protocol.ConsumerMetadata.Response{} - :sys.replace_state(pid, fn(state) -> + + :sys.replace_state(pid, fn state -> %{state | :consumer_metadata => consumer_metadata} end) + :timer.sleep(105) new_consumer_metadata = :sys.get_state(pid).consumer_metadata @@ -95,43 +142,65 @@ defmodule KafkaEx.ConsumerGroup.Test do end test "worker does not update metadata when consumer_group is disabled" do - {:ok, pid} = KafkaEx.create_worker(:no_consumer_metadata_update, [uris: uris(), consumer_group: :no_consumer_group, consumer_group_update_interval: 100]) + {:ok, pid} = + KafkaEx.create_worker( + :no_consumer_metadata_update, + uris: uris(), + consumer_group: :no_consumer_group, + consumer_group_update_interval: 100 + ) + consumer_metadata = %KafkaEx.Protocol.ConsumerMetadata.Response{} - :sys.replace_state(pid, fn(state) -> + + :sys.replace_state(pid, fn state -> %{state | :consumer_metadata => consumer_metadata} end) + :timer.sleep(105) new_consumer_metadata = :sys.get_state(pid).consumer_metadata assert new_consumer_metadata == consumer_metadata end - #fetch + # fetch test "fetch auto_commits offset by default" do worker_name = :fetch_test_worker topic = "kafka_ex_consumer_group_test" consumer_group = "auto_commit_consumer_group" - KafkaEx.create_worker(worker_name, - uris: Application.get_env(:kafka_ex, :brokers), - consumer_group: consumer_group) + + KafkaEx.create_worker( + worker_name, + uris: Application.get_env(:kafka_ex, :brokers), + consumer_group: consumer_group + ) offset_before = TestHelper.latest_offset_number(topic, 0, worker_name) + Enum.each(1..10, fn _ -> - msg = %Proto.Produce.Message{value: "hey #{inspect :os.timestamp}"} - KafkaEx.produce(%Proto.Produce.Request{topic: topic, - partition: 0, - required_acks: 1, - messages: [msg]}, - worker_name: worker_name) + msg = %Proto.Produce.Message{value: "hey #{inspect(:os.timestamp())}"} + + KafkaEx.produce( + %Proto.Produce.Request{ + topic: topic, + partition: 0, + required_acks: 1, + messages: [msg] + }, + worker_name: worker_name + ) end) offset_after = TestHelper.latest_offset_number(topic, 0, worker_name) assert offset_after == offset_before + 10 - [logs] = KafkaEx.fetch(topic, - 0, - offset: offset_before, - worker_name: worker_name) + [logs] = + KafkaEx.fetch( + topic, + 0, + offset: offset_before, + worker_name: worker_name + ) + [partition] = logs.partitions message_set = partition.message_set assert 10 == length(message_set) @@ -139,9 +208,11 @@ defmodule KafkaEx.ConsumerGroup.Test do last_message = List.last(message_set) offset_of_last_message = last_message.offset - offset_request = %Proto.OffsetFetch.Request{topic: topic, - partition: 0, - consumer_group: consumer_group} + offset_request = %Proto.OffsetFetch.Request{ + topic: topic, + partition: 0, + consumer_group: consumer_group + } [offset_fetch_response] = KafkaEx.offset_fetch(worker_name, offset_request) [partition] = offset_fetch_response.partitions @@ -155,9 +226,27 @@ defmodule KafkaEx.ConsumerGroup.Test do test "fetch starts consuming from last committed offset" do random_string = generate_random_string() KafkaEx.create_worker(:fetch_test_committed_worker) - Enum.each(1..10, fn _ -> KafkaEx.produce(%Proto.Produce.Request{topic: random_string, partition: 0, required_acks: 1, messages: [%Proto.Produce.Message{value: "hey"}]}) end) - KafkaEx.offset_commit(:fetch_test_committed_worker, %Proto.OffsetCommit.Request{topic: random_string, offset: 3, partition: 0}) - logs = KafkaEx.fetch(random_string, 0, worker: :fetch_test_committed_worker) |> hd |> Map.get(:partitions) |> hd |> Map.get(:message_set) + + Enum.each(1..10, fn _ -> + KafkaEx.produce(%Proto.Produce.Request{ + topic: random_string, + partition: 0, + required_acks: 1, + messages: [%Proto.Produce.Message{value: "hey"}] + }) + end) + + KafkaEx.offset_commit( + :fetch_test_committed_worker, + %Proto.OffsetCommit.Request{topic: random_string, offset: 3, partition: 0} + ) + + logs = + KafkaEx.fetch(random_string, 0, worker: :fetch_test_committed_worker) + |> hd + |> Map.get(:partitions) + |> hd + |> Map.get(:message_set) first_message = logs |> hd @@ -169,54 +258,166 @@ defmodule KafkaEx.ConsumerGroup.Test do topic = generate_random_string() worker_name = :fetch_no_auto_commit_worker KafkaEx.create_worker(worker_name) - Enum.each(1..10, fn _ -> KafkaEx.produce(%Proto.Produce.Request{topic: topic, partition: 0, required_acks: 1, messages: [%Proto.Produce.Message{value: "hey"}]}, worker_name: worker_name) end) - offset = KafkaEx.fetch(topic, 0, offset: 0, worker: worker_name, auto_commit: false) |> hd |> Map.get(:partitions) |> hd |> Map.get(:message_set) |> Enum.reverse |> hd |> Map.get(:offset) - offset_fetch_response = KafkaEx.offset_fetch(worker_name, %Proto.OffsetFetch.Request{topic: topic, partition: 0}) |> hd - offset_fetch_response_offset = offset_fetch_response.partitions |> hd |> Map.get(:offset) + + Enum.each(1..10, fn _ -> + KafkaEx.produce( + %Proto.Produce.Request{ + topic: topic, + partition: 0, + required_acks: 1, + messages: [%Proto.Produce.Message{value: "hey"}] + }, + worker_name: worker_name + ) + end) + + offset = + KafkaEx.fetch( + topic, + 0, + offset: 0, + worker: worker_name, + auto_commit: false + ) + |> hd + |> Map.get(:partitions) + |> hd + |> Map.get(:message_set) + |> Enum.reverse() + |> hd + |> Map.get(:offset) + + offset_fetch_response = + KafkaEx.offset_fetch(worker_name, %Proto.OffsetFetch.Request{ + topic: topic, + partition: 0 + }) + |> hd + + offset_fetch_response_offset = + offset_fetch_response.partitions |> hd |> Map.get(:offset) refute offset == offset_fetch_response_offset end - #offset_fetch + # offset_fetch test "offset_fetch does not override consumer_group" do topic = generate_random_string() worker_name = :offset_fetch_consumer_group consumer_group = "bar#{topic}" - KafkaEx.create_worker(worker_name, consumer_group: consumer_group, uris: Application.get_env(:kafka_ex, :brokers)) - Enum.each(1..10, fn _ -> KafkaEx.produce(%Proto.Produce.Request{topic: topic, required_acks: 1, partition: 0, messages: [%Proto.Produce.Message{value: "hey"}]}, worker_name: worker_name) end) - KafkaEx.offset_fetch(worker_name, %KafkaEx.Protocol.OffsetFetch.Request{topic: topic, partition: 0}) + KafkaEx.create_worker( + worker_name, + consumer_group: consumer_group, + uris: Application.get_env(:kafka_ex, :brokers) + ) + + Enum.each(1..10, fn _ -> + KafkaEx.produce( + %Proto.Produce.Request{ + topic: topic, + required_acks: 1, + partition: 0, + messages: [%Proto.Produce.Message{value: "hey"}] + }, + worker_name: worker_name + ) + end) - assert :sys.get_state(:offset_fetch_consumer_group).consumer_group == consumer_group + KafkaEx.offset_fetch(worker_name, %KafkaEx.Protocol.OffsetFetch.Request{ + topic: topic, + partition: 0 + }) + + assert :sys.get_state(:offset_fetch_consumer_group).consumer_group == + consumer_group end - #offset_commit + # offset_commit test "offset_commit commits an offset and offset_fetch retrieves the committed offset" do random_string = generate_random_string() - Enum.each(1..10, fn _ -> KafkaEx.produce(%Proto.Produce.Request{topic: random_string, partition: 0, required_acks: 1, messages: [%Proto.Produce.Message{value: "hey"}]}) end) - assert KafkaEx.offset_commit(Config.default_worker, %Proto.OffsetCommit.Request{topic: random_string, offset: 9, partition: 0}) == - [%Proto.OffsetCommit.Response{partitions: [0], topic: random_string}] - assert KafkaEx.offset_fetch(Config.default_worker, %Proto.OffsetFetch.Request{topic: random_string, partition: 0}) == - [%Proto.OffsetFetch.Response{partitions: [%{metadata: "", error_code: :no_error, offset: 9, partition: 0}], topic: random_string}] + + Enum.each(1..10, fn _ -> + KafkaEx.produce(%Proto.Produce.Request{ + topic: random_string, + partition: 0, + required_acks: 1, + messages: [%Proto.Produce.Message{value: "hey"}] + }) + end) + + assert KafkaEx.offset_commit( + Config.default_worker(), + %Proto.OffsetCommit.Request{ + topic: random_string, + offset: 9, + partition: 0 + } + ) == + [ + %Proto.OffsetCommit.Response{ + partitions: [0], + topic: random_string + } + ] + + assert KafkaEx.offset_fetch( + Config.default_worker(), + %Proto.OffsetFetch.Request{topic: random_string, partition: 0} + ) == + [ + %Proto.OffsetFetch.Response{ + partitions: [ + %{ + metadata: "", + error_code: :no_error, + offset: 9, + partition: 0 + } + ], + topic: random_string + } + ] end - #stream + # stream test "stream auto_commits offset by default" do random_string = generate_random_string() - KafkaEx.create_worker(:stream_auto_commit, uris: uris(), consumer_group: "kafka_ex") - KafkaEx.produce(%Proto.Produce.Request{topic: random_string, partition: 0, required_acks: 1, messages: [ + + KafkaEx.create_worker( + :stream_auto_commit, + uris: uris(), + consumer_group: "kafka_ex" + ) + + KafkaEx.produce(%Proto.Produce.Request{ + topic: random_string, + partition: 0, + required_acks: 1, + messages: [ %Proto.Produce.Message{value: "hey"}, - %Proto.Produce.Message{value: "hi"}, + %Proto.Produce.Message{value: "hi"} ] }) - stream = KafkaEx.stream(random_string, 0, worker_name: :stream_auto_commit, offset: 0) - log = TestHelper.wait_for_any( - fn() -> Enum.take(stream, 2) end - ) + + stream = + KafkaEx.stream( + random_string, + 0, + worker_name: :stream_auto_commit, + offset: 0 + ) + + log = TestHelper.wait_for_any(fn -> Enum.take(stream, 2) end) refute Enum.empty?(log) - [offset_fetch_response | _] = KafkaEx.offset_fetch(:stream_auto_commit, %Proto.OffsetFetch.Request{topic: random_string, partition: 0}) + [offset_fetch_response | _] = + KafkaEx.offset_fetch(:stream_auto_commit, %Proto.OffsetFetch.Request{ + topic: random_string, + partition: 0 + }) + error_code = offset_fetch_response.partitions |> hd |> Map.get(:error_code) offset = offset_fetch_response.partitions |> hd |> Map.get(:offset) @@ -228,20 +429,46 @@ defmodule KafkaEx.ConsumerGroup.Test do random_string = generate_random_string() consumer_group = "kafka_ex" worker_name = :stream_last_committed_offset - KafkaEx.create_worker(worker_name, uris: uris(), consumer_group: consumer_group) - Enum.each(1..10, fn _ -> KafkaEx.produce(%Proto.Produce.Request{topic: random_string, partition: 0, required_acks: 1, messages: [%Proto.Produce.Message{value: "hey"}]}, worker_name: worker_name) end) - KafkaEx.offset_commit(worker_name, %Proto.OffsetCommit.Request{topic: random_string, partition: 0, offset: 3}) + + KafkaEx.create_worker( + worker_name, + uris: uris(), + consumer_group: consumer_group + ) + + Enum.each(1..10, fn _ -> + KafkaEx.produce( + %Proto.Produce.Request{ + topic: random_string, + partition: 0, + required_acks: 1, + messages: [%Proto.Produce.Message{value: "hey"}] + }, + worker_name: worker_name + ) + end) + + KafkaEx.offset_commit(worker_name, %Proto.OffsetCommit.Request{ + topic: random_string, + partition: 0, + offset: 3 + }) # make sure the offset commit is actually committed before we # start streaming again - :ok = TestHelper.wait_for(fn() -> - 3 == TestHelper.latest_consumer_offset_number(random_string, 0, consumer_group, worker_name) - end) + :ok = + TestHelper.wait_for(fn -> + 3 == + TestHelper.latest_consumer_offset_number( + random_string, + 0, + consumer_group, + worker_name + ) + end) stream = KafkaEx.stream(random_string, 0, worker_name: worker_name) - log = TestHelper.wait_for_any( - fn() -> Enum.take(stream, 2) end - ) + log = TestHelper.wait_for_any(fn -> Enum.take(stream, 2) end) refute Enum.empty?(log) first_message = log |> hd @@ -253,34 +480,44 @@ defmodule KafkaEx.ConsumerGroup.Test do consumer_group = "stream_test" KafkaEx.create_worker(:stream, uris: uris()) - KafkaEx.produce(%Proto.Produce.Request{ - topic: topic_name, partition: 0, required_acks: 1, messages: [ - %Proto.Produce.Message{value: "message 2"}, - %Proto.Produce.Message{value: "message 3"}, - %Proto.Produce.Message{value: "message 4"}, - %Proto.Produce.Message{value: "message 5"}, - ] - }, worker_name: :stream) - - stream = KafkaEx.stream( - topic_name, - 0, - worker_name: :stream, - offset: 0, - auto_commit: true, - consumer_group: consumer_group + + KafkaEx.produce( + %Proto.Produce.Request{ + topic: topic_name, + partition: 0, + required_acks: 1, + messages: [ + %Proto.Produce.Message{value: "message 2"}, + %Proto.Produce.Message{value: "message 3"}, + %Proto.Produce.Message{value: "message 4"}, + %Proto.Produce.Message{value: "message 5"} + ] + }, + worker_name: :stream ) + stream = + KafkaEx.stream( + topic_name, + 0, + worker_name: :stream, + offset: 0, + auto_commit: true, + consumer_group: consumer_group + ) + [m1, m2] = Enum.take(stream, 2) assert "message 2" == m1.value assert "message 3" == m2.value - offset = TestHelper.latest_consumer_offset_number( - topic_name, - 0, - consumer_group, - :stream - ) + offset = + TestHelper.latest_consumer_offset_number( + topic_name, + 0, + consumer_group, + :stream + ) + assert offset == m2.offset end @@ -289,59 +526,73 @@ defmodule KafkaEx.ConsumerGroup.Test do consumer_group = "stream_test" KafkaEx.create_worker(:stream, uris: uris()) - KafkaEx.produce(%Proto.Produce.Request{ - topic: topic_name, partition: 0, required_acks: 1, messages: [ - %Proto.Produce.Message{value: "message 2"}, - %Proto.Produce.Message{value: "message 3"}, - %Proto.Produce.Message{value: "message 4"}, - %Proto.Produce.Message{value: "message 5"}, - ] - }, worker_name: :stream) - - stream = KafkaEx.stream( - topic_name, - 0, - worker_name: :stream, - offset: 0, - no_wait_at_logend: true, - auto_commit: true, - consumer_group: consumer_group + + KafkaEx.produce( + %Proto.Produce.Request{ + topic: topic_name, + partition: 0, + required_acks: 1, + messages: [ + %Proto.Produce.Message{value: "message 2"}, + %Proto.Produce.Message{value: "message 3"}, + %Proto.Produce.Message{value: "message 4"}, + %Proto.Produce.Message{value: "message 5"} + ] + }, + worker_name: :stream ) + stream = + KafkaEx.stream( + topic_name, + 0, + worker_name: :stream, + offset: 0, + no_wait_at_logend: true, + auto_commit: true, + consumer_group: consumer_group + ) + [m1, m2, m3, m4] = Enum.take(stream, 10) assert "message 2" == m1.value assert "message 3" == m2.value assert "message 4" == m3.value assert "message 5" == m4.value - offset = TestHelper.latest_consumer_offset_number( - topic_name, - 0, - consumer_group, - :stream - ) + offset = + TestHelper.latest_consumer_offset_number( + topic_name, + 0, + consumer_group, + :stream + ) + assert offset == m4.offset other_consumer_group = "another_consumer_group" - map_stream = KafkaEx.stream( - topic_name, - 0, - worker_name: :stream, - no_wait_at_logend: true, - auto_commit: true, - offset: 0, - consumer_group: other_consumer_group - ) + + map_stream = + KafkaEx.stream( + topic_name, + 0, + worker_name: :stream, + no_wait_at_logend: true, + auto_commit: true, + offset: 0, + consumer_group: other_consumer_group + ) assert ["message 2", "message 3", "message 4", "message 5"] = - Enum.map(map_stream, fn(m) -> m.value end) + Enum.map(map_stream, fn m -> m.value end) + + offset = + TestHelper.latest_consumer_offset_number( + topic_name, + 0, + other_consumer_group, + :stream + ) - offset = TestHelper.latest_consumer_offset_number( - topic_name, - 0, - other_consumer_group, - :stream - ) # should have the same offset as the first stream assert offset == m4.offset end @@ -351,10 +602,13 @@ defmodule KafkaEx.ConsumerGroup.Test do consumer_group = "stream_test" KafkaEx.create_worker(:stream, uris: uris()) - messages_in = Enum.map( - 1 .. 10, - fn(ix) -> %Proto.Produce.Message{value: "Msg #{ix}"} end - ) + + messages_in = + Enum.map( + 1..10, + fn ix -> %Proto.Produce.Message{value: "Msg #{ix}"} end + ) + KafkaEx.produce( %Proto.Produce.Request{ topic: topic_name, @@ -365,32 +619,36 @@ defmodule KafkaEx.ConsumerGroup.Test do worker_name: :stream ) - stream1 = KafkaEx.stream( - topic_name, - 0, - worker_name: :stream, - no_wait_at_logend: true, - auto_commit: true, - offset: 0, - consumer_group: consumer_group - ) - - assert ["Msg 1", "Msg 2"] == stream1 - |> Enum.take(2) - |> Enum.map(fn(msg) -> msg.value end) - - stream2 = KafkaEx.stream( - topic_name, - 0, - worker_name: :stream, - no_wait_at_logend: true, - auto_commit: true, - consumer_group: consumer_group - ) - - message_values = stream2 - |> Enum.take(2) - |> Enum.map(fn(msg) -> msg.value end) + stream1 = + KafkaEx.stream( + topic_name, + 0, + worker_name: :stream, + no_wait_at_logend: true, + auto_commit: true, + offset: 0, + consumer_group: consumer_group + ) + + assert ["Msg 1", "Msg 2"] == + stream1 + |> Enum.take(2) + |> Enum.map(fn msg -> msg.value end) + + stream2 = + KafkaEx.stream( + topic_name, + 0, + worker_name: :stream, + no_wait_at_logend: true, + auto_commit: true, + consumer_group: consumer_group + ) + + message_values = + stream2 + |> Enum.take(2) + |> Enum.map(fn msg -> msg.value end) assert ["Msg 3", "Msg 4"] == message_values end diff --git a/test/integration/integration_test.exs b/test/integration/integration_test.exs index ee1ecfa0..2e075fa2 100644 --- a/test/integration/integration_test.exs +++ b/test/integration/integration_test.exs @@ -7,23 +7,34 @@ defmodule KafkaEx.Integration.Test do @moduletag :integration test "default worker starts on Application start up" do - pid = Process.whereis(Config.default_worker) + pid = Process.whereis(Config.default_worker()) assert is_pid(pid) end - #create_worker + # create_worker test "KafkaEx.Supervisor dynamically creates workers" do {:ok, pid} = KafkaEx.create_worker(:bar, uris: uris()) assert Process.whereis(:bar) == pid end test "Creates a worker even when the one of the provided brokers is not available" do - {:ok, pid} = KafkaEx.create_worker(:no_broker_worker, uris: uris() ++ [{"bad_host", 9000}]) + {:ok, pid} = + KafkaEx.create_worker( + :no_broker_worker, + uris: uris() ++ [{"bad_host", 9000}] + ) + assert Process.whereis(:no_broker_worker) == pid end test "create_worker allows custom metadata_update_interval" do - {:ok, pid} = KafkaEx.create_worker(:metadata_update_interval_custom, uris: uris(), metadata_update_interval: 10) + {:ok, pid} = + KafkaEx.create_worker( + :metadata_update_interval_custom, + uris: uris(), + metadata_update_interval: 10 + ) + metadata_update_interval = :sys.get_state(pid).metadata_update_interval assert metadata_update_interval == 10 @@ -36,22 +47,43 @@ defmodule KafkaEx.Integration.Test do assert metadata_update_interval == 30000 end - #update_metadata + # update_metadata test "worker updates metadata after specified interval" do random_string = generate_random_string() - KafkaEx.create_worker(:update_metadata, [uris: uris(), consumer_group: "foo", metadata_update_interval: 100]) + + KafkaEx.create_worker( + :update_metadata, + uris: uris(), + consumer_group: "foo", + metadata_update_interval: 100 + ) + previous_metadata = KafkaEx.metadata(worker_name: :update_metadata) - KafkaEx.produce(%Proto.Produce.Request{topic: random_string, partition: 0, required_acks: 0, messages: [%Proto.Produce.Message{value: "hey"}]}) + + KafkaEx.produce(%Proto.Produce.Request{ + topic: random_string, + partition: 0, + required_acks: 0, + messages: [%Proto.Produce.Message{value: "hey"}] + }) + :timer.sleep(105) new_metadata = KafkaEx.metadata(worker_name: :update_metadata) refute previous_metadata == new_metadata end - test "default worker generates metadata on start up" do - pid = Process.whereis(Config.default_worker) - KafkaEx.produce("food", 0, "hey", worker_name: Config.default_worker, required_acks: 1) + pid = Process.whereis(Config.default_worker()) + + KafkaEx.produce( + "food", + 0, + "hey", + worker_name: Config.default_worker(), + required_acks: 1 + ) + metadata = :sys.get_state(pid).metadata refute metadata == %Proto.Metadata.Response{} @@ -63,23 +95,41 @@ defmodule KafkaEx.Integration.Test do assert pid == Process.whereis(:test_server) end - #produce + # produce test "produce/4 without an acq required returns :ok" do assert KafkaEx.produce("food", 0, "hey") == :ok end test "produce/4 with ack required returns an ack" do - {:ok, offset} = KafkaEx.produce("food", 0, "hey", worker_name: Config.default_worker, required_acks: 1) + {:ok, offset} = + KafkaEx.produce( + "food", + 0, + "hey", + worker_name: Config.default_worker(), + required_acks: 1 + ) refute offset == nil end test "produce without an acq required returns :ok" do - assert KafkaEx.produce(%Proto.Produce.Request{topic: "food", partition: 0, required_acks: 0, messages: [%Proto.Produce.Message{value: "hey"}]}) == :ok + assert KafkaEx.produce(%Proto.Produce.Request{ + topic: "food", + partition: 0, + required_acks: 0, + messages: [%Proto.Produce.Message{value: "hey"}] + }) == :ok end test "produce with ack required returns an ack" do - {:ok, offset} = KafkaEx.produce(%Proto.Produce.Request{topic: "food", partition: 0, required_acks: 1, messages: [%Proto.Produce.Message{value: "hey"}]}) + {:ok, offset} = + KafkaEx.produce(%Proto.Produce.Request{ + topic: "food", + partition: 0, + required_acks: 1, + messages: [%Proto.Produce.Message{value: "hey"}] + }) refute offset == nil end @@ -87,13 +137,23 @@ defmodule KafkaEx.Integration.Test do test "produce updates metadata" do {:ok, pid} = KafkaEx.create_worker(:update_metadata_test) empty_metadata = %Proto.Metadata.Response{} - :sys.replace_state(pid, fn(state) -> + + :sys.replace_state(pid, fn state -> %{state | :metadata => empty_metadata} end) assert empty_metadata.brokers == [] - KafkaEx.produce(%Proto.Produce.Request{topic: "food", partition: 0, required_acks: 1, messages: [%Proto.Produce.Message{value: "hey"}]}, worker_name: :update_metadata_test) + KafkaEx.produce( + %Proto.Produce.Request{ + topic: "food", + partition: 0, + required_acks: 1, + messages: [%Proto.Produce.Message{value: "hey"}] + }, + worker_name: :update_metadata_test + ) + metadata = :sys.get_state(pid).metadata refute metadata == empty_metadata @@ -102,48 +162,95 @@ defmodule KafkaEx.Integration.Test do test "produce creates log for a non-existing topic" do random_string = generate_random_string() - KafkaEx.produce(%Proto.Produce.Request{topic: random_string, partition: 0, required_acks: 1, messages: [%Proto.Produce.Message{value: "hey"}]}) - pid = Process.whereis(Config.default_worker) + + KafkaEx.produce(%Proto.Produce.Request{ + topic: random_string, + partition: 0, + required_acks: 1, + messages: [%Proto.Produce.Message{value: "hey"}] + }) + + pid = Process.whereis(Config.default_worker()) metadata = :sys.get_state(pid).metadata - assert Enum.find_value(metadata.topic_metadatas, &(&1.topic == random_string)) + assert Enum.find_value( + metadata.topic_metadatas, + &(&1.topic == random_string) + ) end - #metadata + # metadata test "metadata works" do random_string = generate_random_string() - Enum.each(1..10, fn _ -> KafkaEx.produce(%Proto.Produce.Request{topic: random_string, partition: 0, required_acks: 1, messages: [%Proto.Produce.Message{value: "hey"}]}) end) - refute Enum.empty?(Enum.flat_map(KafkaEx.metadata.topic_metadatas, fn(metadata) -> metadata.partition_metadatas end)) + Enum.each(1..10, fn _ -> + KafkaEx.produce(%Proto.Produce.Request{ + topic: random_string, + partition: 0, + required_acks: 1, + messages: [%Proto.Produce.Message{value: "hey"}] + }) + end) + + refute Enum.empty?( + Enum.flat_map(KafkaEx.metadata().topic_metadatas, fn metadata -> + metadata.partition_metadatas + end) + ) end test "metadata for a non-existing topic creates a new topic" do random_string = generate_random_string() - metadata = TestHelper.wait_for_value( - fn() -> KafkaEx.metadata(topic: random_string) end, - fn(metadata) -> metadata != nil && length(metadata.topic_metadatas) > 0 end - ) - random_topic_metadata = Enum.find(metadata.topic_metadatas, &(&1.topic == random_string)) + + metadata = + TestHelper.wait_for_value( + fn -> KafkaEx.metadata(topic: random_string) end, + fn metadata -> + metadata != nil && length(metadata.topic_metadatas) > 0 + end + ) + + random_topic_metadata = + Enum.find(metadata.topic_metadatas, &(&1.topic == random_string)) refute random_topic_metadata.partition_metadatas == [] - assert Enum.all?(random_topic_metadata.partition_metadatas, &(&1.error_code == :no_error)) - pid = Process.whereis(Config.default_worker) + assert Enum.all?( + random_topic_metadata.partition_metadatas, + &(&1.error_code == :no_error) + ) + + pid = Process.whereis(Config.default_worker()) metadata = :sys.get_state(pid).metadata - random_topic_metadata = Enum.find(metadata.topic_metadatas, &(&1.topic == random_string)) + + random_topic_metadata = + Enum.find(metadata.topic_metadatas, &(&1.topic == random_string)) refute random_topic_metadata.partition_metadatas == [] - assert Enum.all?(random_topic_metadata.partition_metadatas, &(&1.error_code == :no_error)) + + assert Enum.all?( + random_topic_metadata.partition_metadatas, + &(&1.error_code == :no_error) + ) end - #fetch + # fetch test "fetch updates metadata" do {:ok, pid} = KafkaEx.create_worker(:fetch_updates_metadata) empty_metadata = %Proto.Metadata.Response{} - :sys.replace_state(pid, fn(state) -> + + :sys.replace_state(pid, fn state -> %{state | :metadata => empty_metadata} end) - KafkaEx.fetch("food", 0, offset: 0, auto_commit: false, worker_name: :fetch_updates_metadata) + + KafkaEx.fetch( + "food", + 0, + offset: 0, + auto_commit: false, + worker_name: :fetch_updates_metadata + ) + :timer.sleep(200) metadata = :sys.get_state(pid).metadata @@ -162,21 +269,33 @@ defmodule KafkaEx.Integration.Test do test "fetch works" do random_string = generate_random_string() - {:ok, offset} = KafkaEx.produce(%Proto.Produce.Request{topic: random_string, partition: 0, required_acks: 1, messages: [%Proto.Produce.Message{value: "hey foo"}]}) - fetch_response = KafkaEx.fetch(random_string, 0, offset: 0, auto_commit: false) |> hd + + {:ok, offset} = + KafkaEx.produce(%Proto.Produce.Request{ + topic: random_string, + partition: 0, + required_acks: 1, + messages: [%Proto.Produce.Message{value: "hey foo"}] + }) + + fetch_response = + KafkaEx.fetch(random_string, 0, offset: 0, auto_commit: false) |> hd + message = fetch_response.partitions |> hd |> Map.get(:message_set) |> hd assert message.value == "hey foo" assert message.offset == offset end - #offset + # offset test "offset updates metadata" do {:ok, pid} = KafkaEx.create_worker(:offset_updates_metadata) empty_metadata = %Proto.Metadata.Response{} - :sys.replace_state(pid, fn(state) -> + + :sys.replace_state(pid, fn state -> %{state | :metadata => empty_metadata} end) + KafkaEx.offset("food", 0, utc_time(), :offset_updates_metadata) metadata = :sys.get_state(pid).metadata @@ -185,7 +304,14 @@ defmodule KafkaEx.Integration.Test do test "offset retrieves most recent offset by time specification" do random_string = generate_random_string() - KafkaEx.produce(%Proto.Produce.Request{topic: random_string, partition: 0, required_acks: 1, messages: [%Proto.Produce.Message{value: "hey"}]}) + + KafkaEx.produce(%Proto.Produce.Request{ + topic: random_string, + partition: 0, + required_acks: 1, + messages: [%Proto.Produce.Message{value: "hey"}] + }) + offset_response = KafkaEx.offset(random_string, 0, utc_time()) |> hd offset = offset_response.partition_offsets |> hd |> Map.get(:offset) |> hd @@ -194,7 +320,14 @@ defmodule KafkaEx.Integration.Test do test "earliest_offset retrieves offset of 0" do random_string = generate_random_string() - KafkaEx.produce(%Proto.Produce.Request{topic: random_string, partition: 0, required_acks: 1, messages: [%Proto.Produce.Message{value: "hey"}]}) + + KafkaEx.produce(%Proto.Produce.Request{ + topic: random_string, + partition: 0, + required_acks: 1, + messages: [%Proto.Produce.Message{value: "hey"}] + }) + offset_response = KafkaEx.earliest_offset(random_string, 0) |> hd offset = offset_response.partition_offsets |> hd |> Map.get(:offset) |> hd @@ -203,7 +336,15 @@ defmodule KafkaEx.Integration.Test do test "latest_offset retrieves offset of 0 for non-existing topic" do random_string = generate_random_string() - {:ok, produce_offset} = KafkaEx.produce(%Proto.Produce.Request{topic: random_string, partition: 0, required_acks: 1, messages: [%Proto.Produce.Message{value: "hey"}]}) + + {:ok, produce_offset} = + KafkaEx.produce(%Proto.Produce.Request{ + topic: random_string, + partition: 0, + required_acks: 1, + messages: [%Proto.Produce.Message{value: "hey"}] + }) + :timer.sleep(300) offset_response = KafkaEx.latest_offset(random_string, 0) |> hd offset = offset_response.partition_offsets |> hd |> Map.get(:offset) |> hd @@ -213,10 +354,17 @@ defmodule KafkaEx.Integration.Test do test "latest_offset retrieves a non-zero offset for a topic published to" do random_string = generate_random_string() - KafkaEx.produce(%Proto.Produce.Request{topic: random_string, partition: 0, required_acks: 1, messages: [%Proto.Produce.Message{value: "foo"}]}) - [offset_response] = TestHelper.wait_for_any( - fn() -> KafkaEx.latest_offset(random_string, 0) end - ) + + KafkaEx.produce(%Proto.Produce.Request{ + topic: random_string, + partition: 0, + required_acks: 1, + messages: [%Proto.Produce.Message{value: "foo"}] + }) + + [offset_response] = + TestHelper.wait_for_any(fn -> KafkaEx.latest_offset(random_string, 0) end) + offset = offset_response.partition_offsets |> hd |> Map.get(:offset) |> hd assert offset != 0 @@ -235,11 +383,16 @@ defmodule KafkaEx.Integration.Test do partition: 0, required_acks: 1, compression: :gzip, - messages: messages} - {:ok, offset} = KafkaEx.produce(produce_request) + messages: messages + } + + {:ok, offset} = KafkaEx.produce(produce_request) + + fetch_response = + KafkaEx.fetch(random_string, 0, offset: 0, auto_commit: false) |> hd - fetch_response = KafkaEx.fetch(random_string, 0, offset: 0, auto_commit: false) |> hd - [got_message1, got_message2] = fetch_response.partitions |> hd |> Map.get(:message_set) + [got_message1, got_message2] = + fetch_response.partitions |> hd |> Map.get(:message_set) assert got_message1.key == message1.key assert got_message1.value == message1.value @@ -261,11 +414,16 @@ defmodule KafkaEx.Integration.Test do partition: 0, required_acks: 1, compression: :snappy, - messages: messages} - {:ok, offset} = KafkaEx.produce(produce_request) + messages: messages + } - fetch_response = KafkaEx.fetch(random_string, 0, offset: 0, auto_commit: false) |> hd - [got_message1, got_message2] = fetch_response.partitions |> hd |> Map.get(:message_set) + {:ok, offset} = KafkaEx.produce(produce_request) + + fetch_response = + KafkaEx.fetch(random_string, 0, offset: 0, auto_commit: false) |> hd + + [got_message1, got_message2] = + fetch_response.partitions |> hd |> Map.get(:message_set) assert got_message1.key == message1.key assert got_message1.value == message1.value @@ -282,7 +440,13 @@ defmodule KafkaEx.Integration.Test do # 10 chars * 1024 repeats ~= 10kb message_value = String.duplicate("ABCDEFGHIJ", 1024) messages = [%Proto.Produce.Message{key: nil, value: message_value}] - produce_request = %Proto.Produce.Request{topic: topic, partition: 0, required_acks: 1, messages: messages} + + produce_request = %Proto.Produce.Request{ + topic: topic, + partition: 0, + required_acks: 1, + messages: messages + } {:ok, offset} = KafkaEx.produce(produce_request) @@ -299,7 +463,14 @@ defmodule KafkaEx.Integration.Test do # 10 chars * 1024 repeats ~= 10kb message_value = String.duplicate("ABCDEFGHIJ", 100) messages = [%Proto.Produce.Message{key: nil, value: message_value}] - produce_request = %Proto.Produce.Request{topic: topic, partition: 0, compression: :gzip, required_acks: 1, messages: messages} + + produce_request = %Proto.Produce.Request{ + topic: topic, + partition: 0, + compression: :gzip, + required_acks: 1, + messages: messages + } {:ok, offset} = KafkaEx.produce(produce_request) @@ -316,7 +487,14 @@ defmodule KafkaEx.Integration.Test do # 10 chars * 1024 repeats ~= 10kb message_value = String.duplicate("ABCDEFGHIJ", 100) messages = [%Proto.Produce.Message{key: nil, value: message_value}] - produce_request = %Proto.Produce.Request{topic: topic, partition: 0, compression: :snappy, required_acks: 1, messages: messages} + + produce_request = %Proto.Produce.Request{ + topic: topic, + partition: 0, + compression: :snappy, + required_acks: 1, + messages: messages + } {:ok, offset} = KafkaEx.produce(produce_request) @@ -331,23 +509,31 @@ defmodule KafkaEx.Integration.Test do test "streams kafka logs" do topic_name = generate_random_string() KafkaEx.create_worker(:stream, uris: uris()) - KafkaEx.produce(%Proto.Produce.Request{ - topic: topic_name, partition: 0, required_acks: 1, messages: [ - %Proto.Produce.Message{value: "message 1"}, - %Proto.Produce.Message{value: "message 2"}, - %Proto.Produce.Message{value: "message 3"}, - %Proto.Produce.Message{value: "message 4"}, - ] - }, worker_name: :stream) - - stream = KafkaEx.stream( - topic_name, - 0, - worker_name: :stream, - offset: 0, - auto_commit: false + + KafkaEx.produce( + %Proto.Produce.Request{ + topic: topic_name, + partition: 0, + required_acks: 1, + messages: [ + %Proto.Produce.Message{value: "message 1"}, + %Proto.Produce.Message{value: "message 2"}, + %Proto.Produce.Message{value: "message 3"}, + %Proto.Produce.Message{value: "message 4"} + ] + }, + worker_name: :stream ) + stream = + KafkaEx.stream( + topic_name, + 0, + worker_name: :stream, + offset: 0, + auto_commit: false + ) + logs = Enum.take(stream, 2) assert 2 == length(logs) [m1, m2] = logs @@ -361,25 +547,33 @@ defmodule KafkaEx.Integration.Test do test "stream with small max_bytes makes multiple requests if necessary" do topic_name = generate_random_string() KafkaEx.create_worker(:stream, uris: uris()) - KafkaEx.produce(%Proto.Produce.Request{ - topic: topic_name, partition: 0, required_acks: 1, messages: [ - %Proto.Produce.Message{value: "message 1"}, - %Proto.Produce.Message{value: "message 2"}, - %Proto.Produce.Message{value: "message 3"}, - %Proto.Produce.Message{value: "message 4"}, - %Proto.Produce.Message{value: "message 5"}, - ] - }, worker_name: :stream) - - stream = KafkaEx.stream( - topic_name, - 0, - worker_name: :stream, - max_bytes: 50, - offset: 0, - auto_commit: false + + KafkaEx.produce( + %Proto.Produce.Request{ + topic: topic_name, + partition: 0, + required_acks: 1, + messages: [ + %Proto.Produce.Message{value: "message 1"}, + %Proto.Produce.Message{value: "message 2"}, + %Proto.Produce.Message{value: "message 3"}, + %Proto.Produce.Message{value: "message 4"}, + %Proto.Produce.Message{value: "message 5"} + ] + }, + worker_name: :stream ) + stream = + KafkaEx.stream( + topic_name, + 0, + worker_name: :stream, + max_bytes: 50, + offset: 0, + auto_commit: false + ) + logs = Enum.take(stream, 4) assert 4 == length(logs) [m1, m2, m3, m4] = logs @@ -395,30 +589,45 @@ defmodule KafkaEx.Integration.Test do test "stream blocks until new messages are available" do topic_name = generate_random_string() KafkaEx.create_worker(:stream, uris: uris()) - KafkaEx.produce(%Proto.Produce.Request{ - topic: topic_name, partition: 0, required_acks: 1, messages: [ - %Proto.Produce.Message{value: "message 1"} - ] - }, worker_name: :stream) - stream = KafkaEx.stream( - topic_name, - 0, - worker_name: :stream, - max_bytes: 50, - offset: 0, - auto_commit: false + KafkaEx.produce( + %Proto.Produce.Request{ + topic: topic_name, + partition: 0, + required_acks: 1, + messages: [ + %Proto.Produce.Message{value: "message 1"} + ] + }, + worker_name: :stream ) + + stream = + KafkaEx.stream( + topic_name, + 0, + worker_name: :stream, + max_bytes: 50, + offset: 0, + auto_commit: false + ) + task = Task.async(fn -> Enum.take(stream, 4) end) - KafkaEx.produce(%Proto.Produce.Request{ - topic: topic_name, partition: 0, required_acks: 1, messages: [ - %Proto.Produce.Message{value: "message 2"}, - %Proto.Produce.Message{value: "message 3"}, - %Proto.Produce.Message{value: "message 4"}, - %Proto.Produce.Message{value: "message 5"}, - ] - }, worker_name: :stream) + KafkaEx.produce( + %Proto.Produce.Request{ + topic: topic_name, + partition: 0, + required_acks: 1, + messages: [ + %Proto.Produce.Message{value: "message 2"}, + %Proto.Produce.Message{value: "message 3"}, + %Proto.Produce.Message{value: "message 4"}, + %Proto.Produce.Message{value: "message 5"} + ] + }, + worker_name: :stream + ) logs = Task.await(task) @@ -433,23 +642,31 @@ defmodule KafkaEx.Integration.Test do test "stream is non-blocking with no_wait_at_logend" do topic_name = generate_random_string() KafkaEx.create_worker(:stream, uris: uris()) - KafkaEx.produce(%Proto.Produce.Request{ - topic: topic_name, partition: 0, required_acks: 1, messages: [ - %Proto.Produce.Message{value: "message 1"} - ] - }, worker_name: :stream) - stream = KafkaEx.stream( - topic_name, - 0, - worker_name: :stream, - max_bytes: 50, - offset: 0, - auto_commit: false, - no_wait_at_logend: true + KafkaEx.produce( + %Proto.Produce.Request{ + topic: topic_name, + partition: 0, + required_acks: 1, + messages: [ + %Proto.Produce.Message{value: "message 1"} + ] + }, + worker_name: :stream ) - logs = Enum.take(stream, 4) + stream = + KafkaEx.stream( + topic_name, + 0, + worker_name: :stream, + max_bytes: 50, + offset: 0, + auto_commit: false, + no_wait_at_logend: true + ) + + logs = Enum.take(stream, 4) assert 1 == length(logs) [m1] = logs @@ -457,7 +674,7 @@ defmodule KafkaEx.Integration.Test do # we can also execute something like 'map' which would otherwise be # open-ended - assert ["message 1"] == Enum.map(stream, fn(m) -> m.value end) + assert ["message 1"] == Enum.map(stream, fn m -> m.value end) end test "doesn't error when re-creating an existing stream" do diff --git a/test/integration/server0_p_10_and_later_test.exs b/test/integration/server0_p_10_and_later_test.exs index 1eb54c19..a7c48509 100644 --- a/test/integration/server0_p_10_and_later_test.exs +++ b/test/integration/server0_p_10_and_later_test.exs @@ -6,7 +6,7 @@ defmodule KafkaEx.Server0P10P1AndLater.Test do @tag :create_topic test "can create a topic" do - name = "create_topic_#{:rand.uniform(2000000)}" + name = "create_topic_#{:rand.uniform(2_000_000)}" request = %{ topic: name, @@ -16,7 +16,8 @@ defmodule KafkaEx.Server0P10P1AndLater.Test do config_entries: [ %{config_name: "cleanup.policy", config_value: "compact"}, %{config_name: "min.compaction.lag.ms", config_value: "0"} - ]} + ] + } resp = KafkaEx.create_topics([request], timeout: 2000) assert {:no_error, name} == parse_create_topic_resp(resp) @@ -25,7 +26,7 @@ defmodule KafkaEx.Server0P10P1AndLater.Test do assert {:topic_already_exists, name} == parse_create_topic_resp(resp) wait_for(fn -> - topics = KafkaEx.metadata.topic_metadatas |> Enum.map(&(&1.topic)) + topics = KafkaEx.metadata().topic_metadatas |> Enum.map(& &1.topic) assert Enum.member?(topics, name) end) end @@ -37,7 +38,9 @@ defmodule KafkaEx.Server0P10P1AndLater.Test do error_code: error_code, topic_name: topic_name } - ]} = response + ] + } = response + {error_code, topic_name} end end diff --git a/test/integration/server0_p_10_p_1_test.exs b/test/integration/server0_p_10_p_1_test.exs index 3211ecb1..8f59f41a 100644 --- a/test/integration/server0_p_10_p_1_test.exs +++ b/test/integration/server0_p_10_p_1_test.exs @@ -7,7 +7,6 @@ defmodule KafkaEx.Server0P10P1.Test do # specific to this server version because we want to test that the api_versions list is exact @tag :api_version test "can retrieve api versions" do - # api_key, max_version, min_version api_versions_kafka_0_10_1_0 = [ [0, 2, 0], @@ -34,12 +33,15 @@ defmodule KafkaEx.Server0P10P1.Test do ] response = KafkaEx.api_versions() + %KafkaEx.Protocol.ApiVersions.Response{ api_versions: api_versions, error_code: :no_error, throttle_time_ms: _ } = response - assert api_versions_kafka_0_10_1_0 == api_versions |> Enum.map(&([&1.api_key, &1.max_version, &1.min_version])) + assert api_versions_kafka_0_10_1_0 == + api_versions + |> Enum.map(&[&1.api_key, &1.max_version, &1.min_version]) end end diff --git a/test/integration/server0_p_8_p_0_test.exs b/test/integration/server0_p_8_p_0_test.exs index 68252cb0..3019a0cf 100644 --- a/test/integration/server0_p_8_p_0_test.exs +++ b/test/integration/server0_p_8_p_0_test.exs @@ -15,54 +15,59 @@ defmodule KafkaEx.Server0P8P0.Test do # we don't want to crash if the worker crashes Process.unlink(worker) - on_exit fn -> + on_exit(fn -> if Process.alive?(worker) do Process.exit(worker, :normal) end - end + end) {:ok, [worker: worker]} end - test "can produce and fetch a message", %{worker: worker}do - now = :erlang.monotonic_time + test "can produce and fetch a message", %{worker: worker} do + now = :erlang.monotonic_time() msg = "test message #{now}" partition = 0 :ok = KafkaEx.produce(@topic, partition, msg, worker_name: worker) wait_for(fn -> - [got] = KafkaEx.fetch( - @topic, - partition, - worker_name: worker, - offset: 0, - auto_commit: false - ) + [got] = + KafkaEx.fetch( + @topic, + partition, + worker_name: worker, + offset: 0, + auto_commit: false + ) + [got_partition] = got.partitions - Enum.any?(got_partition.message_set, fn(m) -> m.value == msg end) + Enum.any?(got_partition.message_set, fn m -> m.value == msg end) end) end test "does not mem leak when message is not a Produce.Message" do - name = "create_topic_#{:rand.uniform(2000000)}" + name = "create_topic_#{:rand.uniform(2_000_000)}" - assert {:error, "Invalid produce request"} == KafkaEx.produce(%KafkaEx.Protocol.Produce.Request{ - messages: [%{key: "key1", value: "value1"}], - partition: 0, - required_acks: 0, - timeout: 500, - topic: name - }) + assert {:error, "Invalid produce request"} == + KafkaEx.produce(%KafkaEx.Protocol.Produce.Request{ + messages: [%{key: "key1", value: "value1"}], + partition: 0, + required_acks: 0, + timeout: 500, + topic: name + }) end test "when the partition is not found", %{worker: worker} do partition = 42 - assert :topic_not_found == KafkaEx.fetch( - @topic, - partition, - worker_name: worker, - offset: 0, - auto_commit: false - ) + + assert :topic_not_found == + KafkaEx.fetch( + @topic, + partition, + worker_name: worker, + offset: 0, + auto_commit: false + ) end end diff --git a/test/integration/server0_p_9_p_0_test.exs b/test/integration/server0_p_9_p_0_test.exs index 265cbf50..e8c8781d 100644 --- a/test/integration/server0_p_9_p_0_test.exs +++ b/test/integration/server0_p_9_p_0_test.exs @@ -11,13 +11,18 @@ defmodule KafkaEx.Server0P9P0.Test do test "can join a consumer group" do random_group = generate_random_string() - KafkaEx.create_worker(:join_group, [uris: uris(), consumer_group: random_group]) + + KafkaEx.create_worker( + :join_group, + uris: uris(), + consumer_group: random_group + ) request = %JoinGroupRequest{ group_name: random_group, member_id: "", topics: ["foo", "bar"], - session_timeout: 6000, + session_timeout: 6000 } answer = KafkaEx.join_group(request, worker_name: :join_group) @@ -31,13 +36,18 @@ defmodule KafkaEx.Server0P9P0.Test do # A lot of repetition with the previous test. Leaving it in now, waiting for # how this pans out eventually as we add more and more 0.9 consumer group code random_group = generate_random_string() - KafkaEx.create_worker(:sync_group, [uris: uris(), consumer_group: random_group]) + + KafkaEx.create_worker( + :sync_group, + uris: uris(), + consumer_group: random_group + ) request = %JoinGroupRequest{ group_name: random_group, member_id: "", topics: ["foo", "bar"], - session_timeout: 6000, + session_timeout: 6000 } answer = KafkaEx.join_group(request, worker_name: :sync_group) @@ -52,11 +62,12 @@ defmodule KafkaEx.Server0P9P0.Test do group_name: random_group, member_id: member_id, generation_id: generation_id, - assignments: assignments, + assignments: assignments } answer = KafkaEx.sync_group(request, worker_name: :sync_group) assert answer.error_code == :no_error + # Parsing happens to return the assignments reversed, which is fine as there's no # ordering. Just reverse what we expect to match assert answer.assignments == Enum.reverse(my_assignments) @@ -66,13 +77,18 @@ defmodule KafkaEx.Server0P9P0.Test do # A lot of repetition with the previous tests. Leaving it in now, waiting for # how this pans out eventually as we add more and more 0.9 consumer group code random_group = generate_random_string() - KafkaEx.create_worker(:leave_group, [uris: uris(), consumer_group: random_group]) + + KafkaEx.create_worker( + :leave_group, + uris: uris(), + consumer_group: random_group + ) request = %JoinGroupRequest{ group_name: random_group, member_id: "", topics: ["foo", "bar"], - session_timeout: 6000, + session_timeout: 6000 } answer = KafkaEx.join_group(request, worker_name: :leave_group) @@ -82,7 +98,7 @@ defmodule KafkaEx.Server0P9P0.Test do request = %LeaveGroupRequest{ group_name: random_group, - member_id: member_id, + member_id: member_id } answer = KafkaEx.leave_group(request, worker_name: :leave_group) @@ -92,13 +108,18 @@ defmodule KafkaEx.Server0P9P0.Test do test "can heartbeat" do # See sync test. Removing repetition in the next iteration random_group = generate_random_string() - KafkaEx.create_worker(:heartbeat, [uris: uris(), consumer_group: random_group]) + + KafkaEx.create_worker( + :heartbeat, + uris: uris(), + consumer_group: random_group + ) request = %JoinGroupRequest{ group_name: random_group, member_id: "", topics: ["foo", "bar"], - session_timeout: 6000, + session_timeout: 6000 } answer = KafkaEx.join_group(request, worker_name: :heartbeat) @@ -113,7 +134,7 @@ defmodule KafkaEx.Server0P9P0.Test do group_name: random_group, member_id: member_id, generation_id: generation_id, - assignments: assignments, + assignments: assignments } answer = KafkaEx.sync_group(request, worker_name: :heartbeat) @@ -122,7 +143,7 @@ defmodule KafkaEx.Server0P9P0.Test do request = %HeartbeatRequest{ group_name: random_group, member_id: member_id, - generation_id: generation_id, + generation_id: generation_id } answer = KafkaEx.heartbeat(request, worker_name: :heartbeat) diff --git a/test/kafka_ex/api_versions_test.exs b/test/kafka_ex/api_versions_test.exs index 7b96e9fc..f4eeb1ad 100644 --- a/test/kafka_ex/api_versions_test.exs +++ b/test/kafka_ex/api_versions_test.exs @@ -1,42 +1,69 @@ defmodule ApiVersionsTest do use ExUnit.Case + @api_versions %{ - 0 => %KafkaEx.Protocol.ApiVersions.ApiVersion { + 0 => %KafkaEx.Protocol.ApiVersions.ApiVersion{ api_key: 0, min_version: 0, - max_version: 3, + max_version: 3 }, - 1 => %KafkaEx.Protocol.ApiVersions.ApiVersion { + 1 => %KafkaEx.Protocol.ApiVersions.ApiVersion{ api_key: 1, min_version: 7, - max_version: 8, + max_version: 8 }, - 3 => %KafkaEx.Protocol.ApiVersions.ApiVersion { + 3 => %KafkaEx.Protocol.ApiVersions.ApiVersion{ api_key: 3, min_version: 2, - max_version: 4, + max_version: 4 } } test "can correctly determine the adequate api version when api versions is not support" do - assert {:ok, 1} == KafkaEx.ApiVersions.find_api_version([:unsupported], :metadata, {1, 3}) + assert {:ok, 1} == + KafkaEx.ApiVersions.find_api_version( + [:unsupported], + :metadata, + {1, 3} + ) end test "KafkaEx.ApiVersions can correctly determine the adequate api version when a match exists" do - assert {:ok, 3} == KafkaEx.ApiVersions.find_api_version(@api_versions, :metadata, {1, 3}) - assert {:ok, 0} == KafkaEx.ApiVersions.find_api_version(@api_versions, :produce, {0, 0}) + assert {:ok, 3} == + KafkaEx.ApiVersions.find_api_version( + @api_versions, + :metadata, + {1, 3} + ) + + assert {:ok, 0} == + KafkaEx.ApiVersions.find_api_version( + @api_versions, + :produce, + {0, 0} + ) end test "KafkaEx.ApiVersions replies an error when there's no version match" do - assert :no_version_supported == KafkaEx.ApiVersions.find_api_version(@api_versions, :fetch, {0, 6}) + assert :no_version_supported == + KafkaEx.ApiVersions.find_api_version(@api_versions, :fetch, {0, 6}) end test "KafkaEx.ApiVersions replies an error when the api_key is unknown to the server" do - assert :unknown_message_for_server == KafkaEx.ApiVersions.find_api_version(@api_versions, :create_topics, {0, 1}) + assert :unknown_message_for_server == + KafkaEx.ApiVersions.find_api_version( + @api_versions, + :create_topics, + {0, 1} + ) end test "KafkaEx.ApiVersions replies an error when the api_key is unknown to the client" do - assert :unknown_message_for_client == KafkaEx.ApiVersions.find_api_version(@api_versions, :this_does_not_exist, {0, 1}) + assert :unknown_message_for_client == + KafkaEx.ApiVersions.find_api_version( + @api_versions, + :this_does_not_exist, + {0, 1} + ) end end - diff --git a/test/kafka_ex/config_test.exs b/test/kafka_ex/config_test.exs index a55bb3c1..02606dde 100644 --- a/test/kafka_ex/config_test.exs +++ b/test/kafka_ex/config_test.exs @@ -6,13 +6,16 @@ defmodule KafkaEx.ConfigTest do setup do # reset application env after each test env_before = Application.get_all_env(:kafka_ex) - on_exit fn -> + + on_exit(fn -> # this is basically Application.put_all_env for {k, v} <- env_before do Application.put_env(:kafka_ex, k, v) end + :ok - end + end) + :ok end @@ -27,12 +30,12 @@ defmodule KafkaEx.ConfigTest do Application.put_env(:kafka_ex, :ssl_options, nil) assert [] == Config.ssl_options() - Application.put_env(:kafka_ex, :ssl_options, [foo: :bar]) + Application.put_env(:kafka_ex, :ssl_options, foo: :bar) assert [] == Config.ssl_options() end test "ssl_options raises an error if use_ssl is true and ssl_options " <> - "are invalid" do + "are invalid" do Application.put_env(:kafka_ex, :use_ssl, true) # when ssl_options is not set we should get an error @@ -73,10 +76,10 @@ defmodule KafkaEx.ConfigTest do end test "brokers with lazy configuration using fun" do - brokers = - fn -> - get_brokers(port: 8888) - end + brokers = fn -> + get_brokers(port: 8888) + end + Application.put_env(:kafka_ex, :brokers, brokers) assert get_brokers(port: 8888) == Config.brokers() @@ -84,6 +87,7 @@ defmodule KafkaEx.ConfigTest do def get_brokers(opts) do port = Keyword.get(opts, :port) + [ {"elixir-lang.org", port} ] diff --git a/test/kafka_ex/consumer_group/partition_assignment_test.exs b/test/kafka_ex/consumer_group/partition_assignment_test.exs index b22dbfa9..d1e16d82 100644 --- a/test/kafka_ex/consumer_group/partition_assignment_test.exs +++ b/test/kafka_ex/consumer_group/partition_assignment_test.exs @@ -9,10 +9,12 @@ defmodule KafkaEx.ConsumerGroup.PartitionAssignmentTest do partitions = [{topic, 0}, {topic, 1}, {topic, 2}] assignments = PartitionAssignment.round_robin(members, partitions) + expected = %{ "member1" => [{topic, 0}, {topic, 2}], "member2" => [{topic, 1}] } + assert expected == assignments end end diff --git a/test/kafka_ex/gen_consumer_test.exs b/test/kafka_ex/gen_consumer_test.exs index b2679b1c..282af842 100644 --- a/test/kafka_ex/gen_consumer_test.exs +++ b/test/kafka_ex/gen_consumer_test.exs @@ -20,7 +20,7 @@ defmodule KafkaEx.GenConsumerTest do test "calling handle_info raises an error if there is no implementation" do assert capture_log(fn -> - TestConsumer.handle_info(nil, nil) - end) =~ "unexpected message in handle_info" + TestConsumer.handle_info(nil, nil) + end) =~ "unexpected message in handle_info" end end diff --git a/test/kafka_ex_test.exs b/test/kafka_ex_test.exs index 4078ad9f..d6bf1b8e 100644 --- a/test/kafka_ex_test.exs +++ b/test/kafka_ex_test.exs @@ -11,7 +11,7 @@ defmodule KafkaExTest do # the supervisor should now have no children and the default worker should not be registered assert [] == Supervisor.which_children(KafkaEx.Supervisor) - assert nil == Process.whereis(Config.default_worker) + assert nil == Process.whereis(Config.default_worker()) # revert the change, restart the application :ok = Application.stop(:kafka_ex) @@ -19,9 +19,10 @@ defmodule KafkaExTest do {:ok, _} = Application.ensure_all_started(:kafka_ex) # we should have the default worker back again - pid = Process.whereis(Config.default_worker) + pid = Process.whereis(Config.default_worker()) assert is_pid(pid) - assert [{:undefined, pid, :worker, [Config.server_impl]}] == - Supervisor.which_children(KafkaEx.Supervisor) + + assert [{:undefined, pid, :worker, [Config.server_impl()]}] == + Supervisor.which_children(KafkaEx.Supervisor) end end diff --git a/test/protocol/consumer_metadata_test.exs b/test/protocol/consumer_metadata_test.exs index 834140a2..1eef1bda 100644 --- a/test/protocol/consumer_metadata_test.exs +++ b/test/protocol/consumer_metadata_test.exs @@ -2,29 +2,61 @@ defmodule KafkaEx.Protocol.ConsumerMetadata.Test do use ExUnit.Case, async: true test "create_request creates a valid consumer metadata request" do - good_request = <<10 :: 16, 0 :: 16, 1 :: 32, 3 :: 16, "foo", 2 :: 16, "we" >> + good_request = <<10::16, 0::16, 1::32, 3::16, "foo", 2::16, "we">> request = KafkaEx.Protocol.ConsumerMetadata.create_request(1, "foo", "we") assert request == good_request end test "parse_response correctly parses a valid response" do - response = <<0, 0, 156, 65, 0, 0, 0, 0, 192, 6, 0, 14, 49, 57, 50, 46, 49, 54, 56, 46, 53, 57, 46, 49, 48, 51, 0, 0, 192, 6>> + response = + <<0, 0, 156, 65, 0, 0, 0, 0, 192, 6, 0, 14, 49, 57, 50, 46, 49, 54, 56, + 46, 53, 57, 46, 49, 48, 51, 0, 0, 192, 6>> - assert KafkaEx.Protocol.ConsumerMetadata.parse_response(response) == %KafkaEx.Protocol.ConsumerMetadata.Response{coordinator_id: 49158, coordinator_host: "192.168.59.103", coordinator_port: 49158, error_code: :no_error} + assert KafkaEx.Protocol.ConsumerMetadata.parse_response(response) == + %KafkaEx.Protocol.ConsumerMetadata.Response{ + coordinator_id: 49158, + coordinator_host: "192.168.59.103", + coordinator_port: 49158, + error_code: :no_error + } end test "Response.broker_for_consumer_group returns correct coordinator_broker" do fake_socket = Port.open({:spawn, "cat"}, []) socket = %KafkaEx.Socket{socket: fake_socket} # Note that just the coordinator_id is used here for comparison - consumer_group_metadata = %KafkaEx.Protocol.ConsumerMetadata.Response{coordinator_host: "127.0.0.1", coordinator_id: 49162, coordinator_port: 8123, error_code: :no_error} + consumer_group_metadata = %KafkaEx.Protocol.ConsumerMetadata.Response{ + coordinator_host: "127.0.0.1", + coordinator_id: 49162, + coordinator_port: 8123, + error_code: :no_error + } brokers = [ - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", port: 9092, node_id: 42, socket: socket}, - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.59.103", port: 49162, node_id: 49162, socket: socket} + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + port: 9092, + node_id: 42, + socket: socket + }, + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.59.103", + port: 49162, + node_id: 49162, + socket: socket + } ] - assert KafkaEx.Protocol.ConsumerMetadata.Response.broker_for_consumer_group(brokers, consumer_group_metadata) == %KafkaEx.Protocol.Metadata.Broker{host: "192.168.59.103", port: 49162, node_id: 49162, socket: socket} + assert KafkaEx.Protocol.ConsumerMetadata.Response.broker_for_consumer_group( + brokers, + consumer_group_metadata + ) == %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.59.103", + port: 49162, + node_id: 49162, + socket: socket + } + Port.close(fake_socket) end @@ -32,27 +64,62 @@ defmodule KafkaEx.Protocol.ConsumerMetadata.Test do fake_socket = Port.open({:spawn, "cat"}, []) socket = %KafkaEx.Socket{socket: fake_socket} Port.close(fake_socket) - consumer_group_metadata = %KafkaEx.Protocol.ConsumerMetadata.Response{coordinator_host: "192.168.0.103", coordinator_id: 9092, coordinator_port: 9092, error_code: :no_error} + + consumer_group_metadata = %KafkaEx.Protocol.ConsumerMetadata.Response{ + coordinator_host: "192.168.0.103", + coordinator_id: 9092, + coordinator_port: 9092, + error_code: :no_error + } brokers = [ - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", port: 9092, socket: socket}, - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.103", port: 9092, socket: socket} + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + port: 9092, + socket: socket + }, + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.103", + port: 9092, + socket: socket + } ] - assert KafkaEx.Protocol.ConsumerMetadata.Response.broker_for_consumer_group(brokers, consumer_group_metadata) == nil + assert KafkaEx.Protocol.ConsumerMetadata.Response.broker_for_consumer_group( + brokers, + consumer_group_metadata + ) == nil end test "Response.broker_for_consumer_group returns 'nil' when the broker does not exist" do fake_socket = Port.open({:spawn, "cat"}, []) socket = %KafkaEx.Socket{socket: fake_socket} - consumer_group_metadata = %KafkaEx.Protocol.ConsumerMetadata.Response{coordinator_host: "192.168.59.103", coordinator_id: 49162, coordinator_port: 49162, error_code: :no_error} + + consumer_group_metadata = %KafkaEx.Protocol.ConsumerMetadata.Response{ + coordinator_host: "192.168.59.103", + coordinator_id: 49162, + coordinator_port: 49162, + error_code: :no_error + } brokers = [ - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", port: 9092, socket: socket}, - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.103", port: 9092, socket: socket} + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + port: 9092, + socket: socket + }, + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.103", + port: 9092, + socket: socket + } ] - assert KafkaEx.Protocol.ConsumerMetadata.Response.broker_for_consumer_group(brokers, consumer_group_metadata) == nil + assert KafkaEx.Protocol.ConsumerMetadata.Response.broker_for_consumer_group( + brokers, + consumer_group_metadata + ) == nil + Port.close(fake_socket) end end diff --git a/test/protocol/fetch_test.exs b/test/protocol/fetch_test.exs index c2542c82..1e93c890 100644 --- a/test/protocol/fetch_test.exs +++ b/test/protocol/fetch_test.exs @@ -3,7 +3,10 @@ defmodule KafkaEx.Protocol.Fetch.Test do alias KafkaEx.Protocol.Fetch.Message test "create_request creates a valid fetch request" do - good_request = << 1 :: 16, 0 :: 16, 1 :: 32, 3 :: 16, "foo" :: binary, -1 :: 32, 10 :: 32, 1 :: 32, 1 :: 32, 3 :: 16, "bar" :: binary, 1 :: 32, 0 :: 32, 1 :: 64, 10000 :: 32 >> + good_request = + <<1::16, 0::16, 1::32, 3::16, "foo"::binary, -1::32, 10::32, 1::32, 1::32, + 3::16, "bar"::binary, 1::32, 0::32, 1::64, 10000::32>> + fetch_request = %KafkaEx.Protocol.Fetch.Request{ correlation_id: 1, client_id: "foo", @@ -12,157 +15,436 @@ defmodule KafkaEx.Protocol.Fetch.Test do offset: 1, wait_time: 10, min_bytes: 1, - max_bytes: 10_000, + max_bytes: 10_000 } + request = KafkaEx.Protocol.Fetch.create_request(fetch_request) assert request == good_request end test "parse_response correctly parses a valid response with a key and a value" do - response = << 0 :: 32, 1 :: 32, 3 :: 16, "bar" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 10 :: 64, 32 :: 32, 1 :: 64, 20 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, 3 :: 32, "foo" :: binary, 3 :: 32, "bar" :: binary >> - expected_response = [%KafkaEx.Protocol.Fetch.Response{partitions: [%{error_code: :no_error, hw_mark_offset: 10, last_offset: 1, message_set: [%Message{attributes: 0, crc: 0, key: "foo", offset: 1, value: "bar"}], partition: 0}], topic: "bar"}] + response = + <<0::32, 1::32, 3::16, "bar"::binary, 1::32, 0::32, 0::16, 10::64, 32::32, + 1::64, 20::32, 0::32, 0::8, 0::8, 3::32, "foo"::binary, 3::32, + "bar"::binary>> + + expected_response = [ + %KafkaEx.Protocol.Fetch.Response{ + partitions: [ + %{ + error_code: :no_error, + hw_mark_offset: 10, + last_offset: 1, + message_set: [ + %Message{ + attributes: 0, + crc: 0, + key: "foo", + offset: 1, + value: "bar" + } + ], + partition: 0 + } + ], + topic: "bar" + } + ] + assert expected_response == KafkaEx.Protocol.Fetch.parse_response(response) end test "parse_response correctly parses a response with excess bytes" do - response = <<0, 0, 0, 1, 0, 0, 0, 1, 0, 4, 102, 111, 111, 100, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 56, 0, 0, 0, 100, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 17, 254, 46, 107, 157, 0, 0, 255, 255, 255, 255, 0, 0, 0, 3, 104, 101, 121, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 17, 254, 46, 107, 157, 0, 0, 255, 255, 255, 255, 0, 0, 0, 3, 104, 101, 121, 0, 0, 0, 0, 0, 0, 0, 2, 0, 0, 0, 17, 254, 46, 107, 157, 0, 0, 255, 255, 255, 255, 0, 0, 0, 3, 104, 101, 121, 0, 0, 0, 0, 0, 0, 0, 3, 0, 0, 0, 17, 254>> + response = + <<0, 0, 0, 1, 0, 0, 0, 1, 0, 4, 102, 111, 111, 100, 0, 0, 0, 1, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 56, 0, 0, 0, 100, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 17, 254, 46, 107, 157, 0, 0, 255, 255, 255, 255, 0, 0, 0, 3, + 104, 101, 121, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 17, 254, 46, 107, 157, + 0, 0, 255, 255, 255, 255, 0, 0, 0, 3, 104, 101, 121, 0, 0, 0, 0, 0, 0, + 0, 2, 0, 0, 0, 17, 254, 46, 107, 157, 0, 0, 255, 255, 255, 255, 0, 0, 0, + 3, 104, 101, 121, 0, 0, 0, 0, 0, 0, 0, 3, 0, 0, 0, 17, 254>> + expected_response = [ - %KafkaEx.Protocol.Fetch.Response{partitions: [ - %{error_code: :no_error, hw_mark_offset: 56, last_offset: 2, message_set: [ - %Message{attributes: 0, crc: 4264455069, key: nil, offset: 0, value: "hey"}, - %Message{attributes: 0, crc: 4264455069, key: nil, offset: 1, value: "hey"}, - %Message{attributes: 0, crc: 4264455069, key: nil, offset: 2, value: "hey"} + %KafkaEx.Protocol.Fetch.Response{ + partitions: [ + %{ + error_code: :no_error, + hw_mark_offset: 56, + last_offset: 2, + message_set: [ + %Message{ + attributes: 0, + crc: 4_264_455_069, + key: nil, + offset: 0, + value: "hey" + }, + %Message{ + attributes: 0, + crc: 4_264_455_069, + key: nil, + offset: 1, + value: "hey" + }, + %Message{ + attributes: 0, + crc: 4_264_455_069, + key: nil, + offset: 2, + value: "hey" + } ], - partition: 0}], topic: "food"} + partition: 0 + } + ], + topic: "food" + } ] + assert expected_response == KafkaEx.Protocol.Fetch.parse_response(response) end test "parse_response correctly parses a valid response with a nil key and a value" do - response = << 0 :: 32, 1 :: 32, 3 :: 16, "bar" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 10 :: 64, 29 :: 32, 1 :: 64, 17 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, - -1 :: 32, 3 :: 32, "bar" :: binary >> - expected_response = [%KafkaEx.Protocol.Fetch.Response{partitions: [%{error_code: :no_error, hw_mark_offset: 10, last_offset: 1, message_set: [%Message{attributes: 0, crc: 0, key: nil, offset: 1, value: "bar"}], partition: 0}], topic: "bar"}] + response = + <<0::32, 1::32, 3::16, "bar"::binary, 1::32, 0::32, 0::16, 10::64, 29::32, + 1::64, 17::32, 0::32, 0::8, 0::8, -1::32, 3::32, "bar"::binary>> + + expected_response = [ + %KafkaEx.Protocol.Fetch.Response{ + partitions: [ + %{ + error_code: :no_error, + hw_mark_offset: 10, + last_offset: 1, + message_set: [ + %Message{attributes: 0, crc: 0, key: nil, offset: 1, value: "bar"} + ], + partition: 0 + } + ], + topic: "bar" + } + ] + assert expected_response == KafkaEx.Protocol.Fetch.parse_response(response) end test "parse_response correctly parses a valid response with a key and a nil value" do - response = << 0 :: 32, 1 :: 32, 3 :: 16, "bar" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 10 :: 64, 29 :: 32, 1 :: 64, 17 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, - 3 :: 32, "foo" :: binary, -1 :: 32 >> - expected_response = [%KafkaEx.Protocol.Fetch.Response{partitions: [%{error_code: :no_error, hw_mark_offset: 10, last_offset: 1, message_set: [%Message{attributes: 0, crc: 0, key: "foo", offset: 1, value: nil}], partition: 0}], topic: "bar"}] + response = + <<0::32, 1::32, 3::16, "bar"::binary, 1::32, 0::32, 0::16, 10::64, 29::32, + 1::64, 17::32, 0::32, 0::8, 0::8, 3::32, "foo"::binary, -1::32>> + + expected_response = [ + %KafkaEx.Protocol.Fetch.Response{ + partitions: [ + %{ + error_code: :no_error, + hw_mark_offset: 10, + last_offset: 1, + message_set: [ + %Message{attributes: 0, crc: 0, key: "foo", offset: 1, value: nil} + ], + partition: 0 + } + ], + topic: "bar" + } + ] + assert expected_response == KafkaEx.Protocol.Fetch.parse_response(response) end test "parse_response correctly parses a valid response with multiple messages" do - response = << 0 :: 32, 1 :: 32, 3 :: 16, "bar" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 10 :: 64, 58 :: 32, 1 :: 64, 17 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, - -1 :: 32, 3 :: 32, "bar" :: binary, 2 :: 64, 17 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, -1 :: 32, 3 :: 32, "baz" :: binary >> - expected_response = [ - %KafkaEx.Protocol.Fetch.Response{partitions: [ - %{error_code: :no_error, hw_mark_offset: 10, last_offset: 2, message_set: [ - %Message{attributes: 0, crc: 0, key: nil, offset: 1, value: "bar"}, - %Message{attributes: 0, crc: 0, key: nil, offset: 2, value: "baz"} - ], partition: 0} - ], topic: "bar"} + response = + <<0::32, 1::32, 3::16, "bar"::binary, 1::32, 0::32, 0::16, 10::64, 58::32, + 1::64, 17::32, 0::32, 0::8, 0::8, -1::32, 3::32, "bar"::binary, 2::64, + 17::32, 0::32, 0::8, 0::8, -1::32, 3::32, "baz"::binary>> + + expected_response = [ + %KafkaEx.Protocol.Fetch.Response{ + partitions: [ + %{ + error_code: :no_error, + hw_mark_offset: 10, + last_offset: 2, + message_set: [ + %Message{ + attributes: 0, + crc: 0, + key: nil, + offset: 1, + value: "bar" + }, + %Message{attributes: 0, crc: 0, key: nil, offset: 2, value: "baz"} + ], + partition: 0 + } + ], + topic: "bar" + } ] + assert expected_response == KafkaEx.Protocol.Fetch.parse_response(response) end test "parse_response correctly parses a valid response with multiple partitions" do - response = << 0 :: 32, 1 :: 32, 3 :: 16, "bar" :: binary, 2 :: 32, 0 :: 32, 0 :: 16, 10 :: 64, 29 :: 32, 1 :: 64, 17 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, - -1 :: 32, 3 :: 32, "bar" :: binary, 1 :: 32, 0 :: 16, 10 :: 64, 29 :: 32, 1 :: 64, 17 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, -1 :: 32, 3 :: 32, "baz" :: binary >> + response = + <<0::32, 1::32, 3::16, "bar"::binary, 2::32, 0::32, 0::16, 10::64, 29::32, + 1::64, 17::32, 0::32, 0::8, 0::8, -1::32, 3::32, "bar"::binary, 1::32, + 0::16, 10::64, 29::32, 1::64, 17::32, 0::32, 0::8, 0::8, -1::32, 3::32, + "baz"::binary>> + expected_response = [ - %KafkaEx.Protocol.Fetch.Response{partitions: [ - %{error_code: :no_error, hw_mark_offset: 10, last_offset: 1, message_set: [%Message{attributes: 0, crc: 0, key: nil, offset: 1, value: "baz"}], partition: 1}, - %{error_code: :no_error, hw_mark_offset: 10, last_offset: 1, message_set: [%Message{attributes: 0, crc: 0, key: nil, offset: 1, value: "bar"}], partition: 0} - ], topic: "bar"} + %KafkaEx.Protocol.Fetch.Response{ + partitions: [ + %{ + error_code: :no_error, + hw_mark_offset: 10, + last_offset: 1, + message_set: [ + %Message{attributes: 0, crc: 0, key: nil, offset: 1, value: "baz"} + ], + partition: 1 + }, + %{ + error_code: :no_error, + hw_mark_offset: 10, + last_offset: 1, + message_set: [ + %Message{attributes: 0, crc: 0, key: nil, offset: 1, value: "bar"} + ], + partition: 0 + } + ], + topic: "bar" + } ] assert expected_response == KafkaEx.Protocol.Fetch.parse_response(response) end test "parse_response correctly parses a valid response with multiple topics" do - response = << 0 :: 32, 2 :: 32, - 3 :: 16, "bar" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 10 :: 64, 29 :: 32, 1 :: 64, 17 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, -1 :: 32, 3 :: 32, "foo" :: binary, - 3 :: 16, "baz" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 10 :: 64, 29 :: 32, 1 :: 64, 17 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, -1 :: 32, 3 :: 32, "bar" :: binary >> + response = + <<0::32, 2::32, 3::16, "bar"::binary, 1::32, 0::32, 0::16, 10::64, 29::32, + 1::64, 17::32, 0::32, 0::8, 0::8, -1::32, 3::32, "foo"::binary, 3::16, + "baz"::binary, 1::32, 0::32, 0::16, 10::64, 29::32, 1::64, 17::32, + 0::32, 0::8, 0::8, -1::32, 3::32, "bar"::binary>> + expected_response = [ - %KafkaEx.Protocol.Fetch.Response{partitions: - [ - %{error_code: :no_error, hw_mark_offset: 10, last_offset: 1, message_set: [%Message{attributes: 0, crc: 0, key: nil, offset: 1, value: "foo"}], partition: 0} + %KafkaEx.Protocol.Fetch.Response{ + partitions: [ + %{ + error_code: :no_error, + hw_mark_offset: 10, + last_offset: 1, + message_set: [ + %Message{attributes: 0, crc: 0, key: nil, offset: 1, value: "foo"} + ], + partition: 0 + } ], - topic: "bar"}, - %KafkaEx.Protocol.Fetch.Response{partitions: - [ - %{error_code: :no_error, hw_mark_offset: 10, last_offset: 1, message_set: [%Message{attributes: 0, crc: 0, key: nil, offset: 1, value: "bar"}], partition: 0} + topic: "bar" + }, + %KafkaEx.Protocol.Fetch.Response{ + partitions: [ + %{ + error_code: :no_error, + hw_mark_offset: 10, + last_offset: 1, + message_set: [ + %Message{attributes: 0, crc: 0, key: nil, offset: 1, value: "bar"} + ], + partition: 0 + } ], - topic: "baz"} + topic: "baz" + } ] assert expected_response == KafkaEx.Protocol.Fetch.parse_response(response) end test "parse_response correctly parses a valid response with a gzip-encoded message" do - response = <<0, 0, 0, 4, 0, 0, 0, 1, 0, 9, 103, 122, 105, 112, 95, 116, 101, 115, 116, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 74, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 62, 38, 244, 178, 37, 0, 1, 255, 255, 255, 255, 0, 0, 0, 48, 31, 139, 8, 0, 0, 0, 0, 0, 0, 0, 99, 96, 128, 3, 169, 101, 15, 206, 246, 50, 48, 252, 7, 2, 32, 143, 167, 36, 181, 184, 68, 33, 55, 181, 184, 56, 49, 61, 21, 0, 10, 31, 112, 82, 38, 0, 0, 0>> - message = %Message{attributes: 0, crc: 2799750541, key: nil, offset: 0, value: "test message"} - partition1 = %{error_code: :no_error, - hw_mark_offset: 1, - last_offset: 0, - partition: 0, - message_set: [message]} + response = + <<0, 0, 0, 4, 0, 0, 0, 1, 0, 9, 103, 122, 105, 112, 95, 116, 101, 115, + 116, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 74, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 62, 38, 244, 178, 37, 0, 1, 255, 255, + 255, 255, 0, 0, 0, 48, 31, 139, 8, 0, 0, 0, 0, 0, 0, 0, 99, 96, 128, 3, + 169, 101, 15, 206, 246, 50, 48, 252, 7, 2, 32, 143, 167, 36, 181, 184, + 68, 33, 55, 181, 184, 56, 49, 61, 21, 0, 10, 31, 112, 82, 38, 0, 0, 0>> + + message = %Message{ + attributes: 0, + crc: 2_799_750_541, + key: nil, + offset: 0, + value: "test message" + } + + partition1 = %{ + error_code: :no_error, + hw_mark_offset: 1, + last_offset: 0, + partition: 0, + message_set: [message] + } + expected_response = [ - %KafkaEx.Protocol.Fetch.Response{partitions: [partition1], - topic: "gzip_test"} + %KafkaEx.Protocol.Fetch.Response{ + partitions: [partition1], + topic: "gzip_test" + } ] + assert expected_response == KafkaEx.Protocol.Fetch.parse_response(response) end - test "parse_response correctly parses a valid response with batched gzip-encoded messages" do - response = <<0, 0, 0, 3, 0, 0, 0, 1, 0, 15, 103, 122, 105, 112, 95, 98, 97, 116, 99, 104, 95, 116, 101, 115, 116, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4, 0, 0, 0, 180, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 74, 112, 213, 163, 157, 0, 1, 255, 255, 255, 255, 0, 0, 0, 60, 31, 139, 8, 0, 0, 0, 0, 0, 0, 0, 99, 96, 128, 3, 169, 119, 54, 19, 103, 51, 48, 252, 7, 2, 32, 143, 39, 41, 177, 36, 57, 67, 161, 36, 181, 184, 68, 193, 16, 170, 130, 17, 164, 170, 220, 244, 128, 34, 86, 85, 70, 0, 83, 29, 3, 53, 76, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 3, 0, 0, 0, 82, 59, 149, 134, 225, 0, 1, 255, 255, 255, 255, 0, 0, 0, 68, 31, 139, 8, 0, 0, 0, 0, 0, 0, 0, 99, 96, 0, 3, 38, 32, 150, 59, 147, 154, 199, 4, 230, 177, 100, 167, 86, 26, 2, 105, 158, 164, 196, 146, 228, 12, 133, 146, 212, 226, 18, 5, 67, 136, 66, 6, 102, 144, 74, 182, 111, 41, 54, 112, 149, 70, 104, 42, 141, 0, 135, 95, 114, 164, 84, 0, 0, 0>> - message1 = %Message{attributes: 0, crc: 3996946843, key: nil, offset: 0, value: "batch test 1"} - message2 = %Message{attributes: 0, crc: 2000011297, key: nil, offset: 1, value: "batch test 2"} - message3 = %Message{attributes: 0, crc: 3429199362, key: "key1", offset: 2, value: "batch test 1"} - message4 = %Message{attributes: 0, crc: 116810812, key: "key2", offset: 3, value: "batch test 2"} - partition1 = %{error_code: :no_error, - hw_mark_offset: 4, - last_offset: 3, - partition: 0, - message_set: [message1, message2, message3, message4]} + test "parse_response correctly parses a valid response with batched gzip-encoded messages" do + response = + <<0, 0, 0, 3, 0, 0, 0, 1, 0, 15, 103, 122, 105, 112, 95, 98, 97, 116, 99, + 104, 95, 116, 101, 115, 116, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 4, 0, 0, 0, 180, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 74, 112, 213, + 163, 157, 0, 1, 255, 255, 255, 255, 0, 0, 0, 60, 31, 139, 8, 0, 0, 0, 0, + 0, 0, 0, 99, 96, 128, 3, 169, 119, 54, 19, 103, 51, 48, 252, 7, 2, 32, + 143, 39, 41, 177, 36, 57, 67, 161, 36, 181, 184, 68, 193, 16, 170, 130, + 17, 164, 170, 220, 244, 128, 34, 86, 85, 70, 0, 83, 29, 3, 53, 76, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 3, 0, 0, 0, 82, 59, 149, 134, 225, 0, 1, 255, + 255, 255, 255, 0, 0, 0, 68, 31, 139, 8, 0, 0, 0, 0, 0, 0, 0, 99, 96, 0, + 3, 38, 32, 150, 59, 147, 154, 199, 4, 230, 177, 100, 167, 86, 26, 2, + 105, 158, 164, 196, 146, 228, 12, 133, 146, 212, 226, 18, 5, 67, 136, + 66, 6, 102, 144, 74, 182, 111, 41, 54, 112, 149, 70, 104, 42, 141, 0, + 135, 95, 114, 164, 84, 0, 0, 0>> + + message1 = %Message{ + attributes: 0, + crc: 3_996_946_843, + key: nil, + offset: 0, + value: "batch test 1" + } + + message2 = %Message{ + attributes: 0, + crc: 2_000_011_297, + key: nil, + offset: 1, + value: "batch test 2" + } + + message3 = %Message{ + attributes: 0, + crc: 3_429_199_362, + key: "key1", + offset: 2, + value: "batch test 1" + } + + message4 = %Message{ + attributes: 0, + crc: 116_810_812, + key: "key2", + offset: 3, + value: "batch test 2" + } + + partition1 = %{ + error_code: :no_error, + hw_mark_offset: 4, + last_offset: 3, + partition: 0, + message_set: [message1, message2, message3, message4] + } + expected_response = [ - %KafkaEx.Protocol.Fetch.Response{partitions: [partition1], topic: "gzip_batch_test"} + %KafkaEx.Protocol.Fetch.Response{ + partitions: [partition1], + topic: "gzip_batch_test" + } ] + assert expected_response == KafkaEx.Protocol.Fetch.parse_response(response) end test "parse_response correctly parses a valid response with a snappy-encoded message" do - response = <<0, 0, 0, 8, 0, 0, 0, 1, 0, 11, 115, 110, 97, 112, 112, 121, 95, 116, 101, 115, 116, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, 0, 0, 0, 83, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 71, 183, 227, 95, 48, 0, 2, 255, 255, 255, 255, 0, 0, 0, 57, 130, 83, 78, 65, 80, 80, 89, 0, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 0, 37, 38, 0, 0, 9, 1, 120, 1, 0, 0, 0, 26, 166, 224, 205, 141, 0, 0, 255, 255, 255, 255, 0, 0, 0, 12, 116, 101, 115, 116, 32, 109, 101, 115, 115, 97, 103, 101>> + response = + <<0, 0, 0, 8, 0, 0, 0, 1, 0, 11, 115, 110, 97, 112, 112, 121, 95, 116, + 101, 115, 116, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, 0, + 0, 0, 83, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 71, 183, 227, 95, 48, 0, 2, + 255, 255, 255, 255, 0, 0, 0, 57, 130, 83, 78, 65, 80, 80, 89, 0, 0, 0, + 0, 1, 0, 0, 0, 1, 0, 0, 0, 37, 38, 0, 0, 9, 1, 120, 1, 0, 0, 0, 26, 166, + 224, 205, 141, 0, 0, 255, 255, 255, 255, 0, 0, 0, 12, 116, 101, 115, + 116, 32, 109, 101, 115, 115, 97, 103, 101>> + value = "test message" - message = %Message{attributes: 0, crc: 2799750541, key: nil, offset: 1, - value: value} - partition1 = %{error_code: :no_error, - hw_mark_offset: 2, - last_offset: 1, - partition: 1, - message_set: [message]} + + message = %Message{ + attributes: 0, + crc: 2_799_750_541, + key: nil, + offset: 1, + value: value + } + + partition1 = %{ + error_code: :no_error, + hw_mark_offset: 2, + last_offset: 1, + partition: 1, + message_set: [message] + } + expected_response = [ - %KafkaEx.Protocol.Fetch.Response{partitions: [partition1], - topic: "snappy_test"} + %KafkaEx.Protocol.Fetch.Response{ + partitions: [partition1], + topic: "snappy_test" + } ] + assert expected_response == KafkaEx.Protocol.Fetch.parse_response(response) end - test "parse_response correctly parses a valid response with batched snappy-encoded messages" do - response = <<0, 0, 0, 14, 0, 0, 0, 1, 0, 17, 115, 110, 97, 112, 112, 121, 95, 98, 97, 116, 99, 104, 95, 116, 101, 115, 116, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, 0, 0, 0, 105, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 93, 70, 199, 142, 116, 0, 2, 255, 255, 255, 255, 0, 0, 0, 79, 130, 83, 78, 65, 80, 80, 89, 0, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 0, 59, 84, 0, 0, 25, 1, 16, 30, 204, 101, 110, 2, 5, 15, 76, 4, 107, 101, 121, 49, 0, 0, 0, 12, 98, 97, 116, 99, 104, 32, 116, 101, 115, 116, 32, 1, 16, 1, 1, 32, 1, 0, 0, 0, 30, 6, 246, 100, 60, 1, 13, 5, 42, 0, 50, 58, 42, 0, 0, 50>> - message1 = %Message{attributes: 0, crc: 3429199362, key: "key1", offset: 0, - value: "batch test 1"} - message2 = %Message{attributes: 0, crc: 116810812, key: "key2", offset: 1, - value: "batch test 2"} - partition1 = %{error_code: :no_error, - hw_mark_offset: 2, - last_offset: 1, - partition: 0, - message_set: [message1, message2]} + test "parse_response correctly parses a valid response with batched snappy-encoded messages" do + response = + <<0, 0, 0, 14, 0, 0, 0, 1, 0, 17, 115, 110, 97, 112, 112, 121, 95, 98, 97, + 116, 99, 104, 95, 116, 101, 115, 116, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 2, 0, 0, 0, 105, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 93, + 70, 199, 142, 116, 0, 2, 255, 255, 255, 255, 0, 0, 0, 79, 130, 83, 78, + 65, 80, 80, 89, 0, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 0, 59, 84, 0, 0, 25, 1, + 16, 30, 204, 101, 110, 2, 5, 15, 76, 4, 107, 101, 121, 49, 0, 0, 0, 12, + 98, 97, 116, 99, 104, 32, 116, 101, 115, 116, 32, 1, 16, 1, 1, 32, 1, 0, + 0, 0, 30, 6, 246, 100, 60, 1, 13, 5, 42, 0, 50, 58, 42, 0, 0, 50>> + + message1 = %Message{ + attributes: 0, + crc: 3_429_199_362, + key: "key1", + offset: 0, + value: "batch test 1" + } + + message2 = %Message{ + attributes: 0, + crc: 116_810_812, + key: "key2", + offset: 1, + value: "batch test 2" + } + + partition1 = %{ + error_code: :no_error, + hw_mark_offset: 2, + last_offset: 1, + partition: 0, + message_set: [message1, message2] + } + expected_response = [ - %KafkaEx.Protocol.Fetch.Response{partitions: [partition1], - topic: "snappy_batch_test"} + %KafkaEx.Protocol.Fetch.Response{ + partitions: [partition1], + topic: "snappy_batch_test" + } ] + assert expected_response == KafkaEx.Protocol.Fetch.parse_response(response) end end diff --git a/test/protocol/heartbeat_test.exs b/test/protocol/heartbeat_test.exs index 7ad16aaa..99203709 100644 --- a/test/protocol/heartbeat_test.exs +++ b/test/protocol/heartbeat_test.exs @@ -3,27 +3,50 @@ defmodule KafkaEx.Protocol.Heartbeat.Test do test "create_request creates a valid heartbeat request" do good_request = << - 12 :: 16, 0 :: 16, 42 :: 32, 9 :: 16, "client_id" :: binary, # Preamble - 8 :: 16, "group_id", - 1234 :: 32, # GenerationId - 9 :: 16, "member_id" :: binary # MemberId + # Preamble + 12::16, + 0::16, + 42::32, + 9::16, + "client_id"::binary, + 8::16, + "group_id", + # GenerationId + 1234::32, + # MemberId + 9::16, + "member_id"::binary >> + heartbeat_request = %KafkaEx.Protocol.Heartbeat.Request{ group_name: "group_id", member_id: "member_id", - generation_id: 1234, + generation_id: 1234 } - request = KafkaEx.Protocol.Heartbeat.create_request(42, "client_id", heartbeat_request) + request = + KafkaEx.Protocol.Heartbeat.create_request( + 42, + "client_id", + heartbeat_request + ) + assert request == good_request end test "parse success response correctly" do response = << - 42 :: 32, # CorrelationId - 0 :: 16 # ErrorCode + # CorrelationId + 42::32, + # ErrorCode + 0::16 >> - expected_response = %KafkaEx.Protocol.Heartbeat.Response{error_code: :no_error} - assert KafkaEx.Protocol.Heartbeat.parse_response(response) == expected_response + + expected_response = %KafkaEx.Protocol.Heartbeat.Response{ + error_code: :no_error + } + + assert KafkaEx.Protocol.Heartbeat.parse_response(response) == + expected_response end end diff --git a/test/protocol/join_group_test.exs b/test/protocol/join_group_test.exs index f67f0495..1be74b84 100644 --- a/test/protocol/join_group_test.exs +++ b/test/protocol/join_group_test.exs @@ -1,46 +1,94 @@ -defmodule KafkaEx.Protocol.JoinGroup.Test do +defmodule KafkaEx.Protocol.JoinGroup.Test do use ExUnit.Case, async: true alias KafkaEx.Protocol.JoinGroup test "create_request creates a valid join group request" do - good_request = << - 11 :: 16, 0 :: 16, 42 :: 32, 9 :: 16, "client_id" :: binary, # Preamble - 5 :: 16, "group" :: binary, # GroupId - 3600 :: 32, # SessionTimeout - 9 :: 16, "member_id" :: binary, # MemberId - 8 :: 16, "consumer" :: binary, # ProtocolType - 1 :: 32, # GroupProtocols array size - 6 :: 16, "assign" :: binary, # Basic strategy, "roundrobin" has some restrictions - 32 :: 32, # length of metadata - 0 :: 16, # v0 - 2 :: 32, 9 :: 16, "topic_one" :: binary, 9 :: 16, "topic_two" :: binary, # Topics array - 0 :: 32 >> # UserData - request = JoinGroup.create_request(42, "client_id", - %JoinGroup.Request{ + good_request = + << + # Preamble + 11::16, + 0::16, + 42::32, + 9::16, + "client_id"::binary, + # GroupId + 5::16, + "group"::binary, + # SessionTimeout + 3600::32, + # MemberId + 9::16, + "member_id"::binary, + # ProtocolType + 8::16, + "consumer"::binary, + # GroupProtocols array size + 1::32, + # Basic strategy, "roundrobin" has some restrictions + 6::16, + "assign"::binary, + # length of metadata + 32::32, + # v0 + 0::16, + # Topics array + 2::32, + 9::16, + "topic_one"::binary, + 9::16, + "topic_two"::binary, + # UserData + 0::32 + >> + + request = + JoinGroup.create_request(42, "client_id", %JoinGroup.Request{ member_id: "member_id", group_name: "group", topics: ["topic_one", "topic_two"], session_timeout: 3600 - } - ) + }) + assert request == good_request end test "parse success response correctly" do response = << - 42 :: 32, # CorrelationId - 0 :: 16, # ErrorCode - 123 :: 32, # GenerationId - 8 :: 16, "consumer" :: binary, # GroupProtocol - 10 :: 16, "member_xxx" :: binary, # LeaderId - 10 :: 16, "member_one" :: binary, # MemberId - 2 :: 32, # Members array - 10 :: 16, "member_one", 12 :: 32, "metadata_one", - 10 :: 16, "member_two", 12 :: 32, "metadata_two" - >> - expected_response = %JoinGroup.Response{error_code: :no_error, - generation_id: 123, leader_id: "member_xxx", - member_id: "member_one", members: ["member_two", "member_one"]} + # CorrelationId + 42::32, + # ErrorCode + 0::16, + # GenerationId + 123::32, + # GroupProtocol + 8::16, + "consumer"::binary, + # LeaderId + 10::16, + "member_xxx"::binary, + # MemberId + 10::16, + "member_one"::binary, + # Members array + 2::32, + 10::16, + "member_one", + 12::32, + "metadata_one", + 10::16, + "member_two", + 12::32, + "metadata_two" + >> + + expected_response = %JoinGroup.Response{ + error_code: :no_error, + generation_id: 123, + leader_id: "member_xxx", + member_id: "member_one", + members: ["member_two", "member_one"] + } + assert JoinGroup.parse_response(response) == expected_response end end diff --git a/test/protocol/leave_group_test.exs b/test/protocol/leave_group_test.exs index 176f5a20..2f997c74 100644 --- a/test/protocol/leave_group_test.exs +++ b/test/protocol/leave_group_test.exs @@ -3,29 +3,44 @@ defmodule KafkaEx.Protocol.LeaveGroup.Test do test "create_request creates a valid leave group request" do good_request = << - 13 :: 16, 0 :: 16, 42 :: 32, 9 :: 16, "client_id" :: binary, # Preamble - byte_size("group") :: 16, "group" :: binary, # GroupId - byte_size("member") :: 16, "member" :: binary, # MemberId - >> + # Preamble + 13::16, + 0::16, + 42::32, + 9::16, + "client_id"::binary, + # GroupId + byte_size("group")::16, + "group"::binary, + # MemberId + byte_size("member")::16, + "member"::binary + >> leave_request = %KafkaEx.Protocol.LeaveGroup.Request{ group_name: "group", - member_id: "member", + member_id: "member" } - request = KafkaEx.Protocol.LeaveGroup.create_request(42, "client_id", leave_request) + request = + KafkaEx.Protocol.LeaveGroup.create_request(42, "client_id", leave_request) assert request == good_request end test "parse_response parses successful response correctly" do response = << - 42 :: 32, # CorrelationId - 0 :: 16, # ErrorCode - >> + # CorrelationId + 42::32, + # ErrorCode + 0::16 + >> - expected_response = %KafkaEx.Protocol.LeaveGroup.Response{error_code: :no_error} + expected_response = %KafkaEx.Protocol.LeaveGroup.Response{ + error_code: :no_error + } - assert KafkaEx.Protocol.LeaveGroup.parse_response(response) == expected_response + assert KafkaEx.Protocol.LeaveGroup.parse_response(response) == + expected_response end end diff --git a/test/protocol/metadata_test.exs b/test/protocol/metadata_test.exs index 5e908a1d..a0e4ff2c 100644 --- a/test/protocol/metadata_test.exs +++ b/test/protocol/metadata_test.exs @@ -2,58 +2,117 @@ defmodule KafkaEx.Protocol.Metadata.Test do use ExUnit.Case, async: true test "create_request with no topics creates a valid metadata request" do - good_request = << 3 :: 16, 0 :: 16, 1 :: 32, 3 :: 16, "foo" :: binary, 0 :: 32 >> + good_request = <<3::16, 0::16, 1::32, 3::16, "foo"::binary, 0::32>> request = KafkaEx.Protocol.Metadata.create_request(1, "foo", []) assert request == good_request end test "create_request with a single topic creates a valid metadata request" do - good_request = << 3 :: 16, 0 :: 16, 1 :: 32, 3 :: 16, "foo" :: binary, 1 :: 32, 3 :: 16, "bar" :: binary >> + good_request = + <<3::16, 0::16, 1::32, 3::16, "foo"::binary, 1::32, 3::16, "bar"::binary>> + request = KafkaEx.Protocol.Metadata.create_request(1, "foo", ["bar"]) assert request == good_request end test "create_request with a multiple topics creates a valid metadata request" do - good_request = << 3 :: 16, 0 :: 16, 1 :: 32, 3 :: 16, "foo" :: binary, 3 :: 32, 3 :: 16, "bar" :: binary, 3 :: 16, "baz" :: binary, 4 :: 16, "food" :: binary >> - request = KafkaEx.Protocol.Metadata.create_request(1, "foo", ["bar", "baz", "food"]) + good_request = + <<3::16, 0::16, 1::32, 3::16, "foo"::binary, 3::32, 3::16, "bar"::binary, + 3::16, "baz"::binary, 4::16, "food"::binary>> + + request = + KafkaEx.Protocol.Metadata.create_request(1, "foo", ["bar", "baz", "food"]) + assert request == good_request end test "parse_response correctly parses a valid response" do - response = << 0 :: 32, 1 :: 32, 0 :: 32, 3 :: 16, "foo" :: binary, 9092 :: 32, 1 :: 32, 0 :: 16, 3 :: 16, "bar" :: binary, - 1 :: 32, 0 :: 16, 0 :: 32, 0 :: 32, 0 :: 32, 1 :: 32, 0 :: 32 >> + response = + <<0::32, 1::32, 0::32, 3::16, "foo"::binary, 9092::32, 1::32, 0::16, + 3::16, "bar"::binary, 1::32, 0::16, 0::32, 0::32, 0::32, 1::32, 0::32>> + expected_response = %KafkaEx.Protocol.Metadata.Response{ - brokers: [%KafkaEx.Protocol.Metadata.Broker{host: "foo", node_id: 0, port: 9092}], + brokers: [ + %KafkaEx.Protocol.Metadata.Broker{host: "foo", node_id: 0, port: 9092} + ], topic_metadatas: [ - %KafkaEx.Protocol.Metadata.TopicMetadata{error_code: :no_error, partition_metadatas: [ - %KafkaEx.Protocol.Metadata.PartitionMetadata{error_code: :no_error, isrs: [0], leader: 0, partition_id: 0, replicas: []} - ], topic: "bar"} + %KafkaEx.Protocol.Metadata.TopicMetadata{ + error_code: :no_error, + partition_metadatas: [ + %KafkaEx.Protocol.Metadata.PartitionMetadata{ + error_code: :no_error, + isrs: [0], + leader: 0, + partition_id: 0, + replicas: [] + } + ], + topic: "bar" + } ] } - assert expected_response == KafkaEx.Protocol.Metadata.parse_response(response) + assert expected_response == + KafkaEx.Protocol.Metadata.parse_response(response) end test "Response.broker_for_topic returns correct broker for a topic" do fake_socket = Port.open({:spawn, "cat"}, []) socket = %KafkaEx.Socket{socket: fake_socket} + metadata = %KafkaEx.Protocol.Metadata.Response{ brokers: [ - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", node_id: 9092, port: 9092}, - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", node_id: 9093, port: 9093} + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + node_id: 9092, + port: 9092 + }, + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + node_id: 9093, + port: 9093 + } ], topic_metadatas: [ - %KafkaEx.Protocol.Metadata.TopicMetadata{error_code: :no_error, partition_metadatas: [ - %KafkaEx.Protocol.Metadata.PartitionMetadata{error_code: :no_error, isrs: [0], leader: 9092, partition_id: 0, replicas: []} - ], topic: "bar"} + %KafkaEx.Protocol.Metadata.TopicMetadata{ + error_code: :no_error, + partition_metadatas: [ + %KafkaEx.Protocol.Metadata.PartitionMetadata{ + error_code: :no_error, + isrs: [0], + leader: 9092, + partition_id: 0, + replicas: [] + } + ], + topic: "bar" + } ] } + brokers = [ - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", port: 9092, socket: socket}, - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", port: 9091, socket: socket} + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + port: 9092, + socket: socket + }, + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + port: 9091, + socket: socket + } ] - assert KafkaEx.Protocol.Metadata.Response.broker_for_topic(metadata, brokers, "bar", 0) == %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", port: 9092, socket: socket} + assert KafkaEx.Protocol.Metadata.Response.broker_for_topic( + metadata, + brokers, + "bar", + 0 + ) == %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + port: 9092, + socket: socket + } Port.close(fake_socket) end @@ -61,44 +120,112 @@ defmodule KafkaEx.Protocol.Metadata.Test do test "Response.broker_for_topic returns nil when the topic is not found" do fake_socket = Port.open({:spawn, "cat"}, []) socket = %KafkaEx.Socket{socket: fake_socket} + metadata = %KafkaEx.Protocol.Metadata.Response{ brokers: [ - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", node_id: 9092, port: 9092}, - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", node_id: 9093, port: 9093} + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + node_id: 9092, + port: 9092 + }, + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + node_id: 9093, + port: 9093 + } ], topic_metadatas: [ - %KafkaEx.Protocol.Metadata.TopicMetadata{error_code: :no_error, partition_metadatas: [ - %KafkaEx.Protocol.Metadata.PartitionMetadata{error_code: :no_error, isrs: [0], leader: 9092, partition_id: 0, replicas: []} - ], topic: "bar"} + %KafkaEx.Protocol.Metadata.TopicMetadata{ + error_code: :no_error, + partition_metadatas: [ + %KafkaEx.Protocol.Metadata.PartitionMetadata{ + error_code: :no_error, + isrs: [0], + leader: 9092, + partition_id: 0, + replicas: [] + } + ], + topic: "bar" + } ] } + brokers = [ - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", port: 9092, socket: socket}, - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", port: 9091, socket: socket} + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + port: 9092, + socket: socket + }, + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + port: 9091, + socket: socket + } ] - assert KafkaEx.Protocol.Metadata.Response.broker_for_topic(metadata, brokers, "foo", 0) == nil + + assert KafkaEx.Protocol.Metadata.Response.broker_for_topic( + metadata, + brokers, + "foo", + 0 + ) == nil end test "Response.broker_for_topic returns nil when the partition is not found" do fake_socket = Port.open({:spawn, "cat"}, []) socket = %KafkaEx.Socket{socket: fake_socket} + metadata = %KafkaEx.Protocol.Metadata.Response{ brokers: [ - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", node_id: 9092, port: 9092}, - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", node_id: 9093, port: 9093} + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + node_id: 9092, + port: 9092 + }, + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + node_id: 9093, + port: 9093 + } ], topic_metadatas: [ - %KafkaEx.Protocol.Metadata.TopicMetadata{error_code: :no_error, partition_metadatas: [ - %KafkaEx.Protocol.Metadata.PartitionMetadata{error_code: :no_error, isrs: [0], leader: 9092, partition_id: 0, replicas: []} - ], topic: "bar"} + %KafkaEx.Protocol.Metadata.TopicMetadata{ + error_code: :no_error, + partition_metadatas: [ + %KafkaEx.Protocol.Metadata.PartitionMetadata{ + error_code: :no_error, + isrs: [0], + leader: 9092, + partition_id: 0, + replicas: [] + } + ], + topic: "bar" + } ] } + brokers = [ - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", port: 9092, socket: socket}, - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", port: 9091, socket: socket} + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + port: 9092, + socket: socket + }, + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + port: 9091, + socket: socket + } ] - assert KafkaEx.Protocol.Metadata.Response.broker_for_topic(metadata, brokers, "bar", 1) == nil + assert KafkaEx.Protocol.Metadata.Response.broker_for_topic( + metadata, + brokers, + "bar", + 1 + ) == nil + Port.close(fake_socket) end @@ -106,45 +233,112 @@ defmodule KafkaEx.Protocol.Metadata.Test do fake_socket = Port.open({:spawn, "cat"}, []) socket = %KafkaEx.Socket{socket: fake_socket} Port.close(fake_socket) + metadata = %KafkaEx.Protocol.Metadata.Response{ brokers: [ - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", node_id: 9092, port: 9092}, - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", node_id: 9093, port: 9093} + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + node_id: 9092, + port: 9092 + }, + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + node_id: 9093, + port: 9093 + } ], topic_metadatas: [ - %KafkaEx.Protocol.Metadata.TopicMetadata{error_code: :no_error, partition_metadatas: [ - %KafkaEx.Protocol.Metadata.PartitionMetadata{error_code: :no_error, isrs: [0], leader: 9092, partition_id: 0, replicas: []} - ], topic: "bar"} + %KafkaEx.Protocol.Metadata.TopicMetadata{ + error_code: :no_error, + partition_metadatas: [ + %KafkaEx.Protocol.Metadata.PartitionMetadata{ + error_code: :no_error, + isrs: [0], + leader: 9092, + partition_id: 0, + replicas: [] + } + ], + topic: "bar" + } ] } + brokers = [ - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", port: 9093, socket: socket}, - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", port: 9091, socket: socket} + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + port: 9093, + socket: socket + }, + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + port: 9091, + socket: socket + } ] - assert KafkaEx.Protocol.Metadata.Response.broker_for_topic(metadata, brokers, "bar", 0) == nil + assert KafkaEx.Protocol.Metadata.Response.broker_for_topic( + metadata, + brokers, + "bar", + 0 + ) == nil end test "Response.broker_for_topic returns nil when a matching broker is not found" do fake_socket = Port.open({:spawn, "cat"}, []) socket = %KafkaEx.Socket{socket: fake_socket} + metadata = %KafkaEx.Protocol.Metadata.Response{ brokers: [ - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", node_id: 9092, port: 9092}, - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", node_id: 9093, port: 9093} + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + node_id: 9092, + port: 9092 + }, + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + node_id: 9093, + port: 9093 + } ], topic_metadatas: [ - %KafkaEx.Protocol.Metadata.TopicMetadata{error_code: :no_error, partition_metadatas: [ - %KafkaEx.Protocol.Metadata.PartitionMetadata{error_code: :no_error, isrs: [0], leader: 9092, partition_id: 0, replicas: []} - ], topic: "bar"} + %KafkaEx.Protocol.Metadata.TopicMetadata{ + error_code: :no_error, + partition_metadatas: [ + %KafkaEx.Protocol.Metadata.PartitionMetadata{ + error_code: :no_error, + isrs: [0], + leader: 9092, + partition_id: 0, + replicas: [] + } + ], + topic: "bar" + } ] } + brokers = [ - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", port: 9093, socket: socket}, - %KafkaEx.Protocol.Metadata.Broker{host: "192.168.0.1", port: 9091, socket: socket} + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + port: 9093, + socket: socket + }, + %KafkaEx.Protocol.Metadata.Broker{ + host: "192.168.0.1", + port: 9091, + socket: socket + } ] - assert KafkaEx.Protocol.Metadata.Response.broker_for_topic(metadata, brokers, "bar", 0) == nil + assert KafkaEx.Protocol.Metadata.Response.broker_for_topic( + metadata, + brokers, + "bar", + 0 + ) == nil + Port.close(fake_socket) end @@ -172,11 +366,11 @@ defmodule KafkaEx.Protocol.Metadata.Test do } assert [0, 1, 2, 3] == - MetadataResponse.partitions_for_topic(metadata, "bar") + MetadataResponse.partitions_for_topic(metadata, "bar") end test "Response.partitions_for_topic returns an empty list for a topic " <> - "that does not exist" do + "that does not exist" do alias KafkaEx.Protocol.Metadata.Response, as: MetadataResponse metadata = %MetadataResponse{topic_metadatas: []} diff --git a/test/protocol/offset_commit_test.exs b/test/protocol/offset_commit_test.exs index d6cb2386..880fef5b 100644 --- a/test/protocol/offset_commit_test.exs +++ b/test/protocol/offset_commit_test.exs @@ -4,14 +4,39 @@ defmodule KafkaEx.Protocol.OffsetCommit.Test do test "create_request creates a valid offset commit message" do corr_id = 3 client_id = "kafka_ex" - offset_commit_request = %KafkaEx.Protocol.OffsetCommit.Request{offset: 10, partition: 0, topic: "foo", consumer_group: "bar", metadata: "baz"} - good_request = << 8 :: 16, 0 :: 16, corr_id :: 32, byte_size(client_id) :: 16, client_id :: binary, 3 :: 16, "bar", 1 :: 32, 3 :: 16, "foo", 1 :: 32, 0 :: 32, 10 :: 64, 3 :: 16, "baz" >> - request = KafkaEx.Protocol.OffsetCommit.create_request(corr_id, client_id, offset_commit_request) + + offset_commit_request = %KafkaEx.Protocol.OffsetCommit.Request{ + offset: 10, + partition: 0, + topic: "foo", + consumer_group: "bar", + metadata: "baz" + } + + good_request = + <<8::16, 0::16, corr_id::32, byte_size(client_id)::16, client_id::binary, + 3::16, "bar", 1::32, 3::16, "foo", 1::32, 0::32, 10::64, 3::16, "baz">> + + request = + KafkaEx.Protocol.OffsetCommit.create_request( + corr_id, + client_id, + offset_commit_request + ) + assert request == good_request end test "parse_response correctly parses a valid response" do - response = <<0, 0, 156, 64, 0, 0, 0, 1, 0, 4, 102, 111, 111, 100, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0>> - assert KafkaEx.Protocol.OffsetCommit.parse_response(response) == [%KafkaEx.Protocol.OffsetCommit.Response{partitions: [0], topic: "food"}] + response = + <<0, 0, 156, 64, 0, 0, 0, 1, 0, 4, 102, 111, 111, 100, 0, 0, 0, 1, 0, 0, + 0, 0, 0, 0>> + + assert KafkaEx.Protocol.OffsetCommit.parse_response(response) == [ + %KafkaEx.Protocol.OffsetCommit.Response{ + partitions: [0], + topic: "food" + } + ] end end diff --git a/test/protocol/offset_fetch_test.exs b/test/protocol/offset_fetch_test.exs index 87151c74..e2761d67 100644 --- a/test/protocol/offset_fetch_test.exs +++ b/test/protocol/offset_fetch_test.exs @@ -4,14 +4,39 @@ defmodule KafkaEx.Protocol.OffsetFetch.Test do test "create_request creates a valid offset commit message" do corr_id = 3 client_id = "kafka_ex" - offset_commit_request = %KafkaEx.Protocol.OffsetFetch.Request{topic: "foo", consumer_group: "bar", partition: 0} - good_request = << 9 :: 16, 0 :: 16, 3 :: 32, 8 :: 16, "kafka_ex" :: binary, 3 :: 16, "bar" :: binary, 1 :: 32, 3 :: 16, "foo" :: binary, 1 :: 32, 0 :: 32 >> - request = KafkaEx.Protocol.OffsetFetch.create_request(corr_id, client_id, offset_commit_request) + + offset_commit_request = %KafkaEx.Protocol.OffsetFetch.Request{ + topic: "foo", + consumer_group: "bar", + partition: 0 + } + + good_request = + <<9::16, 0::16, 3::32, 8::16, "kafka_ex"::binary, 3::16, "bar"::binary, + 1::32, 3::16, "foo"::binary, 1::32, 0::32>> + + request = + KafkaEx.Protocol.OffsetFetch.create_request( + corr_id, + client_id, + offset_commit_request + ) + assert request == good_request end test "parse_response correctly parses a valid response" do - response = <<0, 0, 156, 66, 0, 0, 0, 1, 0, 4, 102, 111, 111, 100, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 9, 0, 0, 0, 0>> - assert KafkaEx.Protocol.OffsetFetch.parse_response(response) == [%KafkaEx.Protocol.OffsetFetch.Response{partitions: [%{metadata: "", error_code: :no_error, offset: 9, partition: 0}], topic: "food"}] + response = + <<0, 0, 156, 66, 0, 0, 0, 1, 0, 4, 102, 111, 111, 100, 0, 0, 0, 1, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 9, 0, 0, 0, 0>> + + assert KafkaEx.Protocol.OffsetFetch.parse_response(response) == [ + %KafkaEx.Protocol.OffsetFetch.Response{ + partitions: [ + %{metadata: "", error_code: :no_error, offset: 9, partition: 0} + ], + topic: "food" + } + ] end end diff --git a/test/protocol/offset_test.exs b/test/protocol/offset_test.exs index 91e4a450..25707a96 100644 --- a/test/protocol/offset_test.exs +++ b/test/protocol/offset_test.exs @@ -2,34 +2,74 @@ defmodule KafkaEx.Protocol.Offset.Test do use ExUnit.Case, async: true test "parse_response correctly parses a valid response with an offset" do - response = << 0 :: 32, 1 :: 32, 3 :: 16, "bar" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 1 :: 32, 10 :: 64 >> - expected_response = [%KafkaEx.Protocol.Offset.Response{partition_offsets: [%{error_code: :no_error, offset: '\n', partition: 0}], topic: "bar"}] + response = + <<0::32, 1::32, 3::16, "bar"::binary, 1::32, 0::32, 0::16, 1::32, 10::64>> + + expected_response = [ + %KafkaEx.Protocol.Offset.Response{ + partition_offsets: [ + %{error_code: :no_error, offset: '\n', partition: 0} + ], + topic: "bar" + } + ] assert expected_response == KafkaEx.Protocol.Offset.parse_response(response) end test "parse_response correctly parses a valid response with multiple offsets" do - response = << 0 :: 32, 1 :: 32, 3 :: 16, "bar" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 2 :: 32, 10 :: 64, 20 :: 64 >> - expected_response = [%KafkaEx.Protocol.Offset.Response{partition_offsets: [%{error_code: :no_error, offset: [10, 20], partition: 0}], topic: "bar"}] + response = + <<0::32, 1::32, 3::16, "bar"::binary, 1::32, 0::32, 0::16, 2::32, 10::64, + 20::64>> + + expected_response = [ + %KafkaEx.Protocol.Offset.Response{ + partition_offsets: [ + %{error_code: :no_error, offset: [10, 20], partition: 0} + ], + topic: "bar" + } + ] assert expected_response == KafkaEx.Protocol.Offset.parse_response(response) end test "parse_response correctly parses a valid response with multiple partitions" do - response = << 0 :: 32, 1 :: 32, 3 :: 16, "bar" :: binary, 2 :: 32, 0 :: 32, 0 :: 16, 1 :: 32, 10 :: 64, 1 :: 32, 0 :: 16, 1 :: 32, 20 :: 64 >> - expected_response = [%KafkaEx.Protocol.Offset.Response{partition_offsets: [ + response = + <<0::32, 1::32, 3::16, "bar"::binary, 2::32, 0::32, 0::16, 1::32, 10::64, + 1::32, 0::16, 1::32, 20::64>> + + expected_response = [ + %KafkaEx.Protocol.Offset.Response{ + partition_offsets: [ %{error_code: :no_error, offset: [20], partition: 1}, %{error_code: :no_error, offset: '\n', partition: 0} - ], topic: "bar"}] + ], + topic: "bar" + } + ] assert expected_response == KafkaEx.Protocol.Offset.parse_response(response) end test "parse_response correctly parses a valid response with multiple topics" do - response = << 0 :: 32, 2 :: 32, 3 :: 16, "bar" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 1 :: 32, 10 :: 64, 3 :: 16, "baz" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 1 :: 32, 20 :: 64 >> + response = + <<0::32, 2::32, 3::16, "bar"::binary, 1::32, 0::32, 0::16, 1::32, 10::64, + 3::16, "baz"::binary, 1::32, 0::32, 0::16, 1::32, 20::64>> + expected_response = [ - %KafkaEx.Protocol.Offset.Response{partition_offsets: [%{error_code: :no_error, offset: [10], partition: 0}], topic: "bar"}, - %KafkaEx.Protocol.Offset.Response{partition_offsets: [%{error_code: :no_error, offset: [20], partition: 0}], topic: "baz"} + %KafkaEx.Protocol.Offset.Response{ + partition_offsets: [ + %{error_code: :no_error, offset: [10], partition: 0} + ], + topic: "bar" + }, + %KafkaEx.Protocol.Offset.Response{ + partition_offsets: [ + %{error_code: :no_error, offset: [20], partition: 0} + ], + topic: "baz" + } ] assert expected_response == KafkaEx.Protocol.Offset.parse_response(response) diff --git a/test/protocol/produce_test.exs b/test/protocol/produce_test.exs index 30c0e855..61777413 100644 --- a/test/protocol/produce_test.exs +++ b/test/protocol/produce_test.exs @@ -2,36 +2,77 @@ defmodule KafkaEx.Protocol.Produce.Test do use ExUnit.Case, async: true test "create_request creates a valid payload" do - expected_request = <<0, 0, 0, 0, 0, 0, 0, 1, 0, 3, 102, 111, 111, 0, 1, 0, 0, 0, 10, 0, 0, 0, 1, 0, 4, 102, 111, 111, 100, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 29, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 17, 106, 86, 37, 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, 3, 104, 101, 121>> - - request = KafkaEx.Protocol.Produce.create_request(1, "foo", %KafkaEx.Protocol.Produce.Request{ - topic: "food", partition: 0, required_acks: 1, timeout: 10, compression: :none, messages: [ - %KafkaEx.Protocol.Produce.Message{key: "", value: "hey"}, - ] - }) + expected_request = + <<0, 0, 0, 0, 0, 0, 0, 1, 0, 3, 102, 111, 111, 0, 1, 0, 0, 0, 10, 0, 0, 0, + 1, 0, 4, 102, 111, 111, 100, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 29, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 17, 106, 86, 37, 142, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 3, 104, 101, 121>> + + request = + KafkaEx.Protocol.Produce.create_request( + 1, + "foo", + %KafkaEx.Protocol.Produce.Request{ + topic: "food", + partition: 0, + required_acks: 1, + timeout: 10, + compression: :none, + messages: [ + %KafkaEx.Protocol.Produce.Message{key: "", value: "hey"} + ] + } + ) assert expected_request == :erlang.iolist_to_binary(request) end test "create_request correctly batches multiple request messages" do - expected_request = <<0, 0, 0, 0, 0, 0, 0, 1, 0, 3, 102, 111, 111, 0, 1, 0, 0, 0, 10, 0, 0, 0, 1, 0, 4, 102, 111, 111, 100, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 17, 106, 86, 37, 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, 3, 104, 101, 121, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 16, 225, 27, 42, 82, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, 104, 105, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 19, 119, 44, 195, 207, 0, 0, 0, 0, 0, 0, 0, 0, 0, 5, 104, 101, 108, 108, 111>> - - request = KafkaEx.Protocol.Produce.create_request(1, "foo", %KafkaEx.Protocol.Produce.Request{ - topic: "food", partition: 0, required_acks: 1, timeout: 10, compression: :none, messages: [ - %KafkaEx.Protocol.Produce.Message{key: "", value: "hey"}, - %KafkaEx.Protocol.Produce.Message{key: "", value: "hi"}, - %KafkaEx.Protocol.Produce.Message{key: "", value: "hello"}, - ] - }) + expected_request = + <<0, 0, 0, 0, 0, 0, 0, 1, 0, 3, 102, 111, 111, 0, 1, 0, 0, 0, 10, 0, 0, 0, + 1, 0, 4, 102, 111, 111, 100, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 88, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 17, 106, 86, 37, 142, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 3, 104, 101, 121, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 16, 225, 27, 42, + 82, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, 104, 105, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 19, 119, 44, 195, 207, 0, 0, 0, 0, 0, 0, 0, 0, 0, 5, 104, 101, + 108, 108, 111>> + + request = + KafkaEx.Protocol.Produce.create_request( + 1, + "foo", + %KafkaEx.Protocol.Produce.Request{ + topic: "food", + partition: 0, + required_acks: 1, + timeout: 10, + compression: :none, + messages: [ + %KafkaEx.Protocol.Produce.Message{key: "", value: "hey"}, + %KafkaEx.Protocol.Produce.Message{key: "", value: "hi"}, + %KafkaEx.Protocol.Produce.Message{key: "", value: "hello"} + ] + } + ) assert expected_request == :erlang.iolist_to_binary(request) end test "create_request correctly encodes messages with gzip" do - expected_request = <<0, 0, 0, 0, 0, 0, 0, 1, 0, 23, 99, 111, 109, 112, 114, 101, 115, 115, 105, 111, 110, 95, 99, 108, 105, 101, 110, 116, 95, 116, 101, 115, 116, 0, 1, 0, 0, 0, 10, 0, 0, 0, 1, 0, 16, 99, 111, 109, 112, 114, 101, 115, 115, 101, 100, 95, 116, 111, 112, 105, 99, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 86, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 74, 79, 44, 46, 209, 0, 1, 255, 255, 255, 255, 0, 0, 0, 60, 31, 139, 8, 0, 0, 0, 0, 0, 0, 3, 99, 96, 128, 3, 153, 135, 115, 4, 255, 131, 89, 172, 217, 169, 149, 10, 137, 64, 6, 103, 110, 106, 113, 113, 98, 122, 42, 152, 3, 87, 199, 242, 37, 117, 30, 66, 93, 18, 178, 186, 36, 0, 127, 205, 212, 97, 80, 0, 0, 0>> + expected_request = + <<0, 0, 0, 0, 0, 0, 0, 1, 0, 23, 99, 111, 109, 112, 114, 101, 115, 115, + 105, 111, 110, 95, 99, 108, 105, 101, 110, 116, 95, 116, 101, 115, 116, + 0, 1, 0, 0, 0, 10, 0, 0, 0, 1, 0, 16, 99, 111, 109, 112, 114, 101, 115, + 115, 101, 100, 95, 116, 111, 112, 105, 99, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, + 0, 86, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 74, 79, 44, 46, 209, 0, 1, 255, + 255, 255, 255, 0, 0, 0, 60, 31, 139, 8, 0, 0, 0, 0, 0, 0, 3, 99, 96, + 128, 3, 153, 135, 115, 4, 255, 131, 89, 172, 217, 169, 149, 10, 137, 64, + 6, 103, 110, 106, 113, 113, 98, 122, 42, 152, 3, 87, 199, 242, 37, 117, + 30, 66, 93, 18, 178, 186, 36, 0, 127, 205, 212, 97, 80, 0, 0, 0>> client_id = "compression_client_test" topic = "compressed_topic" + messages = [ %KafkaEx.Protocol.Produce.Message{key: "key a", value: "message a"}, %KafkaEx.Protocol.Produce.Message{key: "key b", value: "message b"} @@ -46,7 +87,9 @@ defmodule KafkaEx.Protocol.Produce.Test do messages: messages } - iolist_request = KafkaEx.Protocol.Produce.create_request(1, client_id, produce) + iolist_request = + KafkaEx.Protocol.Produce.create_request(1, client_id, produce) + request = :erlang.iolist_to_binary(iolist_request) # The exact binary contents of the message can change as zlib changes, @@ -62,20 +105,21 @@ defmodule KafkaEx.Protocol.Produce.Test do # should be the same. post_crc_header_size = 10 - << pre_crc_header :: binary-size(pre_crc_header_size), - _crc :: binary-size(crc_size), - post_crc_header :: binary-size(post_crc_header_size), - compressed_message_set :: binary >> = request + <> = request - << expect_pre_crc_header :: binary-size(pre_crc_header_size), - _expect_crc :: binary-size(crc_size), - expect_post_crc_header :: binary-size(post_crc_header_size), - expect_compressed_message_set :: binary >> = expected_request + <> = expected_request assert pre_crc_header == expect_pre_crc_header assert post_crc_header == expect_post_crc_header decompressed_message_set = :zlib.gunzip(compressed_message_set) + expect_decompressed_message_set = :zlib.gunzip(expect_compressed_message_set) @@ -83,7 +127,16 @@ defmodule KafkaEx.Protocol.Produce.Test do end test "create_request correctly encodes messages with snappy" do - expected_request = <<0, 0, 0, 0, 0, 0, 0, 1, 0, 23, 99, 111, 109, 112, 114, 101, 115, 115, 105, 111, 110, 95, 99, 108, 105, 101, 110, 116, 95, 116, 101, 115, 116, 0, 1, 0, 0, 0, 10, 0, 0, 0, 1, 0, 16, 99, 111, 109, 112, 114, 101, 115, 115, 101, 100, 95, 116, 111, 112, 105, 99, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 86, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 74, 67, 64, 33, 66, 0, 2, 255, 255, 255, 255, 0, 0, 0, 60, 80, 0, 0, 25, 1, 16, 28, 225, 156, 17, 255, 5, 15, 64, 5, 107, 101, 121, 32, 97, 0, 0, 0, 9, 109, 101, 115, 115, 97, 103, 101, 5, 13, 17, 1, 16, 28, 4, 244, 101, 158, 5, 13, 5, 40, 52, 98, 0, 0, 0, 9, 109, 101, 115, 115, 97, 103, 101, 32, 98>> + expected_request = + <<0, 0, 0, 0, 0, 0, 0, 1, 0, 23, 99, 111, 109, 112, 114, 101, 115, 115, + 105, 111, 110, 95, 99, 108, 105, 101, 110, 116, 95, 116, 101, 115, 116, + 0, 1, 0, 0, 0, 10, 0, 0, 0, 1, 0, 16, 99, 111, 109, 112, 114, 101, 115, + 115, 101, 100, 95, 116, 111, 112, 105, 99, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, + 0, 86, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 74, 67, 64, 33, 66, 0, 2, 255, + 255, 255, 255, 0, 0, 0, 60, 80, 0, 0, 25, 1, 16, 28, 225, 156, 17, 255, + 5, 15, 64, 5, 107, 101, 121, 32, 97, 0, 0, 0, 9, 109, 101, 115, 115, 97, + 103, 101, 5, 13, 17, 1, 16, 28, 4, 244, 101, 158, 5, 13, 5, 40, 52, 98, + 0, 0, 0, 9, 109, 101, 115, 115, 97, 103, 101, 32, 98>> produce = %KafkaEx.Protocol.Produce.Request{ topic: "compressed_topic", @@ -97,30 +150,55 @@ defmodule KafkaEx.Protocol.Produce.Test do ] } - request = KafkaEx.Protocol.Produce.create_request(1, - "compression_client_test", - produce) + request = + KafkaEx.Protocol.Produce.create_request( + 1, + "compression_client_test", + produce + ) assert expected_request == :erlang.iolist_to_binary(request) end test "parse_response correctly parses a valid response with single topic and partition" do - response = << 0 :: 32, 1 :: 32, 3 :: 16, "bar" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 10 :: 64 >> - expected_response = [%KafkaEx.Protocol.Produce.Response{partitions: [%{error_code: :no_error, offset: 10, partition: 0}], topic: "bar"}] - assert expected_response == KafkaEx.Protocol.Produce.parse_response(response) + response = + <<0::32, 1::32, 3::16, "bar"::binary, 1::32, 0::32, 0::16, 10::64>> + + expected_response = [ + %KafkaEx.Protocol.Produce.Response{ + partitions: [%{error_code: :no_error, offset: 10, partition: 0}], + topic: "bar" + } + ] + + assert expected_response == + KafkaEx.Protocol.Produce.parse_response(response) end test "parse_response correctly parses a valid response with multiple topics and partitions" do - response = << 0 :: 32, 2 :: 32, 3 :: 16, "bar" :: binary, 2 :: 32, 0 :: 32, 0 :: 16, 10 :: 64, 1 :: 32, 0 :: 16, 20 :: 64, 3 :: 16, "baz" :: binary, 2 :: 32, 0 :: 32, 0 :: 16, 30 :: 64, 1 :: 32, 0 :: 16, 40 :: 64 >> - expected_response = [%KafkaEx.Protocol.Produce.Response{partitions: [ - %{error_code: :no_error, offset: 20, partition: 1}, - %{error_code: :no_error, offset: 10, partition: 0} - ], topic: "bar"}, - %KafkaEx.Protocol.Produce.Response{partitions: [ - %{error_code: :no_error, offset: 40, partition: 1}, - %{error_code: :no_error, offset: 30, partition: 0} - ], topic: "baz"}] - - assert expected_response == KafkaEx.Protocol.Produce.parse_response(response) + response = + <<0::32, 2::32, 3::16, "bar"::binary, 2::32, 0::32, 0::16, 10::64, 1::32, + 0::16, 20::64, 3::16, "baz"::binary, 2::32, 0::32, 0::16, 30::64, 1::32, + 0::16, 40::64>> + + expected_response = [ + %KafkaEx.Protocol.Produce.Response{ + partitions: [ + %{error_code: :no_error, offset: 20, partition: 1}, + %{error_code: :no_error, offset: 10, partition: 0} + ], + topic: "bar" + }, + %KafkaEx.Protocol.Produce.Response{ + partitions: [ + %{error_code: :no_error, offset: 40, partition: 1}, + %{error_code: :no_error, offset: 30, partition: 0} + ], + topic: "baz" + } + ] + + assert expected_response == + KafkaEx.Protocol.Produce.parse_response(response) end end diff --git a/test/protocol/sync_group_test.exs b/test/protocol/sync_group_test.exs index aadccddf..0b24f021 100644 --- a/test/protocol/sync_group_test.exs +++ b/test/protocol/sync_group_test.exs @@ -3,61 +3,122 @@ defmodule KafkaEx.Protocol.SyncGroup.Test do test "create_request creates a valid sync group request" do first_assignments = << - 0 :: 16, # Version - 1 :: 32, # PartitionAssignment array size - 6 :: 16, "topic1", # Topic - 3 :: 32, 1 :: 32, 3 :: 32, 5 :: 32, # Partition array - 0 :: 32, # UserData - >> + # Version + 0::16, + # PartitionAssignment array size + 1::32, + # Topic + 6::16, + "topic1", + # Partition array + 3::32, + 1::32, + 3::32, + 5::32, + # UserData + 0::32 + >> + second_assignments = << - 0 :: 16, # Version - 1 :: 32, # PartitionAssignment array size - 6 :: 16, "topic1", - 3 :: 32, 2 :: 32, 4 :: 32, 6 :: 32, # Partition array - 0 :: 32, # UserData - >> + # Version + 0::16, + # PartitionAssignment array size + 1::32, + 6::16, + "topic1", + # Partition array + 3::32, + 2::32, + 4::32, + 6::32, + # UserData + 0::32 + >> + good_request = << - 14 :: 16, 0 :: 16, 42 :: 32, 9 :: 16, "client_id" :: binary, # Preamble - 5 :: 16, "group" :: binary, # GroupId - 1 :: 32, # Generation ID - 10 :: 16, "member_one" :: binary, # MemberId - 2 :: 32, # GroupAssignment array size - 10 :: 16, "member_one" :: binary, # First member ID - byte_size(first_assignments) :: 32, first_assignments :: binary, - 10 :: 16, "member_two" :: binary, # Second member ID - byte_size(second_assignments) :: 32, second_assignments :: binary - >> + # Preamble + 14::16, + 0::16, + 42::32, + 9::16, + "client_id"::binary, + # GroupId + 5::16, + "group"::binary, + # Generation ID + 1::32, + # MemberId + 10::16, + "member_one"::binary, + # GroupAssignment array size + 2::32, + # First member ID + 10::16, + "member_one"::binary, + byte_size(first_assignments)::32, + first_assignments::binary, + # Second member ID + 10::16, + "member_two"::binary, + byte_size(second_assignments)::32, + second_assignments::binary + >> sync_request = %KafkaEx.Protocol.SyncGroup.Request{ group_name: "group", member_id: "member_one", generation_id: 1, - assignments: [{"member_one", [{"topic1", [1, 3, 5]}]}, {"member_two", [{"topic1", [2, 4, 6]}]}], + assignments: [ + {"member_one", [{"topic1", [1, 3, 5]}]}, + {"member_two", [{"topic1", [2, 4, 6]}]} + ] } - request = KafkaEx.Protocol.SyncGroup.create_request(42, "client_id", sync_request) + request = + KafkaEx.Protocol.SyncGroup.create_request(42, "client_id", sync_request) + assert request == good_request end test "parse success response correctly" do - member_assignment = << 0 :: 16, 1 :: 32, 6 :: 16, "topic1", 3 :: 32, 1 :: 32, 3 :: 32, 5 :: 32 >> + member_assignment = + <<0::16, 1::32, 6::16, "topic1", 3::32, 1::32, 3::32, 5::32>> + response = << - 42 :: 32, # CorrelationId - 0 :: 16, # ErrorCode - byte_size(member_assignment) :: 32, member_assignment :: binary # MemberAssignment - >> - expected_response = %KafkaEx.Protocol.SyncGroup.Response{error_code: :no_error, - assignments: [{"topic1", [5, 3, 1]}]} - assert KafkaEx.Protocol.SyncGroup.parse_response(response) == expected_response + # CorrelationId + 42::32, + # ErrorCode + 0::16, + # MemberAssignment + byte_size(member_assignment)::32, + member_assignment::binary + >> + + expected_response = %KafkaEx.Protocol.SyncGroup.Response{ + error_code: :no_error, + assignments: [{"topic1", [5, 3, 1]}] + } + + assert KafkaEx.Protocol.SyncGroup.parse_response(response) == + expected_response end test "parse empty assignments correctly" do response = << - 42 :: 32, # CorrelationId - 0 :: 16, # ErrorCode - 0 :: 32, # MemberAssignment (empty) - >> - expected_response = %KafkaEx.Protocol.SyncGroup.Response{error_code: :no_error, assignments: []} - assert KafkaEx.Protocol.SyncGroup.parse_response(response) == expected_response + # CorrelationId + 42::32, + # ErrorCode + 0::16, + # MemberAssignment (empty) + 0::32 + >> + + expected_response = %KafkaEx.Protocol.SyncGroup.Response{ + error_code: :no_error, + assignments: [] + } + + assert KafkaEx.Protocol.SyncGroup.parse_response(response) == + expected_response end end diff --git a/test/socket_test.exs b/test/socket_test.exs index 6a1229cc..9a71fc20 100644 --- a/test/socket_test.exs +++ b/test/socket_test.exs @@ -3,7 +3,14 @@ defmodule KafkaEx.Socket.Test do defmodule Server do def start(port) do - {:ok, listen_socket} = :gen_tcp.listen(port, [:binary, {:active, false}, {:reuseaddr, true}, {:packet, 0}]) + {:ok, listen_socket} = + :gen_tcp.listen(port, [ + :binary, + {:active, false}, + {:reuseaddr, true}, + {:packet, 0} + ]) + spawn_link(fn -> listen(listen_socket) end) end @@ -17,6 +24,7 @@ defmodule KafkaEx.Socket.Test do case :gen_tcp.recv(conn, 0) do {:ok, data} -> :ok = :gen_tcp.send(conn, data) + {:error, :closed} -> :ok end @@ -25,18 +33,30 @@ defmodule KafkaEx.Socket.Test do defmodule SSLServer do def start(port) do - {:ok, listen_socket} = :ssl.listen(port, [:binary, {:active, false}, {:reuseaddr, true}, {:packet, 0}, {:certfile, 'test/fixtures/server.crt'}, {:keyfile, 'test/fixtures/server.key'}]) + {:ok, listen_socket} = + :ssl.listen(port, [ + :binary, + {:active, false}, + {:reuseaddr, true}, + {:packet, 0}, + {:certfile, 'test/fixtures/server.crt'}, + {:keyfile, 'test/fixtures/server.key'} + ]) + spawn_link(fn -> listen(listen_socket) end) end defp listen(socket) do - case :ssl.transport_accept(socket) do + case :ssl.transport_accept(socket) do {:ok, conn} -> :ok = :ssl.ssl_accept(conn) pid = spawn_link(fn -> recv(conn) end) :ssl.controlling_process(socket, pid) - _ -> :ok + + _ -> + :ok end + listen(socket) end @@ -44,6 +64,7 @@ defmodule KafkaEx.Socket.Test do case :ssl.recv(conn, 0) do {:ok, data} -> :ok = :ssl.send(conn, data) + {:error, :closed} -> :ok end @@ -51,27 +72,48 @@ defmodule KafkaEx.Socket.Test do end setup_all do - :ssl.start + :ssl.start() SSLServer.start(3030) Server.start(3040) {:ok, [ssl_port: 3030, port: 3040]} end test "create a non SSL socket", context do - {:ok, socket} = KafkaEx.Socket.create('localhost', context[:port], [:binary, {:packet, 0}], false) + {:ok, socket} = + KafkaEx.Socket.create( + 'localhost', + context[:port], + [:binary, {:packet, 0}], + false + ) + assert socket.ssl == false KafkaEx.Socket.close(socket) end test "send and receive using a non SSL socket", context do - {:ok, socket} = KafkaEx.Socket.create('localhost', context[:port], [:binary, {:packet, 0}, {:active, false}], false) + {:ok, socket} = + KafkaEx.Socket.create( + 'localhost', + context[:port], + [:binary, {:packet, 0}, {:active, false}], + false + ) + KafkaEx.Socket.send(socket, 'ping') - assert {:ok, "ping" } == KafkaEx.Socket.recv(socket, 0) + assert {:ok, "ping"} == KafkaEx.Socket.recv(socket, 0) KafkaEx.Socket.close(socket) end test "retrieve info from a non SSL socket", context do - {:ok, socket} = KafkaEx.Socket.create('localhost', context[:port], [:binary, {:packet, 0}, {:active, false}], false) + {:ok, socket} = + KafkaEx.Socket.create( + 'localhost', + context[:port], + [:binary, {:packet, 0}, {:active, false}], + false + ) + info = KafkaEx.Socket.info(socket) assert info[:name] == 'tcp_inet' KafkaEx.Socket.close(socket) @@ -79,20 +121,41 @@ defmodule KafkaEx.Socket.Test do end test "create a SSL socket", context do - {:ok, socket} = KafkaEx.Socket.create('localhost', context[:ssl_port], [:binary, {:packet, 0}], true) + {:ok, socket} = + KafkaEx.Socket.create( + 'localhost', + context[:ssl_port], + [:binary, {:packet, 0}], + true + ) + assert socket.ssl == true KafkaEx.Socket.close(socket) end test "send and receive using a SSL socket", context do - {:ok, socket} = KafkaEx.Socket.create('localhost', context[:ssl_port], [:binary, {:packet, 0}, {:active, false}], true) + {:ok, socket} = + KafkaEx.Socket.create( + 'localhost', + context[:ssl_port], + [:binary, {:packet, 0}, {:active, false}], + true + ) + KafkaEx.Socket.send(socket, 'ping') - assert {:ok, "ping" } == KafkaEx.Socket.recv(socket, 0) + assert {:ok, "ping"} == KafkaEx.Socket.recv(socket, 0) KafkaEx.Socket.close(socket) end test "retrieve info from a SSL socket", context do - {:ok, socket} = KafkaEx.Socket.create('localhost', context[:ssl_port], [:binary, {:packet, 0}, {:active, false}], true) + {:ok, socket} = + KafkaEx.Socket.create( + 'localhost', + context[:ssl_port], + [:binary, {:packet, 0}, {:active, false}], + true + ) + info = KafkaEx.Socket.info(socket) assert info[:name] == 'tcp_inet' KafkaEx.Socket.close(socket) diff --git a/test/test_helper.exs b/test/test_helper.exs index 7498ab0d..11aeaf44 100644 --- a/test/test_helper.exs +++ b/test/test_helper.exs @@ -1,12 +1,20 @@ ExUnit.start() -ExUnit.configure timeout: 120*1000, exclude: [integration: true, consumer_group: true, server_0_p_10_p_1: true, server_0_p_9_p_0: true, server_0_p_8_p_0: true] +ExUnit.configure( + timeout: 120 * 1000, + exclude: [ + integration: true, + consumer_group: true, + server_0_p_10_p_1: true, + server_0_p_9_p_0: true, + server_0_p_8_p_0: true + ] +) defmodule TestHelper do - def generate_random_string(string_length \\ 20) do 1..string_length - |> Enum.map(fn _ -> round(:rand.uniform * 25 + 65) end) + |> Enum.map(fn _ -> round(:rand.uniform() * 25 + 65) end) |> to_string end @@ -20,7 +28,7 @@ defmodule TestHelper do # Wait for condn to return false or nil; passes through to wait_for_value # returns :ok on success def wait_for(condn, dwell \\ 500, max_tries \\ 200) do - wait_for_value(fn() -> :ok end, fn(:ok) -> condn.() end, dwell, max_tries) + wait_for_value(fn -> :ok end, fn :ok -> condn.() end, dwell, max_tries) end # execute value_getter, which should return a list, and accumulate @@ -40,43 +48,58 @@ defmodule TestHelper do end def utc_time do - {x, {a,b,c}} = :calendar.local_time |> :calendar.local_time_to_universal_time_dst |> hd - {x, {a,b,c + 60}} + {x, {a, b, c}} = + :calendar.local_time() + |> :calendar.local_time_to_universal_time_dst() + |> hd + + {x, {a, b, c + 60}} end def latest_offset_number(topic, partition_id, worker \\ :kafka_ex) do - offset = KafkaEx.latest_offset(topic, partition_id, worker) + offset = + KafkaEx.latest_offset(topic, partition_id, worker) |> first_partition_offset offset || 0 end - def latest_consumer_offset_number(topic, - partition, - consumer_group, - worker \\ :kafka_ex) do - request = %KafkaEx.Protocol.OffsetFetch.Request{topic: topic, - partition: partition, - consumer_group: consumer_group} + def latest_consumer_offset_number( + topic, + partition, + consumer_group, + worker \\ :kafka_ex + ) do + request = %KafkaEx.Protocol.OffsetFetch.Request{ + topic: topic, + partition: partition, + consumer_group: consumer_group + } + KafkaEx.offset_fetch(worker, request) - |> KafkaEx.Protocol.OffsetFetch.Response.last_offset + |> KafkaEx.Protocol.OffsetFetch.Response.last_offset() end defp first_partition_offset(:topic_not_found) do nil end + defp first_partition_offset(response) do [%KafkaEx.Protocol.Offset.Response{partition_offsets: partition_offsets}] = response + first_partition = hd(partition_offsets) first_partition.offset |> hd end - defp wait_for_value(_value_getter, _condn, _dwell, max_tries, n) when n >= max_tries do + defp wait_for_value(_value_getter, _condn, _dwell, max_tries, n) + when n >= max_tries do raise "too many tries waiting for condition" end + defp wait_for_value(value_getter, condn, dwell, max_tries, n) do value = value_getter.() + if condn.(value) do value else @@ -86,11 +109,14 @@ defmodule TestHelper do end defp wait_for_accum(_value_getter, acc, min_length, _dwell, _max_tries) - when length(acc) >= min_length do + when length(acc) >= min_length do acc end + defp wait_for_accum(value_getter, acc, min_length, dwell, max_tries) do - value = wait_for_value(value_getter, fn(v) -> length(v) > 0 end, dwell, max_tries) + value = + wait_for_value(value_getter, fn v -> length(v) > 0 end, dwell, max_tries) + wait_for_accum(value_getter, acc ++ value, min_length, dwell, max_tries) end end