From 7b78fc95f730b635f37bae6faca3f6b111ea44d8 Mon Sep 17 00:00:00 2001 From: Zaiming Shi Date: Tue, 15 May 2018 09:26:07 +0200 Subject: [PATCH] Update kafka_protocol to get ready for kafka 1.1 --- .travis.yml | 9 +- Makefile | 27 +- README.md | 5 + changelog.md | 13 + docker/Dockerfile | 18 +- docker/docker-compose-basic.yml | 2 + docker/docker-compose-kafka-1.yml | 10 +- docker/docker-compose-kafka-2.yml | 24 +- docker/docker-entrypoint.sh | 17 +- docker/jaas-plain-scram.conf | 9 + docker/{jaas.conf => jaas-plain.conf} | 4 +- elvis.config | 4 +- erlang.mk | 8 - include/brod.hrl | 8 +- include/brod_int.hrl | 4 +- rebar.config | 2 +- rebar.config.script | 29 +- scripts/setup-test-env.sh | 30 +- scripts/vsn-check.sh | 2 +- src/brod.app.src | 4 +- src/brod.erl | 278 +++++----- src/brod_auth_backend.erl | 38 -- src/brod_cg_commits.erl | 12 +- src/brod_cli.erl | 206 +++---- src/brod_cli_pipe.erl | 38 +- src/brod_client.erl | 542 ++++++++----------- src/brod_consumer.erl | 224 ++++---- src/brod_consumers_sup.erl | 7 +- src/brod_group_coordinator.erl | 255 ++++----- src/brod_group_member.erl | 2 +- src/brod_group_subscriber.erl | 9 +- src/brod_kafka_apis.erl | 197 +++---- src/brod_kafka_request.erl | 142 ++--- src/brod_kafka_requests.erl | 114 ---- src/brod_producer.erl | 184 +++---- src/brod_producer_buffer.erl | 139 ++--- src/brod_producers_sup.erl | 5 +- src/brod_sock.erl | 678 ------------------------ src/brod_sup.erl | 3 +- src/brod_topic_subscriber.erl | 7 +- src/brod_utils.erl | 676 +++++++++++------------ test/brod_cg_commits_SUITE.erl | 6 +- test/brod_cli_pipe_tests.erl | 2 +- test/brod_cli_tests.erl | 2 +- test/brod_client_SUITE.erl | 119 ++--- test/brod_compression_SUITE.erl | 8 +- test/brod_consumer_SUITE.erl | 40 +- test/brod_demo_cg_collector.erl | 3 +- test/brod_demo_group_subscriber_koc.erl | 7 +- test/brod_demo_group_subscriber_loc.erl | 6 +- test/brod_demo_topic_subscriber.erl | 3 +- test/brod_group_subscriber_SUITE.erl | 4 +- test/brod_kafka_apis_tests.erl | 84 ++- test/brod_producer_SUITE.erl | 8 +- test/brod_producer_buffer_SUITE.erl | 85 ++- test/brod_producer_stub_SUITE.erl | 268 +++++----- test/brod_sock_SUITE.erl | 109 ---- test/brod_topic_subscriber_SUITE.erl | 8 +- 58 files changed, 1678 insertions(+), 3069 deletions(-) create mode 100644 docker/jaas-plain-scram.conf rename docker/{jaas.conf => jaas-plain.conf} (59%) delete mode 100644 src/brod_auth_backend.erl delete mode 100644 src/brod_kafka_requests.erl delete mode 100644 src/brod_sock.erl delete mode 100644 test/brod_sock_SUITE.erl diff --git a/.travis.yml b/.travis.yml index a159495b..e8642562 100644 --- a/.travis.yml +++ b/.travis.yml @@ -11,21 +11,18 @@ before_install: - curl -L https://github.com/docker/compose/releases/download/1.6.0/docker-compose-`uname -s`-`uname -m` > docker-compose - chmod +x docker-compose - sudo mv docker-compose /usr/local/bin/ - - git clone https://github.com/erlang/rebar3.git; cd rebar3; ./bootstrap; sudo cp rebar3 /usr/bin; cd .. - git clone https://github.com/inaka/elvis.git; cd elvis; rebar3 escriptize; sudo cp _build/default/bin/elvis /usr/bin; cd .. notifications: email: false otp_release: - - 20.0 + - 21.0 + - 20.3 - 19.3 - - 18.2.1 - - 17.5 script: - set -e - - make vsn-check - elvis rock - echo "verifying rebar compilation" @@ -43,5 +40,5 @@ script: - make xref - make test-env - make t - - if [ "$(erl -noshell -eval 'io:format(erlang:system_info(otp_release)), halt(0)')" -gt 18 ]; then make dialyze; fi + - if [ "$(erl -noshell -eval 'io:format(erlang:system_info(otp_release)), halt(0)')" -eq 21 ]; then make dialyze; fi diff --git a/Makefile b/Makefile index 64c6b511..70fb361f 100644 --- a/Makefile +++ b/Makefile @@ -1,22 +1,26 @@ PROJECT = brod PROJECT_DESCRIPTION = Kafka client library in Erlang -PROJECT_VERSION = 3.5.2 +PROJECT_VERSION = 3.6.0 DEPS = supervisor3 kafka_protocol TEST_DEPS = docopt jsone meck proper REL_DEPS = docopt jsone -ERLC_OPTS = -Werror +warn_unused_vars +warn_shadow_vars +warn_unused_import +warn_obsolete_guard +debug_info -TEST_ERLC_OPTS = -Werror +warn_unused_vars +warn_shadow_vars +warn_unused_import +warn_obsolete_guard +debug_info +COMMON_ERLC_OPTS = -Werror +warn_unused_vars +warn_shadow_vars +warn_unused_import +warn_obsolete_guard +debug_info -Dbuild_brod_cli +ERLC_OPTS = $(COMMON_ERLC_OPTS) +TEST_ERLC_OPTS = $(COMMON_ERLC_OPTS) + dep_supervisor3_commit = 1.1.5 -dep_kafka_protocol_commit = 1.1.2 +dep_kafka_protocol_commit = 2.0.0 +dep_kafka_protocol = git https://github.com/klarna/kafka_protocol.git $(dep_kafka_protocol_commit) dep_docopt = git https://github.com/zmstone/docopt-erl.git 0.1.3 ERTS_VSN = $(shell erl -noshell -eval 'io:put_chars(erlang:system_info(version)), halt()') ESCRIPT_FILE = scripts/brod_cli ESCRIPT_EMU_ARGS = -sname brod_cli +EDOC_OPTS = preprocess, {macros, [{build_brod_cli, true}]} COVER = true EUNIT_OPTS = verbose @@ -25,19 +29,6 @@ CT_OPTS = -ct_use_short_names true ERL_LIBS := $(ERL_LIBS):$(CURDIR) -ifeq ($(MAKECMDGOALS),) - export BROD_CLI=true -else ifneq ($(filter rel,$(MAKECMDGOALS)),) - export BROD_CLI=true -else ifneq ($(filter escript,$(MAKECMDGOALS)),) - export BROD_CLI=true -endif - -ifeq ($(BROD_CLI),true) - ERLC_OPTS += -DBROD_CLI - TEST_ERLC_OPTS += -DBROD_CLI -endif - ## Make app the default target ## To avoid building a release when brod is used as a erlang.mk project's dependency app:: @@ -51,7 +42,7 @@ rel:: escript @tar -pczf _rel/brod.tar.gz -C _rel brod test-env: - ./scripts/setup-test-env.sh + ./scripts/setup-test-env.sh 1.1 t: eunit ct ./scripts/cover-summary.escript eunit.coverdata ct.coverdata diff --git a/README.md b/README.md index 97e1e344..2578f92b 100644 --- a/README.md +++ b/README.md @@ -505,3 +505,8 @@ NOTE: This feature is designed for force overwriting commits, not for regular us ./scripts/brod commits -b localhost:9092 -i the-group-id -t topic-name -o "0:10000" --protocol range ``` +## TODOs + +* HTML tagged EDoc +* Support scram-sasl in brod-cli + diff --git a/changelog.md b/changelog.md index fb1928a0..fa7edd22 100644 --- a/changelog.md +++ b/changelog.md @@ -89,3 +89,16 @@ * 3.5.2 * Fix issue #263: Kafka 0.11 may send empty batch in fetch response when messages are deleted in compacted topics. +* 3.6.0 + * Moved 3 modules to kafka_protocol: + - `brod_sock` -> `kpro_connection` + - `brod_auth_backed` -> `kpro_auth_backend` + - `brod_kafka_requests` -> `kpro_sent_reqs` + * `#kafka_message.key` and `#kafka_message.value` are now always `binary()` + (they were of spec `undefined | binary()` prior to this version). + i.e. empty bytes are now decoded as `<<>>` instead of `undefined`. + This may cause dialyzer check failures. + * `brod_client` no longer logs about metadata socket down, it had been confusing rather than being helpful + * There is no more cool-down delay for metadata socket re-establishment + * `brod_group_coordinator` default session timeout changed from 10 seconds to 30, + and heartbeat interval changed from 2 seconds to 5. diff --git a/docker/Dockerfile b/docker/Dockerfile index 36b959d2..d5ace635 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -1,17 +1,20 @@ FROM java:openjdk-8-jre -ENV SCALA_VERSION 2.11 +ARG KAFKA_VERSION -# can also be 0.11.0.2 or 0.10.2.1 -ENV KAFKA_VERSION 1.0.0 -ENV KAFKA_DOWNLOAD_URL="https://apache.org/dist/kafka/${KAFKA_VERSION}/kafka_${SCALA_VERSION}-${KAFKA_VERSION}.tgz" -#ENV KAFKA_VERSION 0.9.0.0 -#ENV KAFKA_DOWNLOAD_URL="https://archive.apache.org/dist/kafka/${KAFKA_VERSION}/kafka_${SCALA_VERSION}-${KAFKA_VERSION}.tgz" +ENV KAFKA_VERSION ${KAFKA_VERSION} +ENV SCALA_VERSION 2.11 RUN apt-get update && \ apt-get install -y zookeeper wget supervisor dnsutils && \ rm -rf /var/lib/apt/lists/* && \ apt-get clean && \ + if [[ "$KAFKA_VERSION" = 0.9* ]]; then \ + DOWNLOAD_URL_PREFIX="https://archive.apache.org/dist/kafka"; \ + else \ + DOWNLOAD_URL_PREFIX="https://apache.org/dist/kafka/"; \ + fi && \ + KAFKA_DOWNLOAD_URL="$DOWNLOAD_URL_PREFIX/${KAFKA_VERSION}/kafka_${SCALA_VERSION}-${KAFKA_VERSION}.tgz" && \ wget -q "${KAFKA_DOWNLOAD_URL}" -O /tmp/kafka_"$SCALA_VERSION"-"$KAFKA_VERSION".tgz && \ tar xfz /tmp/kafka_"$SCALA_VERSION"-"$KAFKA_VERSION".tgz -C /opt && \ rm /tmp/kafka_"$SCALA_VERSION"-"$KAFKA_VERSION".tgz && \ @@ -22,7 +25,8 @@ COPY docker-entrypoint.sh /docker-entrypoint.sh COPY server.properties /etc/kafka/server.properties COPY server.jks /etc/kafka/server.jks COPY truststore.jks /etc/kafka/truststore.jks -COPY jaas.conf /etc/kafka/jaas.conf +COPY jaas-plain.conf /etc/kafka/jaas-plain.conf +COPY jaas-plain-scram.conf /etc/kafka/jaas-plain-scram.conf ENTRYPOINT ["/docker-entrypoint.sh"] diff --git a/docker/docker-compose-basic.yml b/docker/docker-compose-basic.yml index 03448bd1..32221c55 100644 --- a/docker/docker-compose-basic.yml +++ b/docker/docker-compose-basic.yml @@ -4,6 +4,8 @@ services: kafka: build: context: . + args: + KAFKA_VERSION: "${KAFKA_VERSION}" zookeeper: image: docker_kafka ports: diff --git a/docker/docker-compose-kafka-1.yml b/docker/docker-compose-kafka-1.yml index eaf94050..56c2daa8 100644 --- a/docker/docker-compose-kafka-1.yml +++ b/docker/docker-compose-kafka-1.yml @@ -10,11 +10,13 @@ services: container_name: kafka_1 ports: - "9092:9092" - - "9192:9192" - - "9292:9292" + - "9093:9093" + - "9094:9094" + - "9095:9095" environment: BROKER_ID: 0 PLAINTEXT_PORT: 9092 - SSL_PORT: 9192 - SASL_SSL_PORT: 9292 + SSL_PORT: 9093 + SASL_SSL_PORT: 9094 + SASL_PLAINTEXT_PORT: 9095 diff --git a/docker/docker-compose-kafka-2.yml b/docker/docker-compose-kafka-2.yml index ca943f5c..69e955af 100644 --- a/docker/docker-compose-kafka-2.yml +++ b/docker/docker-compose-kafka-2.yml @@ -10,23 +10,27 @@ services: container_name: kafka_1 ports: - "9092:9092" - - "9192:9192" - - "9292:9292" + - "9093:9093" + - "9094:9094" + - "9095:9095" environment: BROKER_ID: 0 PLAINTEXT_PORT: 9092 - SSL_PORT: 9192 - SASL_SSL_PORT: 9292 + SSL_PORT: 9093 + SASL_SSL_PORT: 9094 + SASL_PLAINTEXT_PORT: 9095 kafka_2: image: docker_kafka container_name: kafka_2 ports: - - "9093:9093" - - "9193:9193" - - "9293:9293" + - "9192:9092" + - "9193:9093" + - "9194:9094" + - "9195:9095" environment: BROKER_ID: 1 - PLAINTEXT_PORT: 9093 - SSL_PORT: 9193 - SASL_SSL_PORT: 9293 + PLAINTEXT_PORT: 9092 + SSL_PORT: 9093 + SASL_SSL_PORT: 9094 + SASL_PLAINTEXT_PORT: 9095 diff --git a/docker/docker-entrypoint.sh b/docker/docker-entrypoint.sh index 06650a40..c41e8ff1 100755 --- a/docker/docker-entrypoint.sh +++ b/docker/docker-entrypoint.sh @@ -7,8 +7,7 @@ fi ## run zookeeper if [ "$2" = "zookeeper" ]; then - /opt/kafka/bin/zookeeper-server-start.sh /opt/kafka/config/zookeeper.properties - exit $? + exec /opt/kafka/bin/zookeeper-server-start.sh /opt/kafka/config/zookeeper.properties fi if [ "$2" != "kafka" ]; then @@ -32,19 +31,23 @@ if [[ "$KAFKA_VERSION" = 0.9* ]]; then sed -r -i "s/^(advertised.listeners)=(.*)/\1=PLAINTEXT:\/\/$ipaddress:$PLAINTEXT_PORT,SSL:\/\/$ipaddress:$SSL_PORT/g" $prop_file sed -r -i "s/^(listeners)=(.*)/\1=PLAINTEXT:\/\/:$PLAINTEXT_PORT,SSL:\/\/:$SSL_PORT/g" $prop_file else - sed -r -i "s/^(advertised.listeners)=(.*)/\1=PLAINTEXT:\/\/$ipaddress:$PLAINTEXT_PORT,SSL:\/\/$ipaddress:$SSL_PORT,SASL_SSL:\/\/$ipaddress:$SASL_SSL_PORT/g" $prop_file - sed -r -i "s/^(listeners)=(.*)/\1=PLAINTEXT:\/\/:$PLAINTEXT_PORT,SSL:\/\/:$SSL_PORT,SASL_SSL:\/\/:$SASL_SSL_PORT/g" $prop_file + sed -r -i "s/^(advertised.listeners)=(.*)/\1=PLAINTEXT:\/\/$ipaddress:$PLAINTEXT_PORT,SSL:\/\/$ipaddress:$SSL_PORT,SASL_SSL:\/\/$ipaddress:$SASL_SSL_PORT,SASL_PLAINTEXT:\/\/$ipaddress:$SASL_PLAINTEXT_PORT/g" $prop_file + sed -r -i "s/^(listeners)=(.*)/\1=PLAINTEXT:\/\/:$PLAINTEXT_PORT,SSL:\/\/:$SSL_PORT,SASL_SSL:\/\/:$SASL_SSL_PORT,SASL_PLAINTEXT:\/\/:$SASL_PLAINTEXT_PORT/g" $prop_file echo "sasl.enabled.mechanisms=PLAIN" >> $prop_file fi -echo "sasl.enabled.mechanisms=PLAIN" >> $prop_file +echo "sasl.enabled.mechanisms=PLAIN,SCRAM-SHA-256,SCRAM-SHA-512" >> $prop_file echo "offsets.topic.replication.factor=1" >> $prop_file +echo "transaction.state.log.min.isr=1" >> $prop_file +echo "transaction.state.log.replication.factor=1" >> $prop_file if [[ "$KAFKA_VERSION" = 0.9* ]]; then JAAS_CONF="" +elif [[ "$KAFKA_VERSION" = 0.10* ]]; then + JAAS_CONF="-Djava.security.auth.login.config=/etc/kafka/jaas-plain.conf" else - JAAS_CONF="-Djava.security.auth.login.config=/etc/kafka/jaas.conf" + JAAS_CONF="-Djava.security.auth.login.config=/etc/kafka/jaas-plain-scram.conf" fi - +#-Djavax.net.debug=all KAFKA_HEAP_OPTS="-Xmx1G -Xms1G $JAAS_CONF" /opt/kafka/bin/kafka-server-start.sh $prop_file diff --git a/docker/jaas-plain-scram.conf b/docker/jaas-plain-scram.conf new file mode 100644 index 00000000..06b8cdca --- /dev/null +++ b/docker/jaas-plain-scram.conf @@ -0,0 +1,9 @@ +KafkaServer { + org.apache.kafka.common.security.plain.PlainLoginModule required + user_admin="nimda" + user_alice="ecila"; + + org.apache.kafka.common.security.scram.ScramLoginModule required + username="admin" + password="nimda"; +}; diff --git a/docker/jaas.conf b/docker/jaas-plain.conf similarity index 59% rename from docker/jaas.conf rename to docker/jaas-plain.conf index 8d3acc6e..521ccd6a 100644 --- a/docker/jaas.conf +++ b/docker/jaas-plain.conf @@ -1,5 +1,5 @@ KafkaServer { org.apache.kafka.common.security.plain.PlainLoginModule required - user_admin="admin-secret" - user_alice="alice-secret"; + user_admin="nimda" + user_alice="ecila"; }; diff --git a/elvis.config b/elvis.config index adeef3fc..fae92f85 100644 --- a/elvis.config +++ b/elvis.config @@ -30,6 +30,8 @@ #{ level => 3, ignore => [ brod_group_coordinator , brod_utils + , brod_cli + , brod_cli_pipe ] }} , {elvis_style, god_modules, @@ -38,7 +40,7 @@ }} , {elvis_style, no_nested_try_catch} , {elvis_style, invalid_dynamic_call, - #{ignore => [brod_sock, brod_group_coordinator] + #{ignore => [brod_group_coordinator] }} , {elvis_style, used_ignored_variable} , {elvis_style, no_behavior_info} diff --git a/erlang.mk b/erlang.mk index 08ade460..32e8ecdd 100644 --- a/erlang.mk +++ b/erlang.mk @@ -2171,14 +2171,6 @@ pkg_kafka_fetch = git pkg_kafka_repo = https://github.com/wooga/kafka-erlang pkg_kafka_commit = master -PACKAGES += kafka_protocol -pkg_kafka_protocol_name = kafka_protocol -pkg_kafka_protocol_description = Kafka protocol Erlang library -pkg_kafka_protocol_homepage = https://github.com/klarna/kafka_protocol -pkg_kafka_protocol_fetch = git -pkg_kafka_protocol_repo = https://github.com/klarna/kafka_protocol.git -pkg_kafka_protocol_commit = master - PACKAGES += kai pkg_kai_name = kai pkg_kai_description = DHT storage by Takeshi Inoue diff --git a/include/brod.hrl b/include/brod.hrl index 4816643b..b53d86a0 100644 --- a/include/brod.hrl +++ b/include/brod.hrl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2014-2017, Klarna AB +%%% Copyright (c) 2014-2018, Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -28,15 +28,15 @@ , partition :: brod:partition() , high_wm_offset :: integer() %% max offset of the partition , messages :: [brod:message()] %% exposed to brod user - | kpro:incomplete_message() %% this union member - %% is internal only + | kpro:incomplete_batch() %% this union member + %% is internal only }). -record(kafka_fetch_error, { topic :: brod:topic() , partition :: brod:partition() , error_code :: brod:error_code() - , error_desc :: binary() + , error_desc = "" }). -record(brod_call_ref, { caller :: pid() diff --git a/include/brod_int.hrl b/include/brod_int.hrl index d73d4a0b..81302906 100644 --- a/include/brod_int.hrl +++ b/include/brod_int.hrl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2014-2017, Klarna AB +%%% Copyright (c) 2014-2018, Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -43,7 +43,7 @@ | ?undef. %% Is kafka error code --define(IS_ERROR(EC), kpro_error_code:is_error(EC)). +-define(IS_ERROR(EC), ((EC) =/= ?no_error)). -define(KV(Key, Value), {Key, Value}). -define(TKV(Ts, Key, Value), {Ts, Key, Value}). diff --git a/rebar.config b/rebar.config index 655d5f8b..8ad83a17 100644 --- a/rebar.config +++ b/rebar.config @@ -1,4 +1,4 @@ {deps, [ {supervisor3, "1.1.5"} - , {kafka_protocol, "1.1.2"} + , {kafka_protocol, "2.0.0"} ]}. {erl_opts, [warn_unused_vars,warn_shadow_vars,warn_unused_import,warn_obsolete_guard,debug_info]}. diff --git a/rebar.config.script b/rebar.config.script index 2b393a41..d6b1e799 100644 --- a/rebar.config.script +++ b/rebar.config.script @@ -1,17 +1,20 @@ IsRebar3 = erlang:function_exported(rebar3, main, 1), case IsRebar3 of - true -> - CONFIG; - false -> - Rebar3Deps = proplists:get_value(deps, CONFIG), - Rebar2Deps = [ - {supervisor3, ".*", - {git, "https://github.com/klarna/supervisor3", - {tag, proplists:get_value(supervisor3, Rebar3Deps)}}}, - {kafka_protocol, ".*", - {git, "https://github.com/klarna/kafka_protocol.git", - {tag, proplists:get_value(kafka_protocol, Rebar3Deps)}}} - ], - lists:keyreplace(deps, 1, CONFIG, {deps, Rebar2Deps}) + true -> + CONFIG; + false -> + URLs = [ {supervisor3, "https://github.com/klarna/supervisor3.git"} + , {kafka_protocol, "https://github.com/klarna/kafka_protocol.git"} + ], + Rebar3Deps = proplists:get_value(deps, CONFIG), + Rebar2Deps = + lists:map( + fun({Name, URL}) -> + case proplists:get_value(Name, Rebar3Deps) of + {git, _, _} = Git -> {Name, ".*", Git}; + Vsn -> {Name, ".*", {git, URL, {tag, Vsn}}} + end + end, URLs), + lists:keyreplace(deps, 1, CONFIG, {deps, Rebar2Deps}) end. diff --git a/scripts/setup-test-env.sh b/scripts/setup-test-env.sh index c04633ea..f0243be0 100755 --- a/scripts/setup-test-env.sh +++ b/scripts/setup-test-env.sh @@ -1,12 +1,28 @@ -#!/bin/bash -e +#!/bin/bash -eu + +case $1 in + 0.9*) + VERSION="0.9.0.0";; + 0.10*) + VERSION="0.10.2.1";; + 0.11*) + VERSION="0.11.0.2";; + 1.*) + VERSION="1.1.0";; + *) + echo "unknown kafka version $1" + exit 1 +esac + +export KAFKA_VERSION=$VERSION THIS_DIR="$(cd "$(dirname "$0")" && pwd)" cd $THIS_DIR/../docker -sudo docker-compose -f docker-compose-kafka-2.yml down || true -sudo docker-compose -f docker-compose-basic.yml build -sudo docker-compose -f docker-compose-kafka-2.yml up -d +sudo KAFKA_VERSION=${KAFKA_VERSION} docker-compose -f docker-compose-kafka-2.yml down || true +sudo KAFKA_VERSION=${KAFKA_VERSION} docker-compose -f docker-compose-basic.yml build +sudo KAFKA_VERSION=${KAFKA_VERSION} docker-compose -f docker-compose-kafka-2.yml up -d n=0 while [ "$(sudo docker exec kafka_1 bash -c '/opt/kafka/bin/kafka-topics.sh --zookeeper zookeeper --describe')" != '' ]; do @@ -41,3 +57,9 @@ create_topic "test-topic" # this is to warm-up kafka group coordinator for deterministic in tests sudo docker exec kafka_1 /opt/kafka/bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --new-consumer --group test-group --describe > /dev/null 2>&1 + +# for kafka 0.11 or later, add sasl-scram test credentials +if [[ "$KAFKA_VERSION" != 0.9* ]] && [[ "$KAFKA_VERSION" != 0.10* ]]; then + sudo docker exec kafka_1 /opt/kafka/bin/kafka-configs.sh --zookeeper zookeeper:2181 --alter --add-config 'SCRAM-SHA-256=[iterations=8192,password=ecila],SCRAM-SHA-512=[password=ecila]' --entity-type users --entity-name alice +fi + diff --git a/scripts/vsn-check.sh b/scripts/vsn-check.sh index 5785e9de..457d6c15 100755 --- a/scripts/vsn-check.sh +++ b/scripts/vsn-check.sh @@ -50,5 +50,5 @@ halt(LoopFun(LoopFun, io:get_line([]), Deps0)). EOF ) -grep -E "dep_.*_commit" $MAKEFILE | sed 's/dep_//' | sed 's/_commit//' | erl -noshell -eval "$ESCRIPT" +grep -E "dep_.*_commit\s=" $MAKEFILE | sed 's/dep_//' | sed 's/_commit//' | erl -noshell -eval "$ESCRIPT" diff --git a/src/brod.app.src b/src/brod.app.src index 9ae31743..4a62da86 100644 --- a/src/brod.app.src +++ b/src/brod.app.src @@ -1,9 +1,9 @@ %% -*- mode:erlang -*- {application,brod, [{description,"Apache Kafka Erlang client library"}, - {vsn,"3.5.2"}, + {vsn,"3.6.0"}, {registered,[]}, - {applications,[kernel,stdlib,ssl,kafka_protocol,supervisor3]}, + {applications,[kernel,stdlib,kafka_protocol,supervisor3]}, {env,[]}, {mod, {brod, []}}, {modules,[]}, diff --git a/src/brod.erl b/src/brod.erl index 02f75b1d..fecc9cbf 100644 --- a/src/brod.erl +++ b/src/brod.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2014-2017, Klarna AB +%%% Copyright (c) 2014-2018, Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -82,8 +82,7 @@ , resolve_offset/4 , resolve_offset/5 , fetch/4 - , fetch/7 - , fetch/8 + , fetch/5 , connect_leader/4 , list_all_groups/2 , list_groups/2 @@ -93,6 +92,15 @@ , fetch_committed_offsets/3 ]). +%% deprecated +-export([ fetch/7 + , fetch/8 + ]). + +-deprecated([ {fetch, 7, next_version} + , {fetch, 8, next_version} + ]). + -export_type([ call_ref/0 , cg/0 , cg_protocol_type/0 @@ -100,12 +108,14 @@ , client_config/0 , client_id/0 , compression/0 + , connection/0 + , conn_config/0 , consumer_config/0 , consumer_option/0 , consumer_options/0 - , corr_id/0 , endpoint/0 , error_code/0 + , fetch_opts/0 , group_config/0 , group_generation_id/0 , group_id/0 @@ -127,7 +137,6 @@ , produce_reply/0 , produce_result/0 , received_assignments/0 - , sock_opts/0 , topic/0 , value/0 ]). @@ -180,9 +189,11 @@ | size_stat_window. -type consumer_options() :: [{consumer_option(), integer()}]. -type consumer_config() :: brod_consumer:config(). +-type connection() :: kpro:connection(). +-type conn_config() :: [{atom(), term()}] | kpro:conn_config(). %% consumer groups --type group_id() :: binary(). +-type group_id() :: kpro:group_id(). -type group_member_id() :: binary(). -type group_member() :: {group_member_id(), #kafka_group_member_metadata{}}. -type group_generation_id() :: non_neg_integer(). @@ -191,10 +202,7 @@ -type received_assignments() :: [#brod_received_assignment{}]. -type cg() :: #brod_cg{}. -type cg_protocol_type() :: binary(). - -%% internals --type corr_id() :: kpro:corr_id(). --type sock_opts() :: brod_sock:options(). +-type fetch_opts() :: kpro:fetch_opts(). %%%_* APIs ===================================================================== @@ -226,59 +234,79 @@ start_client(BootstrapEndpoints, ClientId) -> start_client(BootstrapEndpoints, ClientId, []). %% @doc Start a client. +%% %% BootstrapEndpoints: %% Kafka cluster endpoints, can be any of the brokers in the cluster %% which does not necessarily have to be a leader of any partition, %% e.g. a load-balanced entrypoint to the remote kakfa cluster. -%% ClientId: -%% Atom to identify the client process -%% Config: -%% Proplist, possible values: -%% restart_delay_seconds (optional, default=10) -%% How much time to wait between attempts to restart brod_client -%% process when it crashes -%% max_metadata_sock_retry (optional, default=1) -%% Number of retries if failed fetching metadata due to socket error -%% get_metadata_timeout_seconds(optional, default=5) -%% Return timeout error from brod_client:get_metadata/2 in case the -%% respons is not received from kafka in this configured time. -%% reconnect_cool_down_seconds (optional, default=1) -%% Delay this configured number of seconds before retrying to -%% estabilish a new connection to the kafka partition leader. -%% allow_topic_auto_creation (optional, default=true) -%% By default, brod respects what is configured in broker about -%% topic auto-creation. i.e. whatever auto.create.topics.enable -%% is set in borker configuration. -%% However if 'allow_topic_auto_creation' is set to 'false' in client -%% config, brod will avoid sending metadata requests that may cause an -%% auto-creation of the topic regardless of what the broker config is. -%% auto_start_producers (optional, default=false) -%% If true, brod client will spawn a producer automatically when -%% user is trying to call 'produce' but did not call -%% brod:start_producer explicitly. Can be useful for applications -%% which don't know beforehand which topics they will be working with. -%% default_producer_config (optional, default=[]) -%% Producer configuration to use when auto_start_producers is true. -%% @see brod_producer:start_link/4. for details about producer config -%% ssl (optional, default=false) -%% true | false | [{certfile, ...},{keyfile, ...},{cacertfile, ...}] -%% When true, brod will try to upgrade tcp connection to ssl using default -%% ssl options. List of ssl options implies ssl=true. -%% sasl (optional, default=undefined) -%% Credentials for SASL/Plain authentication. -%% {plain, "username", "password"} -%% connect_timeout (optional, default=5000) -%% Timeout when trying to connect to one endpoint. -%% request_timeout (optional, default=240000, constraint: >= 1000) -%% Timeout when waiting for a response, socket restart when timedout. -%% query_api_versions (optional, default=true) -%% Must be set to false to work with kafka versions prior to 0.10, -%% When set to 'true', brod_sock will send a query request to get -%% the broker supported API version ranges. When set to 'false', brod -%% will alway use the lowest supported API version when sending requests -%% to kafka. Supported API version ranges can be found in: -%% `brod_kafka_apis:supported_versions/1' -%% @end +%% +%% ClientId: Atom to identify the client process. +%% +%% Config is a proplist, possible values: +%% restart_delay_seconds (optional, default=10) +%% How much time to wait between attempts to restart brod_client +%% process when it crashes +%% +%% get_metadata_timeout_seconds (optional, default=5) +%% Return `{error, timeout}' from `brod_client:get_xxx' calls if +%% responses for APIs such as `metadata', `find_coordinator' +%% is not received in time. +%% +%% reconnect_cool_down_seconds (optional, default=1) +%% Delay this configured number of seconds before retrying to +%% estabilish a new connection to the kafka partition leader. +%% +%% allow_topic_auto_creation (optional, default=true) +%% By default, brod respects what is configured in broker about +%% topic auto-creation. i.e. whatever `auto.create.topics.enable' +%% is set in borker configuration. +%% However if `allow_topic_auto_creation' is set to `false' in client +%% config, brod will avoid sending metadata requests that may cause an +%% auto-creation of the topic regardless of what broker config is. +%% +%% auto_start_producers (optional, default=false) +%% If true, brod client will spawn a producer automatically when +%% user is trying to call `produce' but did not call `brod:start_producer' +%% explicitly. Can be useful for applications which don't know beforehand +%% which topics they will be working with. +%% +%% default_producer_config (optional, default=[]) +%% Producer configuration to use when auto_start_producers is true. +%% @see brod_producer:start_link/4. for details about producer config +%% +%% Connection config entries can be added in the same proplist. +%% see `kpro_connection.erl' in `kafka_protocol' for more details. +%% +%% ssl (optional, default=false) +%% `true | false | ssl:ssl_option()' +%% `true' is translated to `[]' as `ssl:ssl_option()' i.e. all default. +%% +%% sasl (optional, default=undefined) +%% Credentials for SASL/Plain authentication. +%% `{mechanism(), Filename}' or `{mechanism(), UserName, Password}' +%% where mechanism can be atoms: plain (for "PLAIN"), scram_sha_256 +%% (for "SCRAM-SHA-256") or scram_sha_512 (for SCRAM-SHA-512). +%% `Filename' should be a file consisting two lines, first line +%% is the username and second line is the password. +%% `Username', `Password' should be `string() | binary()' +%% +%% connect_timeout (optional, default=5000) +%% Timeout when trying to connect to an endpoint. +%% +%% request_timeout (optional, default=240000, constraint: >= 1000) +%% Timeout when waiting for a response, connection restart when timed out. +%% query_api_versions (optional, default=true) +%% Must be set to false to work with kafka versions prior to 0.10, +%% When set to `true', at connection start, brod will send a query request +%% to get the broker supported API version ranges. +%% When set to 'false', brod will alway use the lowest supported API version +%% when sending requests to kafka. +%% Supported API version ranges can be found in: +%% `brod_kafka_apis:supported_versions/1' +%% extra_sock_opts (optional, default=[]) +%% Extra socket options to tune socket performance. +%% e.g. [{sndbuf, 1 bsl 20}]. +%% ref: http://erlang.org/doc/man/gen_tcp.html#type-option -spec start_client([endpoint()], client_id(), client_config()) -> ok | {error, any()}. start_client(BootstrapEndpoints, ClientId, Config) -> @@ -316,7 +344,6 @@ stop_client(Client) when is_pid(Client) -> %% @doc Dynamically start a per-topic producer. %% @see brod_producer:start_link/4. for details about producer config. -%% @end -spec start_producer(client(), topic(), producer_config()) -> ok | {error, any()}. start_producer(Client, TopicName, ProducerConfig) -> @@ -324,7 +351,6 @@ start_producer(Client, TopicName, ProducerConfig) -> %% @doc Dynamically start a topic consumer. %% @see brod_consumer:start_link/5. for details about consumer config. -%% @end -spec start_consumer(client(), topic(), consumer_config()) -> ok | {error, any()}. start_consumer(Client, TopicName, ConsumerConfig) -> @@ -336,7 +362,6 @@ start_consumer(Client, TopicName, ConsumerConfig) -> %% is not statically configured for them. %% It is up to the callers how they want to distribute their data %% (e.g. random, roundrobin or consistent-hashing) to the partitions. -%% @end -spec get_partitions_count(client(), topic()) -> {ok, pos_integer()} | {error, any()}. get_partitions_count(Client, Topic) -> @@ -370,7 +395,6 @@ produce(Pid, Value) -> %% or a message set if Value is a (nested) kv-list, in this case Key %% is discarded (only the keys in kv-list are sent to kafka). %% The pid should be a partition producer pid, NOT client pid. -%% @end -spec produce(pid(), key(), value()) -> {ok, call_ref()} | {error, any()}. produce(ProducerPid, Key, Value) -> @@ -382,7 +406,6 @@ produce(ProducerPid, Key, Value) -> %% instead of PartFun). %% This function first lookup the producer pid, %% then call produce/3 to do the real work. -%% @end -spec produce(client(), topic(), partition() | partition_fun(), key(), value()) -> {ok, call_ref()} | {error, any()}. produce(Client, Topic, PartFun, Key, Value) when is_function(PartFun) -> @@ -408,7 +431,6 @@ produce_sync(Pid, Value) -> %% This function will not return until a response is received from kafka, %% however if producer is started with required_acks set to 0, this function %% will return onece the messages is buffered in the producer process. -%% @end -spec produce_sync(pid(), key(), value()) -> ok | {error, any()}. produce_sync(Pid, Key, Value) -> @@ -424,7 +446,6 @@ produce_sync(Pid, Key, Value) -> %% This function will not return until a response is received from kafka, %% however if producer is started with required_acks set to 0, this function %% will return once the messages are buffered in the producer process. -%% @end -spec produce_sync(client(), topic(), partition() | partition_fun(), key(), value()) -> ok | {error, any()}. produce_sync(Client, Topic, Partition, Key, Value) -> @@ -436,7 +457,6 @@ produce_sync(Client, Topic, Partition, Key, Value) -> %% @doc Version of produce_sync/5 that returns the offset assigned by Kafka %% If producer is started with required_acks set to 0, the offset will be %% ?BROD_PRODUCE_UNKNOWN_OFFSET. -%% @end -spec produce_sync_offset(client(), topic(), partition() | partition_fun(), key(), value()) -> {ok, offset()} | {error, any()}. produce_sync_offset(Client, Topic, Partition, Key, Value) -> @@ -463,7 +483,6 @@ sync_produce_request(CallRef, Timeout) -> %% @doc As sync_produce_request_offset/1, but also returning assigned offset %% See produce_sync_offset/5. -%% @end -spec sync_produce_request_offset(call_ref()) -> {ok, offset()} | {error, Reason :: any()}. sync_produce_request_offset(CallRef) -> @@ -486,7 +505,6 @@ sync_produce_request_offset(CallRef, Timeout) -> %% -include_lib(brod/include/brod.hrl) to access the records. %% In case #kafka_fetch_error{} is received the subscriber should re-subscribe %% itself to resume the data stream. -%% @end -spec subscribe(client(), pid(), topic(), partition(), consumer_options()) -> {ok, pid()} | {error, any()}. subscribe(Client, SubscriberPid, Topic, Partition, Options) -> @@ -597,27 +615,24 @@ start_link_topic_subscriber(Client, Topic, Partitions, CbModule, CbInitArg). %% @doc Fetch broker metadata -%% Return the message body of metadata_response. -%% See kpro_schema.erl for details -%% @end +%% Return the message body of `metadata' response. +%% See `kpro_schema.erl' for details -spec get_metadata([endpoint()]) -> {ok, kpro:struct()} | {error, any()}. get_metadata(Hosts) -> brod_utils:get_metadata(Hosts). -%% @doc Fetch broker metadata -%% Return the message body of metadata_response. +%% @doc Fetch broker/topic metadata +%% Return the message body of `metadata' response. %% See `kpro_schema.erl' for struct details -%% @end -spec get_metadata([endpoint()], [topic()]) -> {ok, kpro:struct()} | {error, any()}. get_metadata(Hosts, Topics) -> brod_utils:get_metadata(Hosts, Topics). -%% @doc Fetch broker metadata -%% Return the message body of metadata_response. +%% @doc Fetch broker/topic metadata +%% Return the message body of `metadata' response. %% See `kpro_schema.erl' for struct details -%% @end --spec get_metadata([endpoint()], [topic()], sock_opts()) -> +-spec get_metadata([endpoint()], [topic()], conn_config()) -> {ok, kpro:struct()} | {error, any()}. get_metadata(Hosts, Topics, Options) -> brod_utils:get_metadata(Hosts, Topics, Options). @@ -636,90 +651,103 @@ resolve_offset(Hosts, Topic, Partition, Time) -> %% @doc Resolve semantic offset or timestamp to real offset. -spec resolve_offset([endpoint()], topic(), partition(), - offset_time(), sock_opts()) -> + offset_time(), conn_config()) -> {ok, offset()} | {error, any()}. -resolve_offset(Hosts, Topic, Partition, Time, Options) when is_list(Options) -> - brod_utils:resolve_offset(Hosts, Topic, Partition, Time, Options). +resolve_offset(Hosts, Topic, Partition, Time, ConnCfg) -> + brod_utils:resolve_offset(Hosts, Topic, Partition, Time, ConnCfg). -%% @equiv fetch(Hosts, Topic, Partition, Offset, 1000, 0, 100000) --spec fetch([endpoint()], topic(), partition(), integer()) -> - {ok, [message()]} | {error, any()}. -fetch(Hosts, Topic, Partition, Offset) -> - fetch(Hosts, Topic, Partition, Offset, - _MaxWaitTime = 1000, _MinBytes = 0, _MaxBytes = 100000). +%% @doc Fetch a single message set from the given topic-partition. +%% The first arg can either be an already established connection to leader, +%% or `{Endpoints, ConnConfig}' so to establish a new connection before fetch. +-spec fetch(connection() | [endpoint()] | {[endpoint()], conn_config()}, + topic(), partition(), integer()) -> + {ok, [message()]} | {error, any()}. +fetch(ConnOrBootstrap, Topic, Partition, Offset) -> + Opts = #{ max_wait_time => 1000 + , min_bytes => 0 + , max_bytes => 100 bsl 10 %% 100K + }, + fetch(ConnOrBootstrap, Topic, Partition, Offset, Opts). -%% @equiv fetch(Hosts, Topic, Partition, Offset, Wait, MinBytes, MaxBytes, []) +%% @doc Fetch a single message set from the given topic-partition. +%% The first arg can either be an already established connection to leader, +%% or `{Endpoints, ConnConfig}' so to establish a new connection before fetch. +-spec fetch(connection() | {[endpoint()], conn_config()}, + topic(), partition(), offset(), fetch_opts()) -> + {ok, [message()]} | {error, any()}. +fetch(Hosts, Topic, Partition, Offset, Opts) when is_list(Hosts) -> + fetch({Hosts, _ConnConfig = []}, Topic, Partition, Offset, Opts); +fetch(ConnOrBootstrap, Topic, Partition, Offset, Opts) -> + brod_utils:fetch(ConnOrBootstrap, Topic, Partition, Offset, Opts). + +%% @deprecated +%% fetch(Hosts, Topic, Partition, Offset, Wait, MinBytes, MaxBytes, []) -spec fetch([endpoint()], topic(), partition(), offset(), non_neg_integer(), non_neg_integer(), pos_integer()) -> {ok, [message()]} | {error, any()}. fetch(Hosts, Topic, Partition, Offset, MaxWaitTime, MinBytes, MaxBytes) -> fetch(Hosts, Topic, Partition, Offset, MaxWaitTime, MinBytes, MaxBytes, []). -%% @doc Fetch a single message set from the given topic-partition. +%% @deprecated Fetch a single message set from the given topic-partition. -spec fetch([endpoint()], topic(), partition(), offset(), non_neg_integer(), non_neg_integer(), pos_integer(), - sock_opts()) -> {ok, [message()]} | {error, any()}. + conn_config()) -> {ok, [message()]} | {error, any()}. fetch(Hosts, Topic, Partition, Offset, - MaxWaitTime, MinBytes, MaxBytes, Options) -> - brod_utils:fetch(Hosts, Topic, Partition, Offset, - MaxWaitTime, MinBytes, MaxBytes, Options). + MaxWaitTime, MinBytes, MaxBytes, ConnConfig) -> + FetchOpts = #{ max_wait_time => MaxWaitTime + , min_bytes => MinBytes + , max_bytes => MaxBytes + }, + fetch({Hosts, ConnConfig}, Topic, Partition, Offset, FetchOpts). %% @doc Connect partition leader. -spec connect_leader([endpoint()], topic(), partition(), - sock_opts()) -> {ok, pid()}. -connect_leader(Hosts, Topic, Partition, Options) -> - {ok, Metadata} = get_metadata(Hosts, [Topic], Options), - {ok, {Host, Port}} = - brod_utils:find_leader_in_metadata(Metadata, Topic, Partition), - %% client id matters only for producer clients - brod_sock:start_link(self(), Host, Port, ?BROD_DEFAULT_CLIENT_ID, Options). + conn_config()) -> {ok, pid()}. +connect_leader(Hosts, Topic, Partition, ConnConfig) -> + kpro:connect_partition_leader(Hosts, ConnConfig, Topic, Partition). %% @doc List ALL consumer groups in the given kafka cluster. -%% NOTE: Exception if failed against any of the coordinator brokers. -%% @end --spec list_all_groups([endpoint()], sock_opts()) -> +%% NOTE: Exception if failed to connect any of the coordinator brokers. +-spec list_all_groups([endpoint()], conn_config()) -> [{endpoint(), [cg()] | {error, any()}}]. -list_all_groups(Endpoints, SockOpts) -> - brod_utils:list_all_groups(Endpoints, SockOpts). +list_all_groups(Endpoints, ConnCfg) -> + brod_utils:list_all_groups(Endpoints, ConnCfg). %% @doc List consumer groups in the given group coordinator broker. --spec list_groups(endpoint(), sock_opts()) -> {ok, [cg()]} | {error, any()}. -list_groups(CoordinatorEndpoint, SockOpts) -> - brod_utils:list_groups(CoordinatorEndpoint, SockOpts). +-spec list_groups(endpoint(), conn_config()) -> {ok, [cg()]} | {error, any()}. +list_groups(CoordinatorEndpoint, ConnCfg) -> + brod_utils:list_groups(CoordinatorEndpoint, ConnCfg). %% @doc Describe consumer groups. The given consumer group IDs should be all %% managed by the coordinator-broker running at the given endpoint. %% Otherwise error codes will be returned in the result structs. -%% Return `describe_groups_response' response body field named `groups'. +%% Return `describe_groups' response body field named `groups'. %% See `kpro_schema.erl' for struct details -%% @end --spec describe_groups(endpoint(), sock_opts(), [group_id()]) -> +-spec describe_groups(endpoint(), conn_config(), [group_id()]) -> {ok, [kpro:struct()]} | {error, any()}. -describe_groups(CoordinatorEndpoint, SockOpts, IDs) -> - brod_utils:describe_groups(CoordinatorEndpoint, SockOpts, IDs). +describe_groups(CoordinatorEndpoint, ConnCfg, IDs) -> + brod_utils:describe_groups(CoordinatorEndpoint, ConnCfg, IDs). %% @doc Connect to consumer group coordinator broker. %% Done in steps: 1) connect to any of the given bootstrap ednpoints; %% 2) send group_coordinator_request to resolve group coordinator endpoint;; -%% 3) connect to the resolved endpoint and return the brod_sock pid -%% @end --spec connect_group_coordinator([endpoint()], sock_opts(), group_id()) -> +%% 3) connect to the resolved endpoint and return the connection pid +-spec connect_group_coordinator([endpoint()], conn_config(), group_id()) -> {ok, pid()} | {error, any()}. -connect_group_coordinator(BootstrapEndpoints, SockOpts, GroupId) -> - brod_utils:connect_group_coordinator(BootstrapEndpoints, SockOpts, GroupId). +connect_group_coordinator(BootstrapEndpoints, ConnCfg, GroupId) -> + Args = #{type => group, id => GroupId}, + kpro:connect_coordinator(BootstrapEndpoints, ConnCfg, Args). %% @doc Fetch committed offsets for ALL topics in the given consumer group. -%% Return the `responses' field of the `offset_fetch_response' response. +%% Return the `responses' field of the `offset_fetch' response. %% See `kpro_schema.erl' for struct details. -%% @end --spec fetch_committed_offsets([endpoint()], sock_opts(), group_id()) -> +-spec fetch_committed_offsets([endpoint()], conn_config(), group_id()) -> {ok, [kpro:struct()]} | {error, any()}. -fetch_committed_offsets(BootstrapEndpoints, SockOpts, GroupId) -> - brod_utils:fetch_committed_offsets(BootstrapEndpoints, SockOpts, GroupId, []). +fetch_committed_offsets(BootstrapEndpoints, ConnCfg, GroupId) -> + brod_utils:fetch_committed_offsets(BootstrapEndpoints, ConnCfg, GroupId, []). %% @doc Same as `fetch_committed_offsets/3', %% but works with a started `brod_client' -%% @end -spec fetch_committed_offsets(client(), group_id()) -> {ok, [kpro:struct()]} | {error, any()}. fetch_committed_offsets(Client, GroupId) -> diff --git a/src/brod_auth_backend.erl b/src/brod_auth_backend.erl deleted file mode 100644 index 637a2a1e..00000000 --- a/src/brod_auth_backend.erl +++ /dev/null @@ -1,38 +0,0 @@ -%%% -%%% Copyright (c) 2017, Klarna AB -%%% -%%% Licensed under the Apache License, Version 2.0 (the "License"); -%%% you may not use this file except in compliance with the License. -%%% You may obtain a copy of the License at -%%% -%%% http://www.apache.org/licenses/LICENSE-2.0 -%%% -%%% Unless required by applicable law or agreed to in writing, software -%%% distributed under the License is distributed on an "AS IS" BASIS, -%%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -%%% See the License for the specific language governing permissions and -%%% limitations under the License. -%%% - --module(brod_auth_backend). - --export([auth/7]). - --callback auth(Host :: string(), Sock :: gen_tcp:socket() | ssl:sslsocket(), - Mod :: gen_tcp | ssl, ClientId :: binary(), - Timeout :: pos_integer(), SaslOpts :: term()) -> - ok | {error, Reason :: term()}. - --spec auth(CallbackModule :: atom(), Host :: string(), - Sock :: gen_tcp:socket() | ssl:sslsocket(), - Mod :: gen_tcp | ssl, ClientId :: binary(), - Timeout :: pos_integer(), SaslOpts :: term()) -> - ok | {error, Reason :: term()}. -auth(CallbackModule, Host, Sock, Mod, ClientId, Timeout, SaslOpts) -> - CallbackModule:auth(Host, Sock, Mod, ClientId, Timeout, SaslOpts). - -%%%_* Emacs ==================================================================== -%%% Local Variables: -%%% allout-layout: t -%%% erlang-indent-level: 2 -%%% End: diff --git a/src/brod_cg_commits.erl b/src/brod_cg_commits.erl index 923e8ec1..e09b2970 100644 --- a/src/brod_cg_commits.erl +++ b/src/brod_cg_commits.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2017 Klarna AB +%%% Copyright (c) 2017-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -16,7 +16,6 @@ %%%============================================================================= %%% @doc This is a utility module to help force commit offsets to kafka. -%%% @end %%%============================================================================= -module(brod_cg_commits). @@ -88,7 +87,6 @@ run(ClientId, GroupInput) -> %% then commit given offsets to kafka. %% In case not all given partitions are assigned to it, %% it will terminate with an exit exception -%% @end -spec start_link(brod:client(), group_input()) -> {ok, pid()} | {error, any()}. start_link(Client, GroupInput) -> gen_server:start_link(?MODULE, {Client, GroupInput}, []). @@ -105,7 +103,6 @@ stop(Pid) -> %% @doc Make a call to the resetter process, the call will be blocked %% until offsets are committed. -%% @end -spec sync(pid()) -> ok. sync(Pid) -> ok = gen_server:call(Pid, sync, infinity). @@ -126,7 +123,6 @@ assignments_revoked(Pid) -> %% @doc This function is called only when `partition_assignment_strategy' %% is set for `callback_implemented' in group config. -%% @end -spec assign_partitions(pid(), [brod:group_member()], [{brod:topic(), brod:partition()}]) -> [{member_id(), [brod:partition_assignment()]}]. @@ -138,7 +134,6 @@ assign_partitions(Pid, Members, TopicPartitionList) -> %% for subscriber. %% NOTE: this function is called only when it is DISABLED to commit offsets %% to kafka. i.e. offset_commit_policy is set to consumer_managed -%% @end -spec get_committed_offsets(pid(), [{brod:topic(), brod:partition()}]) -> {ok, [{{brod:topic(), brod:partition()}, brod:offset()}]}. get_committed_offsets(_Pid, _TopicPartitions) -> {ok, []}. @@ -289,7 +284,6 @@ maybe_reply_sync(#state{pending_sync = From} = State) -> log(State, info, "done\n", []), State#state{pending_sync = ?undef}. -%% @private -spec assign_all_to_self(pid(), [brod:group_member()], [{topic(), partition()}]) -> [{member_id(), [brod:partition_assignment()]}]. @@ -298,14 +292,13 @@ assign_all_to_self(CoordinatorPid, Members, TopicPartitions) -> Groupped = brod_utils:group_per_key(TopicPartitions), [{MyMemberId, Groupped}]. -%% @private I am the current leader because I am assigning partitions, +%% I am the current leader because I am assigning partitions, %% however my member ID is kept in `brod_group_coordinator' looping state, %% i.e. not exposed to callback module. So I make use of the coordinator %% pid in member's `user_data' to find my member ID. %% %% NOTE: I can not make a gen_server call to `brod_group_coordinator' because %% I am currently being called by coordinator -- deadlock otherwise. -%% @end find_my_member_id(CoordinatorPid, [H | T]) -> {MemberId, #kafka_group_member_metadata{user_data = UD}} = H, try @@ -317,7 +310,6 @@ find_my_member_id(CoordinatorPid, [H | T]) -> find_my_member_id(CoordinatorPid, T) end. -%% @private log(#state{groupId = GroupId}, Level, Fmt, Args) -> brod_utils:log(Level, "Group member (~s,coor=~p):\n" ++ Fmt, diff --git a/src/brod_cli.erl b/src/brod_cli.erl index 5e002cc2..49fc749f 100644 --- a/src/brod_cli.erl +++ b/src/brod_cli.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2017, Klarna AB +%%% Copyright (c) 2017-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -16,7 +16,7 @@ -module(brod_cli). --ifdef(BROD_CLI). +-ifdef(build_brod_cli). -export([main/1, main/2]). @@ -24,11 +24,19 @@ -define(CLIENT, brod_cli_client). +-ifdef(OTP_RELEASE). +-define(BIND_STACKTRACE(Var), :Var). +-define(GET_STACKTRACE(Var), ok). +-else. +-define(BIND_STACKTRACE(Var), ). +-define(GET_STACKTRACE(Var), Var = erlang:get_stacktrace()). +-endif. + %% 'halt' is for escript, stop the vm immediately %% 'exit' is for testing, we want eunit or ct to be able to capture -define(STOP(How), begin - application:stop(brod), + _ = application:stop(brod), case How of 'halt' -> erlang:halt(?LINE); 'exit' -> erlang:exit(?LINE) @@ -146,7 +154,7 @@ options: to a consultable Erlang term format. Expr: An Erlang expression to be evaluated for each message. Bound variable to be used in the - expression: Offset, Key, Value, CRC, TsType, Ts. + expression: Offset, Key, Value, TsType, Ts. Print nothing if the evaluation result in 'ok', otherwise print the evaluated io-list. [default: v] @@ -297,8 +305,10 @@ options: -type command() :: string(). -main(Args) -> main(Args, halt). +main(Args) -> + _ = main(Args, halt). +-spec main([string()], halt | exit) -> no_return(). main(["-h" | _], _Stop) -> print(?MAIN_DOC); main(["--help" | _], _Stop) -> @@ -324,7 +334,6 @@ main(_, Stop) -> print(?MAIN_DOC), ?STOP(Stop). -%% @private -spec main(command(), string(), [string()], halt | exit) -> _ | no_return(). main(Command, Doc, Args0, Stop) -> IsHelp = lists:member("--help", Args0) orelse lists:member("-h", Args0), @@ -343,7 +352,6 @@ main(Command, Doc, Args0, Stop) -> main(Command, Doc, Args, Stop, LogLevel) end. -%% @private -spec main(command(), string(), [string()], halt | exit, log_level()) -> _ | no_return(). main(Command, Doc, Args, Stop, LogLevel) -> @@ -351,8 +359,8 @@ main(Command, Doc, Args, Stop, LogLevel) -> try docopt:docopt(Doc, Args, [debug || LogLevel =:= ?LOG_LEVEL_DEBUG]) catch - C1 : E1 -> - Stack1 = erlang:get_stacktrace(), + C1 : E1 ?BIND_STACKTRACE(Stack1) -> + ?GET_STACKTRACE(Stack1), verbose("~p:~p\n~p\n", [C1, E1, Stack1]), print(Doc), ?STOP(Stop) @@ -371,26 +379,24 @@ main(Command, Doc, Args, Stop, LogLevel) -> {ok, _} = application:ensure_all_started(brod), try Brokers = parse(ParsedArgs, "--brokers", fun parse_brokers/1), - SockOpts = parse_sock_opts(ParsedArgs), + ConnConfig0 = parse_connection_config(ParsedArgs), Paths = parse(ParsedArgs, "--ebin-paths", fun parse_paths/1), NoApiQuery = parse(ParsedArgs, "--no-api-vsn-query", fun parse_boolean/1), ok = code:add_pathsa(Paths), - verbose("sock opts: ~p\n", [SockOpts]), - ClientConfig = [{query_api_versions, not NoApiQuery} | SockOpts], - run(Command, Brokers, ClientConfig, ParsedArgs) + SockOpts = [{query_api_versions, not NoApiQuery} | ConnConfig0], + verbose("connection config: ~p\n", [SockOpts]), + run(Command, Brokers, SockOpts, ParsedArgs) catch throw : Reason when is_binary(Reason) -> %% invalid options etc. logerr([Reason, "\n"]), ?STOP(Stop); - C2 : E2 -> - %% crashed - Stack2 = erlang:get_stacktrace(), + C2 : E2 ?BIND_STACKTRACE(Stack2) -> + ?GET_STACKTRACE(Stack2), logerr("~p:~p\n~p\n", [C2, E2, Stack2]), ?STOP(Stop) end. -%% @private run(?META_CMD, Brokers, Topic, SockOpts, Args) -> Topics = case Topic of <<"*">> -> []; %% fetch all topics @@ -432,8 +438,8 @@ run(?FETCH_CMD, Brokers, Topic, SockOpts, Args) -> MaxBytes = parse(Args, "--max-bytes", fun parse_size/1), {ok, Sock} = brod:connect_leader(Brokers, Topic, Partition, SockOpts), Offset = resolve_begin_offset(Sock, Topic, Partition, Offset0), - FetchFun = brod_utils:make_fetch_fun(Sock, Topic, Partition, - Wait, _MinBytes = 1, MaxBytes), + FetchOpts = #{max_wait_time => Wait, max_bytes => MaxBytes}, + FetchFun = brod_utils:make_fetch_fun(Sock, Topic, Partition, FetchOpts), Count = case Count0 < 0 of true -> 1000000000; %% as if an infinite loop false -> Count0 @@ -528,7 +534,6 @@ run(?PIPE_CMD, Brokers, Topic, SockOpts, Args) -> _ = erlang:monitor(process, ReaderPid), pipe(ReaderPid, SendFun, queue:new()). -%% @private run(?GROUPS_CMD, Brokers, SockOpts, Args) -> IDs = parse(Args, "--ids", fun parse_cg_ids/1), cg(Brokers, SockOpts, IDs); @@ -549,7 +554,6 @@ run(Cmd, Brokers, SockOpts, Args) -> Topic = parse(Args, "--topic", fun bin/1), run(Cmd, Brokers, Topic, SockOpts, Args). -%% @private resolve_offsets_print(Topic, all, Time, IsOneLine) -> Offsets = resolve_offsets(Topic, Time), Outputs = @@ -566,7 +570,6 @@ resolve_offsets_print(Topic, Partition, Time, _) when is_integer(Partition) -> {ok, Offset} = resolve_offset(Topic, Partition, Time), print(integer_to_list(Offset)). -%% @private resolve_offsets(Topic, Time) -> {ok, Count} = brod_client:get_partitions_count(?CLIENT, Topic), Partitions = lists:seq(0, Count - 1), @@ -576,12 +579,10 @@ resolve_offsets(Topic, Time) -> {P, Offset} end, Partitions). -%% @private resolve_offset(Topic, Partition, Time) -> {ok, SockPid} = brod_client:get_leader_connection(?CLIENT, Topic, Partition), brod_utils:resolve_offset(SockPid, Topic, Partition, Time). -%% @private show_commits(GroupId, Topic) -> case brod:fetch_committed_offsets(?CLIENT, GroupId) of {ok, PerTopicStructs0} -> @@ -592,7 +593,6 @@ show_commits(GroupId, Topic) -> throw_bin("Failed to fetch commited offsets ~p\n", [Reason]) end. -%% @private reset_commits(ID, Topic, Args) -> Retention = parse(Args, "--retention", fun parse_retention/1), ProtocolName = parse(Args, "--protocol", fun(X) -> X end), @@ -610,7 +610,6 @@ reset_commits(ID, Topic, Args) -> ], brod_cg_commits:run(?CLIENT, Group). -%% @private parse_commit_offsets_input("latest") -> latest; parse_commit_offsets_input("earliest") -> earliest; parse_commit_offsets_input(PartitionOffsets) -> @@ -621,7 +620,6 @@ parse_commit_offsets_input(PartitionOffsets) -> end, lists:map(F, Pairs). -%% @private parse_retention("-1") -> -1; parse_retention([_|_] = R) -> case lists:last(R) of @@ -641,23 +639,20 @@ parse_retention([_|_] = R) -> end end. -%% @private print_commits(Struct) -> Topic = kf(topic, Struct), PartRsps = kf(partition_responses, Struct), print([Topic, ":\n"]), print([pp_fmt_struct(1, P) || P <- PartRsps]). -%% @private cg(BootstrapEndpoints, SockOpts, all) -> %% List all groups All = list_groups(BootstrapEndpoints, SockOpts), lists:foreach(fun print_cg_cluster/1, All); cg(BootstrapEndpoints, SockOpts, IDs) -> CgClusters = list_groups(BootstrapEndpoints, SockOpts), - describe_cgs(CgClusters, SockOpts, IDs). + describe_cgs(CgClusters, SockOpts, lists:usort(IDs)). -%% @private describe_cgs(_, _SockOpts, []) -> ok; describe_cgs([], _SockOpts, IDs) -> logerr("Unknown group IDs: ~s", [infix(IDs, ", ")]); @@ -668,9 +663,7 @@ describe_cgs([{Coordinator, CgList} | Rest], SockOpts, IDs) -> IDsRest = IDs -- ThisIDs, describe_cgs(Rest, SockOpts, IDsRest). -%% @private -do_describe_cgs(_Coordinator, _SockOpts, []) -> - ok; +do_describe_cgs(_Coordinator, _SockOpts, []) -> ok; do_describe_cgs(Coordinator, SockOpts, IDs) -> case brod:describe_groups(Coordinator, SockOpts, IDs) of {ok, DescArray} -> @@ -681,7 +674,6 @@ do_describe_cgs(Coordinator, SockOpts, IDs) -> [infix(IDs, ","), fmt_endpoint(Coordinator), Reason]) end. -%% @private print_cg_desc(Desc) -> EC = kf(error_code, Desc), GroupId = kf(group_id, Desc), @@ -689,19 +681,29 @@ print_cg_desc(Desc) -> true -> logerr("Failed to describe group id=~s\nreason:~p\n", [GroupId, EC]); false -> - D1 = lists:keydelete(error_code, 1, Desc), - D = lists:keydelete(group_id, 1, D1), + D1 = lists:keydelete(error_code, 1, ensure_list(Desc)), + D = lists:keydelete(group_id, 1, ensure_list(D1)), print(" ~s\n~s", [GroupId, pp_fmt_struct(_Indent = 2, D)]) end. -%% @private -pp_fmt_struct(_Indent, []) -> []; -pp_fmt_struct(Indent, [{Field, Value} | Rest]) -> - [ indent_fmt(Indent, "~p: ~s", [Field, pp_fmt_struct_value(Indent, Value)]) - | pp_fmt_struct(Indent, Rest) +ensure_list(Struct) when is_map(Struct) -> maps:to_list(Struct); +ensure_list(List) when is_list(List) -> List. + +pp_fmt_struct(Indent, Map) when is_map(Map) -> + pp_fmt_struct(Indent, maps:to_list(Map)); +pp_fmt_struct(Indent, Fields0) when is_list(Fields0) -> + Fields = case Fields0 of + [_] -> Fields0; + _ -> lists:keydelete(no_error, 2, Fields0) + end, + F = fun(IsFirst, {N, V}) -> + indent_fmt(IsFirst, Indent, + "~p: ~s", [N, pp_fmt_struct_value(Indent, V)]) + end, + [ F(true, hd(Fields)) + | lists:map(fun(Fi) -> F(false, Fi) end, tl(Fields)) ]. -%% @private pp_fmt_struct_value(_Indent, X) when is_integer(X) orelse is_atom(X) orelse is_binary(X) orelse @@ -723,17 +725,16 @@ pp_fmt_struct_value(Indent, Array) when is_list(Array) -> [[pp_fmt_prim(V) || V <- Array], "\n"] end. -%% @private pp_fmt_prim([]) -> "[]"; pp_fmt_prim(N) when is_integer(N) -> integer_to_list(N); pp_fmt_prim(A) when is_atom(A) -> atom_to_list(A); pp_fmt_prim(S) when is_binary(S) -> S. -%% @private -indent_fmt(Indent, Fmt, Args) -> +indent_fmt(true, Indent, Fmt, Args) -> + ["- ", indent_fmt(false, Indent - 1, Fmt, Args)]; +indent_fmt(false, Indent, Fmt, Args) -> io_lib:format(lists:duplicate(Indent * 2, $\s) ++ Fmt, Args). -%% @private print_cg_cluster({Endpoint, Cgs}) -> ok = print([fmt_endpoint(Endpoint), "\n"]), IoData = [ io_lib:format(" ~s (~s)\n", [Id, Type]) @@ -741,18 +742,15 @@ print_cg_cluster({Endpoint, Cgs}) -> ], print(IoData). -%% @private fmt_endpoint({Host, Port}) -> bin(io_lib:format("~s:~B", [Host, Port])). -%% @private Return consumer groups clustered by group coordinator +%% Return consumer groups clustered by group coordinator %% {CoordinatorEndpoint, [group_id()]}. -%% @end list_groups(Brokers, SockOpts) -> Cgs = brod:list_all_groups(Brokers, SockOpts), lists:keysort(1, lists:foldl(fun do_list_groups/2, [], Cgs)). -%% @private do_list_groups({_Endpoint, []}, Acc) -> Acc; do_list_groups({Endpoint, {error, Reason}}, Acc) -> logerr("Failed to list groups at kafka ~s\nreason~p", @@ -761,7 +759,6 @@ do_list_groups({Endpoint, {error, Reason}}, Acc) -> do_list_groups({Endpoint, Cgs}, Acc) -> [{Endpoint, Cgs} | Acc]. -%% @private pipe(ReaderPid, SendFun, PendingAcks0) -> PendingAcks1 = flush_pending_acks(PendingAcks0, _Timeout = 0), receive @@ -779,7 +776,6 @@ pipe(ReaderPid, SendFun, PendingAcks0) -> pipe(ReaderPid, SendFun, PendingAcks) end. -%% @private flush_pending_acks(Queue, Timeout) -> case queue:peek(Queue) of empty -> @@ -795,7 +791,6 @@ flush_pending_acks(Queue, Timeout) -> end end. -%% @private fetch_loop(_FmtFun, _FetchFun, _Offset, 0) -> verbose("done (count)\n"), ok; @@ -819,7 +814,7 @@ fetch_loop(FmtFun, FetchFun, Offset, Count) -> fun(M) -> #kafka_message{offset = O, key = K, value = V} = M, R = case is_function(FmtFun, 3) of - true -> FmtFun(O, ensure_kafka_bin(K), ensure_kafka_bin(V)); + true -> FmtFun(O, K, V); false -> FmtFun(M) end, case R of @@ -830,7 +825,6 @@ fetch_loop(FmtFun, FetchFun, Offset, Count) -> fetch_loop(FmtFun, FetchFun, LastOffset + 1, NewCount) end. -%% @private resolve_begin_offset(_Sock, _T, _P, Offset) when is_integer(Offset) -> Offset; resolve_begin_offset(Sock, Topic, Partition, last) -> @@ -844,7 +838,6 @@ resolve_begin_offset(Sock, Topic, Partition, Time) -> {ok, Offset} = brod_utils:resolve_offset(Sock, Topic, Partition, Time), Offset. -%% @private parse_source("stdin") -> standard_io; parse_source("@" ++ Path) -> @@ -855,7 +848,6 @@ parse_source(Path) -> false -> erlang:throw(bin(["bad file ", Path])) end. -%% @private parse_size(Size) -> case lists:reverse(Size) of "K" ++ N -> int(lists:reverse(N)) * (1 bsl 10); @@ -863,7 +855,6 @@ parse_size(Size) -> N -> int(lists:reverse(N)) end. -%% @private format_metadata(Metadata, Format, IsList, IsToListUrp) -> Brokers = kf(brokers, Metadata), Topics0 = kf(topic_metadata, Metadata), @@ -900,26 +891,24 @@ format_metadata(Metadata, Format, IsList, IsToListUrp) -> end end. -%% @private format_broker_lines(Brokers) -> Header = io_lib:format("brokers [~p]:\n", [length(Brokers)]), F = fun(Broker) -> Id = kf(node_id, Broker), Host = kf(host, Broker), Port = kf(port, Broker), - Rack = kf(rack, Broker, ?undef), + Rack = kf(rack, Broker, <<>>), HostStr = fmt_endpoint({Host, Port}), format_broker_line(Id, Rack, HostStr) end, [Header, lists:map(F, Brokers)]. -%% @private -format_broker_line(Id, ?undef, Endpoint) -> +format_broker_line(Id, Rack, Endpoint) + when Rack =:= ?kpro_null orelse Rack =:= <<>> -> io_lib:format(" ~p: ~s\n", [Id, Endpoint]); format_broker_line(Id, Rack, Endpoint) -> io_lib:format(" ~p(~s): ~s\n", [Id, Rack, Endpoint]). -%% @private format_topics_lines(Topics, true) -> Header = io_lib:format("topics [~p]:\n", [length(Topics)]), [Header, lists:map(fun format_topic_list_line/1, Topics)]; @@ -927,14 +916,12 @@ format_topics_lines(Topics, false) -> Header = io_lib:format("topics [~p]:\n", [length(Topics)]), [Header, lists:map(fun format_topic_lines/1, Topics)]. -%% @private format_topic_list_line({Name, Partitions}) when is_list(Partitions) -> io_lib:format(" ~s\n", [Name]); format_topic_list_line({Name, ErrorCode}) -> ErrorStr = format_error_code(ErrorCode), io_lib:format(" ~s: [ERROR] ~s\n", [Name, ErrorStr]). -%% @private format_topic_lines({Name, Partitions}) when is_list(Partitions) -> Header = io_lib:format(" ~s [~p]:\n", [Name, length(Partitions)]), PartitionsText = format_partitions_lines(Partitions), @@ -943,11 +930,9 @@ format_topic_lines({Name, ErrorCode}) -> ErrorStr = format_error_code(ErrorCode), io_lib:format(" ~s: [ERROR] ~s\n", [Name, ErrorStr]). -%% @private format_error_code(E) when is_atom(E) -> atom_to_list(E); format_error_code(E) when is_integer(E) -> integer_to_list(E). -%% @private format_partitions_lines(Partitions0) -> Partitions1 = lists:map(fun({Pnr, Info}) -> @@ -956,7 +941,6 @@ format_partitions_lines(Partitions0) -> Partitions = lists:keysort(1, Partitions1), lists:map(fun format_partition_lines/1, Partitions). -%% @private format_partition_lines({Partition, Info}) -> LeaderNodeId = kf(leader, Info), Status = kf(status, Info), @@ -976,41 +960,29 @@ format_partition_lines({Partition, Info}) -> integer_to_list(LeaderNodeId), ReplicaList, MaybeWarning]). -%% @private format_list(List, Mark) -> infix(lists:map(fun(I) -> [integer_to_list(I), Mark] end, List), ","). -%% @private infix([], _Sep) -> []; infix([_] = L, _Sep) -> L; infix([H | T], Sep) -> [H, Sep, infix(T, Sep)]. -%% @private format_topics(Topics) -> TL = lists:map(fun format_topic/1, Topics), lists:keysort(1, TL). -%% @private format_topic(Topic) -> - ErrorCode = kf(topic_error_code, Topic), TopicName = kf(topic, Topic), PL = kf(partition_metadata, Topic), - Data = - case ?IS_ERROR(ErrorCode) of - true -> ErrorCode; - false -> format_partitions(PL) - end, - {TopicName, Data}. + {TopicName, format_partitions(PL)}. -%% @private format_partitions(Partitions) -> PL = lists:map(fun format_partition/1, Partitions), lists:keysort(1, PL). -%% @private format_partition(P) -> - ErrorCode = kf(partition_error_code, P), - PartitionNr = kf(partition_id, P), + ErrorCode = kf(error_code, P), + PartitionNr = kf(partition, P), LeaderNodeId = kf(leader, P), Replicas = kf(replicas, P), Isr = kf(isr, P), @@ -1021,27 +993,25 @@ format_partition(P) -> ], {integer_to_binary(PartitionNr), Data}. -%% @private Return true if a topics is under-replicated +%% Return true if a topics is under-replicated is_ur_topic(Topic) -> - ErrorCode = kf(topic_error_code, Topic), + ErrorCode = kf(error_code, Topic), Partitions = kf(partition_metadata, Topic), %% when there is an error, we do not know if %% it is under-replicated or not %% retrun true to alert user ?IS_ERROR(ErrorCode) orelse lists:any(fun is_ur_partition/1, Partitions). -%% @private Return true if a partition is under-replicated +%% Return true if a partition is under-replicated is_ur_partition(Partition) -> - ErrorCode = kf(partition_error_code, Partition), + ErrorCode = kf(error_code, Partition), Replicas = kf(replicas, Partition), Isr = kf(isr, Partition), ?IS_ERROR(ErrorCode) orelse lists:sort(Isr) =/= lists:sort(Replicas). -%% @private parse_delimiter("none") -> none; parse_delimiter(EscappedStr) -> eval_str(EscappedStr). -%% @private eval_str([]) -> []; eval_str([$\\, $n | Rest]) -> [$\n | eval_str(Rest)]; @@ -1052,7 +1022,6 @@ eval_str([$\\, $s | Rest]) -> eval_str([C | Rest]) -> [C | eval_str(Rest)]. -%% @private parse_fmt("v", _KvDel, MsgDeli) -> fun(_Offset, _Key, Value) -> [Value, MsgDeli] end; parse_fmt("kv", KvDeli, MsgDeli) -> @@ -1071,11 +1040,8 @@ parse_fmt(FunLiteral0, _KvDeli, _MsgDeli) -> {ok, Tokens, _Line} = erl_scan:string(FunLiteral), {ok, [Expr]} = erl_parse:parse_exprs(Tokens), fun(#kafka_message{offset = Offset, - magic_byte = MagicByte, - attributes = Attributes, key = Key, value = Value, - crc = CRC, ts_type = TsType, ts = Ts }) -> @@ -1085,11 +1051,8 @@ parse_fmt(FunLiteral0, _KvDeli, _MsgDeli) -> erl_eval:add_binding(VarName, VarValue, Acc) end, erl_eval:new_bindings(), [ {'Offset', Offset} - , {'MagicByte', MagicByte} - , {'Attributes', Attributes} , {'Key', Key} , {'Value', Value} - , {'CRC', CRC} , {'TsType', TsType} , {'Ts', Ts} ]), @@ -1103,16 +1066,14 @@ parse_fmt(FunLiteral0, _KvDeli, _MsgDeli) -> end end. -%% @private Append a dot to the function literal. +%% Append a dot to the function literal. ensure_end_with_dot(Str0) -> Str = rstrip(Str0, [$\n, $\t, $\s, $.]), Str ++ ".". -%% @private rstrip(Str, CharSet) -> lists:reverse(lstrip(lists:reverse(Str), CharSet)). -%% @private lstrip([], _) -> []; lstrip([C | Rest] = Str, CharSet) -> case lists:member(C, CharSet) of @@ -1120,7 +1081,6 @@ lstrip([C | Rest] = Str, CharSet) -> false -> Str end. -%% @private parse_partition("random") -> fun(_Topic, PartitionsCount, _Key, _Value) -> {_, _, Micro} = os:timestamp(), @@ -1139,14 +1099,12 @@ parse_partition(I) -> erlang:throw(bin(["Bad partition: ", I])) end. -%% @private parse_acks("all") -> -1; parse_acks("-1") -> -1; parse_acks("0") -> 0; parse_acks("1") -> 1; parse_acks(X) -> erlang:throw(bin(["Bad --acks value: ", X])). -%% @private parse_timeout(Str) -> case lists:reverse(Str) of "s" ++ R -> int(lists:reverse(R)) * 1000; @@ -1154,20 +1112,17 @@ parse_timeout(Str) -> _ -> int(Str) end. -%% @private parse_compression("none") -> no_compression; parse_compression("gzip") -> gzip; parse_compression("snappy") -> snappy; parse_compression(X) -> erlang:throw(bin(["Unknown --compresion value: ", X])). -%% @private parse_offset_time("earliest") -> earliest; parse_offset_time("latest") -> latest; parse_offset_time("last") -> last; parse_offset_time(T) -> int(T). -%% @private -parse_sock_opts(Args) -> +parse_connection_config(Args) -> SslBool = parse(Args, "--ssl", fun parse_boolean/1), CaCertFile = parse(Args, "--cacertfile", fun parse_file/1), CertFile = parse(Args, "--certfile", fun parse_file/1), @@ -1188,23 +1143,19 @@ parse_sock_opts(Args) -> SaslOpts = sasl_opts(SaslOpt), lists:filter(FilterPred, [{ssl, SslOpt} | SaslOpts]). -%% @private sasl_opts(?undef) -> []; sasl_opts(File) -> [{sasl, {plain, File}}]. -%% @private parse_boolean(true) -> true; parse_boolean(false) -> false; parse_boolean("true") -> true; parse_boolean("false") -> false; parse_boolean(?undef) -> ?undef. -%% @private parse_cg_ids("") -> []; parse_cg_ids("all") -> all; parse_cg_ids(Str) -> [bin(I) || I <- string:tokens(Str, ",")]. -%% @private parse_file(?undef) -> ?undef; parse_file(Path) -> @@ -1213,15 +1164,14 @@ parse_file(Path) -> false -> erlang:throw(bin(["bad file ", Path])) end. -%% @private parse(Args, OptName, ParseFun) -> case lists:keyfind(OptName, 1, Args) of {_, Arg} -> try ParseFun(Arg) catch - C : E -> - Stack = erlang:get_stacktrace(), + C : E ?BIND_STACKTRACE(Stack) -> + ?GET_STACKTRACE(Stack), verbose("~p:~p\n~p\n", [C, E, Stack]), Reason = case Arg of @@ -1235,59 +1185,42 @@ parse(Args, OptName, ParseFun) -> erlang:throw(bin(Reason)) end. -%% @private print_version() -> _ = application:load(brod), {_, _, V} = lists:keyfind(brod, 1, application:loaded_applications()), print([V, "\n"]). -%% @private print(IoData) -> io:put_chars(IoData). -%% @private -print(Fmt, Args) -> io:format(Fmt, Args). +print(Fmt, Args) -> io:format(user, Fmt, Args). -%% @private logerr(IoData) -> io:put_chars(standard_error, ["*** ", IoData]). -%% @private logerr(Fmt, Args) -> io:format(standard_error, "*** " ++ Fmt, Args). -%% @private verbose(Str) -> verbose(Str, []). -%% @private verbose(Fmt, Args) -> case erlang:get(brod_cli_log_level) >= ?LOG_LEVEL_VERBOSE of true -> io:format(standard_error, "[verbo]: " ++ Fmt, Args); false -> ok end. -%% @private debug(Fmt, Args) -> case erlang:get(brod_cli_log_level) >= ?LOG_LEVEL_DEBUG of true -> io:format(standard_error, "[debug]: " ++ Fmt, Args); false -> ok end. -%% @private int(Str) -> list_to_integer(trim(Str)). -%% @private trim_h([$\s | T]) -> trim_h(T); trim_h(X) -> X. -%% @private trim(Str) -> trim_h(lists:reverse(trim_h(lists:reverse(Str)))). -%% @private bin(IoData) -> iolist_to_binary(IoData). -%% @private -ensure_kafka_bin(?undef) -> <<>>; -ensure_kafka_bin(Bin) -> Bin. - -%% @private parse_brokers(HostsStr) -> F = fun(HostPortStr) -> Pair = string:tokens(HostPortStr, ":"), @@ -1298,32 +1231,29 @@ parse_brokers(HostsStr) -> end, shuffle(lists:map(F, string:tokens(HostsStr, ","))). -%% @private Parse code paths. +%% Parse code paths. parse_paths(?undef) -> []; parse_paths(Str) -> string:tokens(Str, ","). -%% @private Randomize the order. +%% Randomize the order. shuffle(L) -> RandList = lists:map(fun(_) -> element(3, os:timestamp()) end, L), {_, SortedL} = lists:unzip(lists:keysort(1, lists:zip(RandList, L))), SortedL. -%% @private -spec kf(kpro:field_name(), kpro:struct()) -> kpro:field_value(). kf(FieldName, Struct) -> kpro:find(FieldName, Struct). -%% @private -spec kf(kpro:field_name(), kpro:struct(), kpro:field_value()) -> kpro:field_value(). kf(FieldName, Struct, Default) -> kpro:find(FieldName, Struct, Default). -%% @private start_client(BootstrapEndpoints, ClientConfig) -> {ok, _} = brod_client:start_link(BootstrapEndpoints, ?CLIENT, ClientConfig), ok. -%% @private +-spec throw_bin(string(), [term()]) -> no_return(). throw_bin(Fmt, Args) -> erlang:throw(bin(io_lib:format(Fmt, Args))). diff --git a/src/brod_cli_pipe.erl b/src/brod_cli_pipe.erl index d60e39fe..6b3e2575 100644 --- a/src/brod_cli_pipe.erl +++ b/src/brod_cli_pipe.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2017, Klarna AB +%%% Copyright (c) 2017-2018, Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -23,7 +23,7 @@ %% @end -module(brod_cli_pipe). --ifdef(BROD_CLI). +-ifdef(build_brod_cli). -behaviour(gen_server). @@ -62,7 +62,7 @@ -type delimiter() :: binary(). -type epoch_ms() :: integer(). -type read_fun() :: - fun((?STDIN | file:io_device(), [binary()]) -> + fun((?STDIN | file:io_device(), [binary()]) -> eof | {[{epoch_ms(), Key :: binary(), Val :: binary()}], [binary()]}). -record(state, { parent :: pid() @@ -70,7 +70,7 @@ , read_fun :: read_fun() , is_eof_exit :: boolean() , is_tail :: boolean() - , io_device :: ?STDIN | file:io_device() + , io_device :: ?undef | ?STDIN | file:io_device() , acc_bytes = [] :: [binary()] , retry_delay :: timeout() }). @@ -88,7 +88,6 @@ %% tell brod-cli to start reading from EOF %% no_exit: boolean(). Do not exit when reaching EOF %% blk_size: Read block size -%% @end -spec start_link([{arg_name(), arg_value()}]) -> {ok, pid()}. start_link(Args) -> Parent = self(), @@ -134,10 +133,7 @@ init(#state{source = Source, is_tail = IsTail} = State0) -> ?STDIN; {file, File} -> {ok, Fd} = file:open(File, [read, binary]), - case IsTail of - true -> file:position(Fd, eof); - false -> ok - end, + IsTail andalso file:position(Fd, eof), Fd end, State = State0#state{io_device = IoDevice}, @@ -176,16 +172,12 @@ terminate(_Reason, _State) -> %%%_* Privates ================================================================= -%% @private send_to_parent(Parent, Msgs0) -> FilterF = fun(?TKV(_T, K, V)) -> K =/= <<>> orelse V =/= <<>> end, - case lists:filter(FilterF, Msgs0) of - [] -> ok; - Msgs -> Parent ! {pipe, self(), Msgs} - end, + Msgs = lists:filter(FilterF, Msgs0), + Msgs =/= [] andalso erlang:send(Parent, {pipe, self(), Msgs}), ok. -%% @private handle_read(#state{ read_fun = ReadFun , acc_bytes = Acc0 , io_device = IoDevice @@ -201,7 +193,6 @@ handle_read(#state{ read_fun = ReadFun {noreply, State} end. -%% @private handle_eof(#state{io_device = ?STDIN} = State) -> %% standard_io pipe closed {stop, normal, State}; @@ -219,17 +210,15 @@ handle_eof(#state{io_device = Fd} = State) -> %% we can not assume the file is truncated to empty {stop, pipe_source_truncated, State}; {ok, _Pos} -> - file:position(Fd, LastPos), + _ = file:position(Fd, LastPos), ok = delay_continue(State), {noreply, State} end. -%% @private delay_continue(#state{retry_delay = Delay}) -> _ = erlang:send_after(Delay, self(), ?CONTINUE_MSG), ok. -%% @private -spec make_prompt_line_reader(none | delimiter()) -> read_fun(). make_prompt_line_reader(_KvDeli = none) -> %% Read only value, no key @@ -254,7 +243,6 @@ make_prompt_line_reader(KvDeli) -> Prompt = "KEY" ++ binary_to_list(KvDeli) ++ "VAL> ", make_line_reader(KvDeli, Prompt). -%% @private -spec make_line_reader(none | binary(), string()) -> read_fun(). make_line_reader(KvDeli, Prompt) -> fun(IoDevice, _Acc) -> @@ -276,7 +264,6 @@ make_line_reader(KvDeli, Prompt) -> end end. -%% @private -spec make_stream_reader(none | delimiter(), delimiter(), pos_integer(), boolean()) -> read_fun(). make_stream_reader(KvDeli, MsgDeli, BlkSize, IsEofExit) -> @@ -311,7 +298,6 @@ make_stream_reader(KvDeli, MsgDeli, BlkSize, IsEofExit) -> end end. -%% @private -spec add_acc(pos_integer(), binary(), [binary()]) -> [binary()]. add_acc(_DeliSize = 1, Bytes, Acc) -> %% Delimiter is only one byte, in no way coult it be cut in half @@ -331,7 +317,6 @@ add_acc(DeliSize, Bytes, [Tail | Header]) -> [NewTail, TailH | Header] end. -%% @private -spec split_messages(binary:cp(), [binary()]) -> {[binary()], [binary()]}. split_messages(MsgDeliCp, [Tail | Header]) -> case binary:split(Tail, MsgDeliCp, [global]) of @@ -348,7 +333,6 @@ split_messages(MsgDeliCp, [Tail | Header]) -> end end. -%% @private -spec split_kv_pairs([binary()], none | delimiter(), boolean()) -> brod:kv_list(). split_kv_pairs(Msgs, none, _IsSameDeli) -> @@ -361,17 +345,14 @@ split_kv_pairs(Msgs, KvDeliCp, _IsSameDeli = false) -> make_msg(K, V) end, Msgs). -%% @private make_msgs([]) -> []; make_msgs([K, V | Rest]) -> [make_msg(K, V) | make_msgs(Rest)]. -%% @private make_msg(K, V) -> CreateTs = brod_utils:epoch_ms(), ?TKV(CreateTs, K, V). -%% @private -spec read_line(?STDIN | file:io_device(), string()) -> eof | binary(). read_line(IoDevice, Prompt) -> case io:get_line(IoDevice, Prompt) of @@ -381,12 +362,10 @@ read_line(IoDevice, Prompt) -> unicode:characters_to_binary(rstrip(Chars, "\n")) end. -%% @private -spec rstrip(string(), string()) -> string(). rstrip(Str, CharSet) -> lists:reverse(lstrip(lists:reverse(Str), CharSet)). -%% @private -spec lstrip(string(), string()) -> string(). lstrip([], _) -> []; lstrip([C | Rest] = Str, CharSet) -> @@ -395,7 +374,6 @@ lstrip([C | Rest] = Str, CharSet) -> false -> Str end. -%% @private -spec bin(iodata()) -> binary(). bin(X) -> iolist_to_binary(X). diff --git a/src/brod_client.erl b/src/brod_client.erl index 02af5447..b877adff 100644 --- a/src/brod_client.erl +++ b/src/brod_client.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2015-2017 Klarna AB +%%% Copyright (c) 2015-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -17,8 +17,7 @@ -module(brod_client). -behaviour(gen_server). --export([ get_connection/3 - , get_consumer/3 +-export([ get_consumer/3 , get_group_coordinator/2 , get_leader_connection/3 , get_metadata/2 @@ -53,10 +52,6 @@ -define(DEFAULT_RECONNECT_COOL_DOWN_SECONDS, 1). -define(DEFAULT_GET_METADATA_TIMEOUT_SECONDS, 5). --define(DEFAULT_MAX_METADATA_SOCK_RETRY, 1). - --define(dead_since(TS, REASON), {dead_since, TS, REASON}). --type dead_socket() :: ?dead_since(erlang:timestamp(), any()). %% ClientId as ets table name. -define(ETS(ClientId), ClientId). @@ -80,8 +75,6 @@ -type partition() :: brod:partition(). -type config() :: proplists:proplist(). -type group_id() :: brod:group_id(). --type hostname() :: brod:hostname(). --type portnum() :: brod:portnum(). -type partition_worker_key() :: ?PRODUCER_KEY(topic(), partition()) | ?CONSUMER_KEY(topic(), partition()). @@ -101,18 +94,19 @@ -type get_worker_error() :: get_producer_error() | get_consumer_error(). --record(sock, - { endpoint :: endpoint() - , sock_pid :: ?undef | pid() | dead_socket() - }). - --type sock() :: #sock{}. - +-define(dead_since(TS, REASON), {dead_since, TS, REASON}). +-type connection() :: kpro:connection(). +-type dead_conn() :: ?dead_since(erlang:timestamp(), any()). +-record(conn, + { endpoint :: endpoint() + , pid :: connection() | dead_conn() + }). +-type conn_state() :: #conn{}. -record(state, { client_id :: client_id() , bootstrap_endpoints :: [endpoint()] - , meta_sock_pid :: ?undef | pid() | dead_socket() - , payload_sockets = [] :: [sock()] + , meta_conn :: ?undef | connection() + , payload_conns = [] :: [conn_state()] , producers_sup :: ?undef | pid() , consumers_sup :: ?undef | pid() , config :: ?undef | config() @@ -141,7 +135,6 @@ stop(Client) -> %% @doc Get producer of the given topic-partition. %% The producer is started if auto_start_producers is %% enabled in client config. -%% @end -spec get_producer(client(), topic(), partition()) -> {ok, pid()} | {error, get_producer_error()}. get_producer(Client, Topic, Partition) -> @@ -164,7 +157,6 @@ get_consumer(Client, Topic, Partition) -> %% @doc Dynamically start a per-topic producer. %% Return ok if the producer is already started. -%% @end -spec start_producer(client(), topic(), brod:producer_config()) -> ok | {error, any()}. start_producer(Client, TopicName, ProducerConfig) -> @@ -185,7 +177,6 @@ stop_producer(Client, TopicName) -> %% @doc Dynamically start a topic consumer. %% Returns ok if the consumer is already started. -%% @end. -spec start_consumer(client(), topic(), brod:consumer_config()) -> ok | {error, any()}. start_consumer(Client, TopicName, ConsumerConfig) -> @@ -204,30 +195,22 @@ start_consumer(Client, TopicName, ConsumerConfig) -> stop_consumer(Client, TopicName) -> safe_gen_call(Client, {stop_consumer, TopicName}, infinity). -%% @doc Get the connection to kafka broker which is a leader -%% for given Topic/Partition. -%% If there is no such connection established yet, try to establish a new one. -%% If the connection is already established per request from another -%% producer/consumer the same socket is returned. +%% @doc Get the connection to kafka broker which is a leader for given +%% Topic-Partition. +%% Return already established connection towards the leader borker, +%% Otherwise a new one is established and cached in client state. %% If the old connection was dead less than a configurable N seconds ago, %% {error, LastReason} is returned. -%% @end -spec get_leader_connection(client(), topic(), partition()) -> {ok, pid()} | {error, any()}. get_leader_connection(Client, Topic, Partition) -> - case get_metadata(Client, Topic) of - {ok, Metadata} -> - case brod_utils:find_leader_in_metadata(Metadata, Topic, Partition) of - {ok, {Host, Port}} -> get_connection(Client, Host, Port); - {error, Reason} -> {error, Reason} - end; - {error, Reason} -> - {error, Reason} - end. + safe_gen_call(Client, {get_leader_connection, Topic, Partition}, infinity). %% @doc Get topic metadata, if topic is 'undefined', will fetch ALL metadata. --spec get_metadata(client(), ?undef | topic()) -> +-spec get_metadata(client(), all | ?undef | topic()) -> {ok, kpro:struct()} | {error, any()}. +get_metadata(Client, ?undef) -> + get_metadata(Client, all); get_metadata(Client, Topic) -> safe_gen_call(Client, {get_metadata, Topic}, infinity). @@ -238,22 +221,21 @@ get_partitions_count(Client, Topic) when is_atom(Client) -> %% Ets =:= ClientId get_partitions_count(Client, Client, Topic); get_partitions_count(Client, Topic) when is_pid(Client) -> - %% TODO: remove this clause when brod:start_link_client/_ is removed. case safe_gen_call(Client, get_workers_table, infinity) of {ok, Ets} -> get_partitions_count(Client, Ets, Topic); {error, Reason} -> {error, Reason} end. -%% @doc Get the endpoint of the group coordinator broker. +%% @doc Get broker endpoint and connection config for +%% connecting a group coordinator. -spec get_group_coordinator(client(), group_id()) -> - {ok, {endpoint(), config()}} | {error, any()}. + {ok, {endpoint(), brod:conn_config()}} | {error, any()}. get_group_coordinator(Client, GroupId) -> safe_gen_call(Client, {get_group_coordinator, GroupId}, infinity). %% @doc Register self() as a partition producer. The pid is registered in an ETS %% table, then the callers may lookup a producer pid from the table and make %% produce requests to the producer process directly. -%% @end -spec register_producer(client(), topic(), partition()) -> ok. register_producer(Client, Topic, Partition) -> Producer = self(), @@ -263,24 +245,11 @@ register_producer(Client, Topic, Partition) -> %% @doc Register self() as a partition consumer. The pid is registered in an ETS %% table, then the callers may lookup a consumer pid from the table ane make %% subscribe calls to the process directly. -%% @end register_consumer(Client, Topic, Partition) -> Consumer = self(), Key = ?CONSUMER_KEY(Topic, Partition), gen_server:cast(Client, {register, Key, Consumer}). -%% @doc Get the connection to kafka broker at Host:Port. -%% If there is no such connection established yet, try to establish a new one. -%% If the connection is already established per request from another producer -%% the same socket is returned. -%% If the old connection was dead less than a configurable N seconds ago, -%% {error, LastReason} is returned. -%% @end --spec get_connection(client(), hostname(), portnum()) -> - {ok, pid()} | {error, any()}. -get_connection(Client, Host, Port) -> - safe_gen_call(Client, {get_connection, Host, Port}, infinity). - %%%_* gen_server callbacks ===================================================== init({BootstrapEndpoints, ClientId, Config}) -> @@ -294,19 +263,16 @@ init({BootstrapEndpoints, ClientId, Config}) -> , workers_tab = Tab }}. -handle_info(init, State) -> - ClientId = State#state.client_id, - Endpoints = State#state.bootstrap_endpoints, - {ok, MetaSock, ReorderedEndpoints} = - start_metadata_socket(ClientId, Endpoints, State#state.config), +handle_info(init, State0) -> + Endpoints = State0#state.bootstrap_endpoints, + State1 = ensure_metadata_connection(State0), {ok, ProducersSupPid} = brod_producers_sup:start_link(), {ok, ConsumersSupPid} = brod_consumers_sup:start_link(), - {noreply, State#state{ bootstrap_endpoints = ReorderedEndpoints - , meta_sock_pid = MetaSock - , producers_sup = ProducersSupPid - , consumers_sup = ConsumersSupPid - }}; - + State = State1#state{ bootstrap_endpoints = Endpoints + , producers_sup = ProducersSupPid + , consumers_sup = ConsumersSupPid + }, + {noreply, State}; handle_info({'EXIT', Pid, Reason}, #state{ client_id = ClientId , producers_sup = Pid } = State) -> @@ -319,23 +285,8 @@ handle_info({'EXIT', Pid, Reason}, #state{ client_id = ClientId error_logger:error_msg("client ~p consumers supervisor down~nReason: ~p", [ClientId, Pid, Reason]), {stop, {consumers_sup_down, Reason}, State}; -handle_info({'EXIT', Pid, Reason}, - #state{ client_id = ClientId - , meta_sock_pid = Pid - , bootstrap_endpoints = BootstrapEndpoints0 - } = State) -> - [{Host, Port} | Rest] = BootstrapEndpoints0, - error_logger:info_msg("client ~p metadata socket down ~s:~p~nReason:~p", - [ClientId, Host, Port, Reason]), - %% move the newly failed endpoint to the last in the list - BootstrapEndpoints = Rest ++ [{Host, Port}], - NewState = - State#state{ meta_sock_pid = ?dead_since(os:timestamp(), Reason) - , bootstrap_endpoints = BootstrapEndpoints - }, - {noreply, NewState}; handle_info({'EXIT', Pid, Reason}, State) -> - {ok, NewState} = handle_socket_down(State, Pid, Reason), + NewState = handle_connection_down(State, Pid, Reason), {noreply, NewState}; handle_info(Info, State) -> error_logger:warning_msg("~p [~p] ~p got unexpected info: ~p", @@ -348,12 +299,11 @@ handle_call({stop_producer, Topic}, _From, State) -> handle_call({stop_consumer, Topic}, _From, State) -> ok = brod_consumers_sup:stop_consumer(State#state.consumers_sup, Topic), {reply, ok, State}; +handle_call({get_leader_connection, Topic, Partition}, _From, State) -> + {Result, NewState} = do_get_leader_connection(State, Topic, Partition), + {reply, Result, NewState}; handle_call({get_group_coordinator, GroupId}, _From, State) -> - #state{config = Config} = State, - Timeout = proplists:get_value(get_metadata_timeout_seconds, Config, - ?DEFAULT_GET_METADATA_TIMEOUT_SECONDS), - {Result, NewState} = - do_get_group_coordinator(State, GroupId, timer:seconds(Timeout)), + {Result, NewState} = do_get_group_coordinator(State, GroupId), {reply, Result, NewState}; handle_call({start_producer, TopicName, ProducerConfig}, _From, State) -> {Reply, NewState} = do_start_producer(TopicName, ProducerConfig, State), @@ -363,17 +313,15 @@ handle_call({start_consumer, TopicName, ConsumerConfig}, _From, State) -> {reply, Reply, NewState}; handle_call({auto_start_producer, Topic}, _From, State) -> Config = State#state.config, - case proplists:get_value(auto_start_producers, Config, false) of + case config(auto_start_producers, Config, false) of true -> - ProducerConfig = - proplists:get_value(default_producer_config, Config, []), + ProducerConfig = config(default_producer_config, Config, []), {Reply, NewState} = do_start_producer(Topic, ProducerConfig, State), {reply, Reply, NewState}; false -> {reply, {error, disabled}, State} end; handle_call(get_workers_table, _From, State) -> - %% TODO: remove this clause when brod:start_link_client/_ is removed {reply, {ok, State#state.workers_tab}, State}; handle_call(get_producers_sup_pid, _From, State) -> {reply, {ok, State#state.producers_sup}, State}; @@ -382,9 +330,6 @@ handle_call(get_consumers_sup_pid, _From, State) -> handle_call({get_metadata, Topic}, _From, State) -> {Result, NewState} = do_get_metadata(Topic, State), {reply, Result, NewState}; -handle_call({get_connection, Host, Port}, _From, State) -> - {NewState, Result} = do_get_connection(State, Host, Port), - {reply, Result, NewState}; handle_call(stop, _From, State) -> {stop, normal, ok, State}; handle_call(Call, _From, State) -> @@ -401,11 +346,11 @@ handle_cast(Cast, State) -> code_change(_OldVsn, State, _Extra) -> {ok, State}. -terminate(Reason, #state{ client_id = ClientId - , meta_sock_pid = MetaSock - , payload_sockets = Sockets - , producers_sup = ProducersSup - , consumers_sup = ConsumersSup +terminate(Reason, #state{ client_id = ClientId + , meta_conn = MetaConn + , payload_conns = PayloadConns + , producers_sup = ProducersSup + , consumers_sup = ConsumersSup }) -> case brod_utils:is_normal_reason(Reason) of true -> @@ -414,15 +359,21 @@ terminate(Reason, #state{ client_id = ClientId error_logger:warning_msg("~p [~p] ~p is terminating\nreason: ~p~n", [?MODULE, self(), ClientId, Reason]) end, - %% stop producers and consumers first because they are monitoring socket pids - brod_utils:shutdown_pid(ProducersSup), - brod_utils:shutdown_pid(ConsumersSup), - brod_utils:shutdown_pid(MetaSock), - lists:foreach( - fun(#sock{sock_pid = Pid}) -> brod_utils:shutdown_pid(Pid) end, Sockets). + %% stop producers and consumers first because they are monitoring connections + shutdown_pid(ProducersSup), + shutdown_pid(ConsumersSup), + Shutdown = fun(#conn{pid = Pid}) -> shutdown_pid(Pid) end, + lists:foreach(Shutdown, PayloadConns), + shutdown_pid(MetaConn). %%%_* Internal Functions ======================================================= +shutdown_pid(Pid) when is_pid(Pid) -> + exit(Pid, shutdown), + ok; +shutdown_pid(_) -> + ok. + -spec get_partition_worker(client(), partition_worker_key()) -> {ok, pid()} | {error, get_worker_error()}. get_partition_worker(ClientPid, Key) when is_pid(ClientPid) -> @@ -488,7 +439,6 @@ find_consumer(Client, Topic, Partition) -> {error, Reason} end. -%% @private -spec validate_topic_existence(topic(), state(), boolean()) -> {Result, state()} when Result :: ok | {error, any()}. validate_topic_existence(Topic, #state{workers_tab = Ets} = State, IsRetry) -> @@ -498,7 +448,7 @@ validate_topic_existence(Topic, #state{workers_tab = Ets} = State, IsRetry) -> {error, Reason} -> {{error, Reason}, State}; false when IsRetry -> - {{error, ?EC_UNKNOWN_TOPIC_OR_PARTITION}, State}; + {{error, ?unknown_topic_or_partition}, State}; false -> %% Try fetch metadata (and populate partition count cache) %% Then validate topic existence again. @@ -506,219 +456,222 @@ validate_topic_existence(Topic, #state{workers_tab = Ets} = State, IsRetry) -> fun(_, S) -> validate_topic_existence(Topic, S, true) end) end. -%% @private Continue with {{ok, Result}, NewState} +%% Continue with {{ok, Result}, NewState} %% return whatever error immediately. -%% @end -spec with_ok(Result, fun((_, state()) -> Result)) -> Result when Result :: {ok | {ok, term()} | {error, any()}, state()}. with_ok({ok, State}, Continue) -> Continue(ok, State); with_ok({{ok, OK}, State}, Continue) -> Continue(OK, State); with_ok({{error, _}, #state{}} = Return, _Continue) -> Return. -%% @private If allow_topic_auto_creation is set 'false', +%% If allow_topic_auto_creation is set 'false', %% do not try to fetch metadata per topic name, fetch all topics instead. %% As sending metadata request with topic name will cause an auto creation %% of the topic if auto.create.topics.enable is enabled in broker config. -%% @end -spec get_metadata_safe(topic(), state()) -> {Result, state()} when Result :: {ok, kpro:struct()} | {error, any()}. get_metadata_safe(Topic0, #state{config = Config} = State) -> Topic = - case proplists:get_value(allow_topic_auto_creation, Config, true) of + case config(allow_topic_auto_creation, Config, true) of true -> Topic0; - false -> ?undef + false -> all end, do_get_metadata(Topic, State). -%% @private --spec do_get_metadata(?undef | topic(), state()) -> {Result, state()} +-spec do_get_metadata(all | topic(), state()) -> {Result, state()} when Result :: {ok, kpro:struct()} | {error, any()}. do_get_metadata(Topic, #state{ client_id = ClientId - , config = Config , workers_tab = Ets - , meta_sock_pid = SockPid - } = State) -> + } = State0) -> Topics = case Topic of - ?undef -> []; %% in case no topic is given, get all - _ -> [Topic] + all -> all; %% in case no topic is given, get all + _ -> [Topic] end, - Request = brod_kafka_request:metadata_request(SockPid, Topics), - Timeout = proplists:get_value(get_metadata_timeout_seconds, Config, - ?DEFAULT_GET_METADATA_TIMEOUT_SECONDS), - {Result, NewState} = request_sync(State, Request, timer:seconds(Timeout)), - case Result of - {ok, #kpro_rsp{tag = metadata_response, msg = Metadata}} -> + State = ensure_metadata_connection(State0), + Conn = get_metadata_connection(State), + Request = brod_kafka_request:metadata(Conn, Topics), + case request_sync(State, Request) of + {ok, #kpro_rsp{api = metadata, msg = Metadata}} -> TopicMetadataArray = kf(topic_metadata, Metadata), ok = update_partitions_count_cache(Ets, TopicMetadataArray), - {{ok, Metadata}, NewState}; + {{ok, Metadata}, State}; {error, Reason} -> error_logger:error_msg("~p failed to fetch metadata for topics: ~p\n" "reason=~p", [ClientId, Topics, Reason]), - {Result, NewState} + {{error, Reason}, State} end. -%% @private --spec do_get_group_coordinator(state(), group_id(), timeout()) -> - {Result, state()} when Result :: {ok, endpoint()} | {error, any()}. -do_get_group_coordinator(#state{config = Config} = State, GroupId, Timeout) -> - Req = kpro:req(group_coordinator_request, _Vsn = 0, [{group_id, GroupId}]), - with_ok( - request_sync(State, Req, Timeout), - fun(#kpro_rsp{msg = Msg}, NewState) -> - EC = kf(error_code, Msg), - Result = - case ?IS_ERROR(EC) of - true -> - {error, EC}; %% OBS: {error, EC} is used by group coordinator - false -> - Coordinator = kf(coordinator, Msg), - Host = kf(host, Coordinator), - Port = kf(port, Coordinator), - {ok, {{binary_to_list(Host), Port}, Config}} - end, - {Result, NewState} - end). +%% Ensure there is at least one metadata connection +ensure_metadata_connection(#state{ bootstrap_endpoints = Endpoints + , meta_conn = ?undef + } = State) -> + ConnConfig = conn_config(State), + Pid = case kpro:connect_any(Endpoints, ConnConfig) of + {ok, PidX} -> PidX; + {error, Reason} -> erlang:exit(Reason) + end, + State#state{meta_conn = Pid}; +ensure_metadata_connection(State) -> + State. + +%% must be called after ensure_metadata_connection +get_metadata_connection(#state{meta_conn = Conn}) -> Conn. + +do_get_leader_connection(State0, Topic, Partition) -> + State = ensure_metadata_connection(State0), + MetaConn = get_metadata_connection(State), + Timeout = timeout(State), + case kpro:discover_partition_leader(MetaConn, Topic, Partition, Timeout) of + {ok, Endpoint} -> maybe_connect(State, Endpoint); + {error, Reason} -> {{error, Reason}, State} + end. -%% @private --spec do_get_connection(state(), hostname(), portnum()) -> - {state(), Result} when Result :: {ok, pid()} | {error, any()}. -do_get_connection(#state{} = State, Host, Port) -> - case find_socket(State, Host, Port) of - {ok, Pid} -> - {State, {ok, Pid}}; +-spec do_get_group_coordinator(state(), group_id()) -> + {Result, state()} when Result :: {ok, connection()} | {error, any()}. +do_get_group_coordinator(State0, GroupId) -> + State = ensure_metadata_connection(State0), + MetaConn = get_metadata_connection(State), + Timeout = timeout(State), + case kpro:discover_coordinator(MetaConn, group, GroupId, Timeout) of + {ok, Endpoint} -> + {{ok, {Endpoint, conn_config(State)}}, State}; {error, Reason} -> - maybe_connect(State, Host, Port, Reason) + {{error, Reason}, State} + end. + +timeout(#state{config = Config}) -> + timeout(Config); +timeout(Config) -> + T = config(get_metadata_timeout_seconds, Config, + ?DEFAULT_GET_METADATA_TIMEOUT_SECONDS), + timer:seconds(T). + +config(Key, Config, Default) -> + proplists:get_value(Key, Config, Default). + +conn_config(#state{client_id = ClientId, config = Config}) -> + Cfg = conn_config(Config, kpro_connection:all_cfg_keys(), []), + maps:from_list([{client_id, ensure_binary(ClientId)} | Cfg]). + +conn_config([], _ConnCfgKeys, Acc) -> Acc; +conn_config([{K, V} | Rest], ConnCfgKeys, Acc) -> + NewAcc = + case lists:member(K, ConnCfgKeys) of + true -> [{K, V} | Acc]; + false -> Acc + end, + conn_config(Rest, ConnCfgKeys, NewAcc); +conn_config([K | Rest], ConnCfgKeys, Acc) when is_atom(K) -> + %% translate proplist boolean mark to tuple + conn_config([{K, true} | Rest], ConnCfgKeys, Acc). + +ensure_binary(ClientId) when is_atom(ClientId) -> + ensure_binary(atom_to_binary(ClientId, utf8)); +ensure_binary(ClientId) when is_binary(ClientId) -> + ClientId. + +-spec maybe_connect(state(), endpoint()) -> + {Result, state()} when Result :: {ok, pid()} | {error, any()}. +maybe_connect(#state{} = State, Endpoint) -> + case find_conn(Endpoint, State#state.payload_conns) of + {ok, Pid} -> {{ok, Pid}, State}; + {error, Reason} -> maybe_connect(State, Endpoint, Reason) end. -%% @private --spec maybe_connect(state(), hostname(), portnum(), Reason) -> - {state(), Result} when - Reason :: not_found | dead_socket(), +-spec maybe_connect(state(), endpoint(), Reason) -> + {Result, state()} when + Reason :: not_found | dead_conn(), Result :: {ok, pid()} | {error, any()}. -maybe_connect(State, Host, Port, not_found) -> +maybe_connect(State, Endpoint, not_found) -> %% connect for the first time - connect(State, Host, Port); + connect(State, Endpoint); maybe_connect(#state{client_id = ClientId} = State, - Host, Port, ?dead_since(Ts, Reason)) -> + {Host, Port} = Endpoint, ?dead_since(Ts, Reason)) -> case is_cooled_down(Ts, State) of true -> - connect(State, Host, Port); + connect(State, Endpoint); false -> error_logger:error_msg("~p (re)connect to ~s:~p aborted, " - "last failure reason:~p", + "last failure: ~p", [ClientId, Host, Port, Reason]), - {State, {error, Reason}} + {{error, Reason}, State} end. -%% @private --spec connect(state(), hostname(), portnum()) -> {state(), Result} +-spec connect(state(), endpoint()) -> {Result, state()} when Result :: {ok, pid()} | {error, any()}. -connect(#state{ client_id = ClientId - , payload_sockets = Sockets - , config = Config - } = State, Host, Port) -> - Endpoint = {Host, Port}, - case brod_sock:start_link(self(), Host, Port, ClientId, Config) of - {ok, Pid} -> - S = #sock{ endpoint = Endpoint - , sock_pid = Pid - }, - error_logger:info_msg("client ~p connected to ~s:~p~n", - [ClientId, Host, Port]), - NewSockets = lists:keystore(Endpoint, #sock.endpoint, Sockets, S), - {State#state{payload_sockets = NewSockets}, {ok, Pid}}; - {error, Reason} -> - error_logger:error_msg("client ~p failed to connect to ~s:~p~n" - "reason:~p", - [ClientId, Host, Port, Reason]), - {ok, Sock} = mark_socket_dead(#sock{endpoint = Endpoint}, Reason), - NewSockets = lists:keystore(Endpoint, #sock.endpoint, Sockets, Sock), - {State#state{payload_sockets = NewSockets}, {error, Reason}} - end. +connect(#state{ client_id = ClientId + , payload_conns = Conns + } = State, {Host, Port} = Endpoint) -> + Conn = + case do_connect(Endpoint, State) of + {ok, Pid} -> + error_logger:info_msg("client ~p connected to ~s:~p~n", + [ClientId, Host, Port]), + #conn{ endpoint = Endpoint + , pid = Pid + }; + {error, Reason} -> + error_logger:error_msg("client ~p failed to connect to ~s:~p~n" + "reason:~p", + [ClientId, Host, Port, Reason]), + #conn{ endpoint = Endpoint + , pid = mark_dead(Reason) + } + end, + NewConns = lists:keystore(Endpoint, #conn.endpoint, Conns, Conn), + Result = case Conn#conn.pid of + P when is_pid(P) -> {ok, P}; + ?dead_since(_, R) -> {error, R} + end, + {Result, State#state{payload_conns = NewConns}}. -%% @private Handle socket pid EXIT event, keep the timestamp. -%% But do not restart yet. Connection will be re-established when a -%% per-partition producer restarts and requests for a connection after +do_connect(Endpoint, State) -> + ConnConfig = conn_config(State), + kpro:connect(Endpoint, ConnConfig). + +%% Handle connection pid EXIT event, for payload sockets keep the timestamp, +%% but do not restart yet. Payload connection will be re-established when a +%% per-partition worker restarts and requests for a connection after %% it is cooled down. -%% @end --spec handle_socket_down(state(), pid(), any()) -> {ok, state()}. -handle_socket_down(#state{ client_id = ClientId - , payload_sockets = Sockets - } = State, Pid, Reason) -> - case lists:keyfind(Pid, #sock.sock_pid, Sockets) of - #sock{endpoint = {Host, Port} = Endpoint} = Socket -> - error_logger:info_msg("client ~p: payload socket down ~s:~p~n" - "reason:~p", - [ClientId, Host, Port, Reason]), - {ok, NewSocket} = mark_socket_dead(Socket, Reason), - NewSockets = lists:keystore(Endpoint, #sock.endpoint, Sockets, NewSocket), - {ok, State#state{payload_sockets = NewSockets}}; +-spec handle_connection_down(state(), pid(), any()) -> state(). +handle_connection_down(#state{meta_conn = Pid} = State, Pid, _Reason) -> + State#state{meta_conn = ?undef}; +handle_connection_down(#state{ payload_conns = Conns + , client_id = ClientId + } = State, Pid, Reason) -> + case lists:keytake(Pid, #conn.pid, Conns) of + {value, #conn{endpoint = {Host, Port}} = Conn, Rest} -> + error_logger:info_msg("client ~p: payload connection down ~s:~p~n" + "reason:~p", [ClientId, Host, Port, Reason]), + NewConn = Conn#conn{pid = mark_dead(Reason)}, + State#state{payload_conns = [NewConn | Rest]}; false -> - %% is_pid_alive is checked and reconnect is done for metadata - %% socket in maybe_restart_metadata_socket, hence the 'EXIT' message - %% of old metadata socket pid may end up in this clause, simply ignore - {ok, State} + %% stale EXIT message + State end. -%% @private --spec mark_socket_dead(sock(), any()) -> {ok, sock()}. -mark_socket_dead(Socket, Reason) -> - {ok, Socket#sock{sock_pid = ?dead_since(os:timestamp(), Reason)}}. +mark_dead(Reason) -> ?dead_since(os:timestamp(), Reason). -%% @private --spec find_socket(state(), hostname(), portnum()) -> +-spec find_conn(endpoint(), [conn_state()]) -> {ok, pid()} %% normal case | {error, not_found} %% first call - | {error, dead_socket()}. -find_socket(#state{payload_sockets = Sockets}, Host, Port) -> - case lists:keyfind({Host, Port}, #sock.endpoint, Sockets) of - #sock{sock_pid = Pid} when is_pid(Pid) -> {ok, Pid}; - #sock{sock_pid = ?dead_since(_, _) = NotAlive} -> {error, NotAlive}; - false -> {error, not_found} + | {error, dead_conn()}. +find_conn(Endpoint, Conns) -> + case lists:keyfind(Endpoint, #conn.endpoint, Conns) of + #conn{pid = Pid} when is_pid(Pid) -> {ok, Pid}; + #conn{pid= ?dead_since(_, _) = NotAlive} -> {error, NotAlive}; + false -> {error, not_found} end. -%% @private Check if the socket is down for long enough to retry. +%% Check if the connection is down for long enough to retry. is_cooled_down(Ts, #state{config = Config}) -> - Threshold = proplists:get_value(reconnect_cool_down_seconds, Config, - ?DEFAULT_RECONNECT_COOL_DOWN_SECONDS), + Threshold = config(reconnect_cool_down_seconds, Config, + ?DEFAULT_RECONNECT_COOL_DOWN_SECONDS), Now = os:timestamp(), Diff = timer:now_diff(Now, Ts) div 1000000, Diff >= Threshold. -%% @private Establish a dedicated socket to kafka cluster bootstrap endpoint(s) -%% for metadata retrievals. Failed endpoints are moved to the end of the list -%% so that future retries will start from one that has not tried yet, or the -%% one failed longest ago. -%% -%% NOTE: This socket is not intended for kafka payload. This is to avoid -%% burst of connection usage when many partition producers (re)start -%% at same time, if we always start a new socket to fetch metadata. -%% NOTE: crash in case failed to connect to all of the endpoints. -%% should be restarted by supervisor. -%% @end --spec start_metadata_socket(client_id(), [endpoint()], config()) -> - {ok, pid(), [endpoint()]}. -start_metadata_socket(ClientId, [_|_] = Endpoints, Config) -> - start_metadata_socket(ClientId, Endpoints, Config, - _FailedEndpoints = [], _Reason = ?undef). - -%% @private -start_metadata_socket(_ClientId, [], _Config, FailedEndpoints, Reason) -> - erlang:error({Reason, FailedEndpoints}); -start_metadata_socket(ClientId, [Endpoint | Rest] = Endpoints, Config, - FailedEndpoints, _Reason) -> - {Host, Port} = Endpoint, - case brod_sock:start_link(self(), Host, Port, ClientId, Config) of - {ok, Pid} -> - ReorderedEndpoints = Endpoints ++ lists:reverse(FailedEndpoints), - {ok, Pid, ReorderedEndpoints}; - {error, Reason} -> - start_metadata_socket(ClientId, Rest, Config, - [Endpoint | FailedEndpoints], Reason) - end. - -%% @private -spec update_partitions_count_cache(ets:tab(), [kpro:struct()]) -> ok. update_partitions_count_cache(_Ets, []) -> ok; update_partitions_count_cache(Ets, [TopicMetadata | Rest]) -> @@ -726,16 +679,15 @@ update_partitions_count_cache(Ets, [TopicMetadata | Rest]) -> case do_get_partitions_count(TopicMetadata) of {ok, Cnt} -> ets:insert(Ets, {?TOPIC_METADATA_KEY(Topic), Cnt, os:timestamp()}); - {error, ?EC_UNKNOWN_TOPIC_OR_PARTITION} = Err -> + {error, ?unknown_topic_or_partition} = Err -> ets:insert(Ets, {?TOPIC_METADATA_KEY(Topic), Err, os:timestamp()}); {error, _Reason} -> ok end, update_partitions_count_cache(Ets, Rest). -%% @private Get partition counter from cache. +%% Get partition counter from cache. %% If cache is not hit, send meta data request to retrieve. -%% @end -spec get_partitions_count(client(), ets:tab(), topic()) -> {ok, pos_integer()} | {error, any()}. get_partitions_count(Client, Ets, Topic) -> @@ -755,7 +707,6 @@ get_partitions_count(Client, Ets, Topic) -> end end. -%% @private -spec lookup_partitions_count_cache(ets:tab(), ?undef | topic()) -> {ok, pos_integer()} | {error, any()} | false. lookup_partitions_count_cache(_Ets, ?undef) -> false; @@ -776,18 +727,16 @@ lookup_partitions_count_cache(Ets, Topic) -> {error, client_down} end. -%% @private -spec do_get_partitions_count(kpro:struct()) -> {ok, pos_integer()} | {error, any()}. do_get_partitions_count(TopicMetadata) -> - ErrorCode = kf(topic_error_code, TopicMetadata), + ErrorCode = kf(error_code, TopicMetadata), Partitions = kf(partition_metadata, TopicMetadata), case ?IS_ERROR(ErrorCode) of true -> {error, ErrorCode}; false -> {ok, erlang:length(Partitions)} end. -%% @private -spec maybe_start_producer(client(), topic(), partition(), {error, any()}) -> ok | {error, any()}. @@ -801,59 +750,13 @@ maybe_start_producer(Client, Topic, Partition, Error) -> {error, Reason} end. -%% @private --spec request_sync(state(), kpro:req(), timeout()) -> - {Result, state()} when Result :: {ok, kpro:rsp()} | {error, any()}. -request_sync(State, Request, Timeout) -> - #state{config = Config} = State, - MaxRetry = proplists:get_value(max_metadata_sock_retry, Config, - ?DEFAULT_MAX_METADATA_SOCK_RETRY), - request_sync(State, Request, Timeout, MaxRetry). - -%% @private -request_sync(State0, Request, Timeout, RetryLeft) -> - {ok, State} = maybe_restart_metadata_socket(State0), - SockPid = State#state.meta_sock_pid, - case brod_sock:request_sync(SockPid, Request, Timeout) of - {error, tcp_closed} when RetryLeft > 0 -> - {ok, NewState} = rotate_endpoints(State, tcp_closed), - request_sync(NewState, Request, Timeout, RetryLeft - 1); - {error, {sock_down, _} = Reason} when RetryLeft > 0 -> - {ok, NewState} = rotate_endpoints(State, Reason), - request_sync(NewState, Request, Timeout, RetryLeft - 1); - Result -> - {Result, State} - end. - -%% @private Move the newly failed endpoint to the last in the list. --spec rotate_endpoints(state(), any()) -> {ok, state()}. -rotate_endpoints(State, Reason) -> - #state{ client_id = ClientId - , bootstrap_endpoints = BootstrapEndpoints0 - } = State, - [{Host, Port} | Rest] = BootstrapEndpoints0, - error_logger:error_msg("client ~p metadata socket down ~s:~p~nReason:~p", - [ClientId, Host, Port, Reason]), - BootstrapEndpoints = Rest ++ [{Host, Port}], - {ok, State#state{bootstrap_endpoints = BootstrapEndpoints}}. - -%% @private Maybe restart the metadata socket pid if it is no longer alive. --spec maybe_restart_metadata_socket(state()) -> {ok, state()}. -maybe_restart_metadata_socket(#state{meta_sock_pid = MetaSockPid} = State) -> - case brod_utils:is_pid_alive(MetaSockPid) of - true -> - {ok, State}; - false -> % can happen when metadata connection closed - ClientId = State#state.client_id, - Endpoints = State#state.bootstrap_endpoints, - {ok, NewMetaSockPid, ReorderedEndpoints} = - start_metadata_socket(ClientId, Endpoints, State#state.config), - {ok, State#state{ bootstrap_endpoints = ReorderedEndpoints - , meta_sock_pid = NewMetaSockPid - }} - end. +-spec request_sync(state(), kpro:req()) -> + {ok, kpro:rsp()} | {error, any()}. +request_sync(State, Request) -> + Pid = get_metadata_connection(State), + Timeout = timeout(State), + kpro:request_sync(Pid, Request, Timeout). -%% @private do_start_producer(TopicName, ProducerConfig, State) -> SupPid = State#state.producers_sup, F = fun() -> @@ -862,7 +765,6 @@ do_start_producer(TopicName, ProducerConfig, State) -> end, ensure_partition_workers(TopicName, State, F). -%% @private do_start_consumer(TopicName, ConsumerConfig, State) -> SupPid = State#state.consumers_sup, F = fun() -> @@ -871,7 +773,6 @@ do_start_consumer(TopicName, ConsumerConfig, State) -> end, ensure_partition_workers(TopicName, State, F). -%% @private ensure_partition_workers(TopicName, State, F) -> with_ok( validate_topic_existence(TopicName, State, _IsRetry = false), @@ -886,7 +787,7 @@ ensure_partition_workers(TopicName, State, F) -> end end). -%% @private Catch noproc exit exception when making gen_server:call. +%% Catch noproc exit exception when making gen_server:call. -spec safe_gen_call(pid() | atom(), Call, Timeout) -> Return when Call :: term(), Timeout :: infinity | integer(), @@ -898,7 +799,6 @@ safe_gen_call(Server, Call, Timeout) -> {error, client_down} end. -%% @private -spec kf(kpro:field_name(), kpro:struct()) -> kpro:field_value(). kf(FieldName, Struct) -> kpro:find(FieldName, Struct). diff --git a/src/brod_consumer.erl b/src/brod_consumer.erl index 879e8a2e..b6e21f88 100644 --- a/src/brod_consumer.erl +++ b/src/brod_consumer.erl @@ -1,4 +1,4 @@ -%%% Copyright (c) 2014-2018, Klarna AB +%%% Copyright (c) 2014-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -42,8 +42,8 @@ -export_type([config/0]). -include("brod_int.hrl"). +-include_lib("kafka_protocol/include/kpro_error_codes.hrl"). --type corr_id() :: brod:corr_id(). -type topic() :: brod:topic(). -type partition() :: brod:partition(). -type offset() :: brod:offset(). @@ -68,7 +68,7 @@ -type pending_acks() :: #pending_acks{}. -record(state, { client_pid :: pid() - , socket_pid :: ?undef | pid() + , connection :: ?undef | pid() , topic :: binary() , partition :: integer() , begin_offset :: offset_time() @@ -77,7 +77,7 @@ , max_bytes_orig :: bytes() , sleep_timeout :: integer() , prefetch_count :: integer() - , last_corr_id :: ?undef | corr_id() + , last_req_ref :: ?undef | reference() , subscriber :: ?undef | pid() , subscriber_mref :: ?undef | reference() , pending_acks :: pending_acks() @@ -103,10 +103,10 @@ -define(DEFAULT_PREFETCH_BYTES, 102400). % 100 KB -define(DEFAULT_OFFSET_RESET_POLICY, reset_by_subscriber). -define(ERROR_COOLDOWN, 1000). --define(SOCKET_RETRY_DELAY_MS, 1000). +-define(CONNECTION_RETRY_DELAY_MS, 1000). -define(SEND_FETCH_REQUEST, send_fetch_request). --define(INIT_SOCKET, init_socket). +-define(INIT_CONNECTION, init_connection). -define(DEFAULT_AVG_WINDOW, 5). %%%_* APIs ===================================================================== @@ -228,7 +228,7 @@ init({ClientPid, Topic, Partition, Config}) -> , sleep_timeout = SleepTimeout , prefetch_count = PrefetchCount , prefetch_bytes = PrefetchBytes - , socket_pid = ?undef + , connection = ?undef , pending_acks = #pending_acks{} , is_suspended = false , offset_reset_policy = OffsetResetPolicy @@ -237,9 +237,9 @@ init({ClientPid, Topic, Partition, Config}) -> , size_stat_window = Cfg(size_stat_window, ?DEFAULT_AVG_WINDOW) }}. -handle_info(?INIT_SOCKET, #state{subscriber = Subscriber} = State0) -> +handle_info(?INIT_CONNECTION, #state{subscriber = Subscriber} = State0) -> case brod_utils:is_pid_alive(Subscriber) andalso - maybe_init_socket(State0) of + maybe_init_connection(State0) of false -> %% subscriber not alive {noreply, State0}; @@ -249,7 +249,7 @@ handle_info(?INIT_SOCKET, #state{subscriber = Subscriber} = State0) -> {{error, _Reason}, State} -> %% failed when connecting to partition leader %% retry after a delay - ok = maybe_send_init_socket(State), + ok = maybe_send_init_connection(State), {noreply, State} end; handle_info({msg, _Pid, Rsp}, State) -> @@ -264,9 +264,9 @@ handle_info({'DOWN', _MonitorRef, process, Pid, _Reason}, }), {noreply, NewState}; handle_info({'DOWN', _MonitorRef, process, Pid, _Reason}, - #state{socket_pid = Pid} = State) -> - ok = maybe_send_init_socket(State), - State1 = State#state{socket_pid = ?undef}, + #state{connection = Pid} = State) -> + ok = maybe_send_init_connection(State), + State1 = State#state{connection = ?undef}, {noreply, State1}; handle_info(Info, State) -> error_logger:warning_msg("~p ~p got unexpected info: ~p", @@ -278,7 +278,7 @@ handle_call({subscribe, Pid, Options}, _From, case (not brod_utils:is_pid_alive(Subscriber)) %% old subscriber died orelse Subscriber =:= Pid of %% re-subscribe true -> - case maybe_init_socket(State0) of + case maybe_init_connection(State0) of {ok, State} -> handle_subscribe_call(Pid, Options, State); {{error, Reason}, State} -> @@ -332,56 +332,49 @@ do_debug(Pid, Debug) -> handle_fetch_response(#kpro_rsp{}, #state{subscriber = ?undef} = State0) -> %% discard fetch response when there is no (dead?) subscriber - State = State0#state{last_corr_id = ?undef}, + State = State0#state{last_req_ref = ?undef}, {noreply, State}; -handle_fetch_response(#kpro_rsp{corr_id = CorrId1}, - #state{ last_corr_id = CorrId2 - } = State) when CorrId1 =/= CorrId2 -> +handle_fetch_response(#kpro_rsp{ref = Ref1}, + #state{ last_req_ref = Ref2 + } = State) when Ref1 =/= Ref2 -> %% Not expected response, discard {noreply, State}; -handle_fetch_response(#kpro_rsp{corr_id = CorrId, msg = Rsp}, State0) -> - CorrId = State0#state.last_corr_id, %% assert - State = State0#state{last_corr_id = ?undef}, - [TopicRsp] = kpro:find(responses, Rsp), - Topic = kpro:find(topic, TopicRsp), - [PartitionRsp] = kpro:find(partition_responses, TopicRsp), - Header = kpro:find(partition_header, PartitionRsp), - ErrorCode = kpro:find(error_code, Header), - Partition = kpro:find(partition, Header), - case ?IS_ERROR(ErrorCode) of - true -> +handle_fetch_response(#kpro_rsp{ref = Ref} = Rsp, + #state{ topic = Topic + , partition = Partition + , last_req_ref = Ref + } = State0) -> + State = State0#state{last_req_ref = ?undef}, + case brod_utils:parse_rsp(Rsp) of + {ok, #{ header := Header + , batches := Batches + }} -> + handle_batches(Header, Batches, State); + {error, ErrorCode} -> Error = #kafka_fetch_error{ topic = Topic , partition = Partition , error_code = ErrorCode - , error_desc = kpro_error_code:desc(ErrorCode) }, - handle_fetch_error(Error, State); - false -> - MsgSetBin = kpro:find(record_set, PartitionRsp), - HighWmOffset = kpro:find(high_watermark, Header), - Msgs = brod_utils:decode_messages(State#state.begin_offset, MsgSetBin), - MsgSet = #kafka_message_set{ topic = Topic - , partition = Partition - , high_wm_offset = HighWmOffset - , messages = Msgs - }, - handle_message_set(MsgSet, State) + handle_fetch_error(Error, State) end. -%% @private -handle_message_set(#kafka_message_set{messages = ?incomplete_message(Size)}, - #state{max_bytes = MaxBytes} = State0) -> - %% max_bytes is too small to fetch ONE complete message +handle_batches(?undef, [], #state{begin_offset = LastOffset} = State0) -> + %% A meta-batch. e.g. transactional session ID initialization message + %% fast-forward to the next offset. + %% This clause is not possilbe for fetch requests prior to version 4 + State = State0#state{begin_offset = LastOffset + 1}, + {noreply, State}; +handle_batches(_Header, ?incomplete_batch(Size), + #state{max_bytes = MaxBytes} = State0) -> + %% max_bytes is too small to fetch ONE complete batch true = Size > MaxBytes, %% assert State1 = State0#state{max_bytes = Size}, State = maybe_send_fetch_request(State1), {noreply, State}; -handle_message_set(#kafka_message_set{messages = [], - high_wm_offset = HmOffset - }, - #state{begin_offset = BeginOffset} = State0) -> +handle_batches(Header, [], #state{begin_offset = BeginOffset} = State0) -> + HighWmOffset = kpro:find(high_watermark, Header), State = - case BeginOffset < HmOffset of + case BeginOffset < HighWmOffset of true -> %% There are chances that kafka may return empty message set %% when messages are delete from a compacted topic. @@ -395,10 +388,20 @@ handle_message_set(#kafka_message_set{messages = [], maybe_delay_fetch_request(State0) end, {noreply, State}; -handle_message_set(#kafka_message_set{messages = Messages} = MsgSet, - #state{ subscriber = Subscriber - , pending_acks = PendingAcks - } = State0) -> +handle_batches(Header, Batches, + #state{ subscriber = Subscriber + , pending_acks = PendingAcks + , begin_offset = BeginOffset + , topic = Topic + , partition = Partition + } = State0) -> + HighWmOffset = kpro:find(high_watermark, Header), + Messages = brod_utils:flatten_batches(BeginOffset, Batches), + MsgSet = #kafka_message_set{ topic = Topic + , partition = Partition + , high_wm_offset = HighWmOffset + , messages = Messages + }, ok = cast_to_subscriber(Subscriber, MsgSet), NewPendingAcks = add_pending_acks(PendingAcks, Messages), {value, ?PENDING(LastOffset, _LastMsgSize)} = @@ -419,7 +422,7 @@ add_pending_ack(#kafka_message{offset = Offset, key = Key, value = Value}, , count = Count , bytes = Bytes } = PendingAcks) -> - Size = bytes(Key) + bytes(Value), + Size = size(Key) + size(Value), NewQueue = queue:in(?PENDING(Offset, Size), Queue), PendingAcks#pending_acks{ queue = NewQueue , count = Count + 1 @@ -455,21 +458,17 @@ update_avg_size(#state{ avg_bytes = AvgBytes [#kafka_message{key = Key, value = Value} | Rest]) -> %% kafka adds 34 bytes of overhead (metadata) for each message %% use 40 to give some room for future kafka protocol versions - MsgBytes = bytes(Key) + bytes(Value) + 40, + MsgBytes = size(Key) + size(Value) + 40, %% See https://en.wikipedia.org/wiki/Moving_average NewAvgBytes = ((WindowSize - 1) * AvgBytes + MsgBytes) / WindowSize, update_avg_size(State#state{avg_bytes = NewAvgBytes}, Rest). -bytes(?undef) -> 0; -bytes(B) when is_binary(B) -> size(B). +err_op(?request_timed_out) -> retry; +err_op(?unknown_topic_or_partition) -> stop; +err_op(?invalid_topic_exception) -> stop; +err_op(?offset_out_of_range) -> reset_offset; +err_op(_) -> restart. -err_op(?EC_REQUEST_TIMED_OUT) -> retry; -err_op(?EC_UNKNOWN_TOPIC_OR_PARTITION) -> stop; -err_op(?EC_INVALID_TOPIC_EXCEPTION) -> stop; -err_op(?EC_OFFSET_OUT_OF_RANGE) -> reset_offset; -err_op(_) -> restart. - -%% @private handle_fetch_error(#kafka_fetch_error{error_code = ErrorCode} = Error, #state{ topic = Topic , partition = Partition @@ -490,7 +489,6 @@ handle_fetch_error(#kafka_fetch_error{error_code = ErrorCode} = Error, {stop, {restart, ErrorCode}, State} end. -%% @private handle_reset_offset(#state{ subscriber = Subscriber , offset_reset_policy = reset_by_subscriber } = State, Error) -> @@ -514,7 +512,6 @@ handle_reset_offset(#state{offset_reset_policy = Policy} = State, _Error) -> NewState = maybe_send_fetch_request(State2), {noreply, NewState}. -%% @private handle_ack(#pending_acks{ queue = Queue , bytes = Bytes , count = Count @@ -529,7 +526,6 @@ handle_ack(#pending_acks{ queue = Queue PendingAcks end. -%% @private cast_to_subscriber(Pid, Msg) -> try Pid ! {self(), Msg}, @@ -538,7 +534,6 @@ cast_to_subscriber(Pid, Msg) -> ok end. -%% @private -spec maybe_delay_fetch_request(state()) -> state(). maybe_delay_fetch_request(#state{sleep_timeout = T} = State) when T > 0 -> _ = erlang:send_after(T, self(), ?SEND_FETCH_REQUEST), @@ -550,13 +545,14 @@ maybe_delay_fetch_request(State) -> maybe_send_fetch_request(#state{subscriber = ?undef} = State) -> %% no subscriber State; -maybe_send_fetch_request(#state{socket_pid = ?undef} = State) -> - %% no socket +maybe_send_fetch_request(#state{connection = ?undef} = State) -> + %% no connection State; maybe_send_fetch_request(#state{is_suspended = true} = State) -> %% waiting for subscriber to re-subscribe State; -maybe_send_fetch_request(#state{last_corr_id = I} = State) when is_integer(I) -> +maybe_send_fetch_request(#state{last_req_ref = R} = State) + when is_reference(R) -> %% Waiting for the last request State; maybe_send_fetch_request(#state{ pending_acks = #pending_acks{ count = Count @@ -573,28 +569,27 @@ maybe_send_fetch_request(#state{ pending_acks = #pending_acks{ count = Count -spec send_fetch_request(state()) -> state(). send_fetch_request(#state{ begin_offset = BeginOffset - , socket_pid = SocketPid + , connection = Connection } = State) -> (is_integer(BeginOffset) andalso BeginOffset >= 0) orelse erlang:error({bad_begin_offset, BeginOffset}), Request = - brod_kafka_request:fetch_request(SocketPid, - State#state.topic, - State#state.partition, - State#state.begin_offset, - State#state.max_wait_time, - State#state.min_bytes, - State#state.max_bytes), - case brod_sock:request_async(SocketPid, Request) of - {ok, CorrId} -> - State#state{last_corr_id = CorrId}; - {error, {sock_down, _Reason}} -> - %% ignore error here, the socket pid 'DOWN' message - %% should trigger the socket re-init loop + brod_kafka_request:fetch(Connection, + State#state.topic, + State#state.partition, + State#state.begin_offset, + State#state.max_wait_time, + State#state.min_bytes, + State#state.max_bytes), + case kpro:request_async(Connection, Request) of + ok -> + State#state{last_req_ref = Request#kpro_req.ref}; + {error, {connection_down, _Reason}} -> + %% ignore error here, the connection pid 'DOWN' message + %% should trigger the re-init loop State end. -%% @private handle_subscribe_call(Pid, Options, #state{subscriber_mref = OldMref} = State0) -> case update_options(Options, State0) of @@ -614,7 +609,6 @@ handle_subscribe_call(Pid, Options, {reply, {error, Reason}, State0} end. -%% @private -spec update_options(options(), state()) -> {ok, state()} | {error, any()}. update_options(Options, #state{begin_offset = OldBeginOffset} = State) -> F = fun(Name, Default) -> proplists:get_value(Name, Options, Default) end, @@ -642,14 +636,13 @@ update_options(Options, #state{begin_offset = OldBeginOffset} = State) -> end, resolve_begin_offset(NewState). -%% @private -spec resolve_begin_offset(state()) -> {ok, state()} | {error, any()}. resolve_begin_offset(#state{ begin_offset = BeginOffset - , socket_pid = SocketPid + , connection = Connection , topic = Topic , partition = Partition } = State) when ?IS_SPECIAL_OFFSET(BeginOffset) -> - case resolve_offset(SocketPid, Topic, Partition, BeginOffset) of + case resolve_offset(Connection, Topic, Partition, BeginOffset) of {ok, NewBeginOffset} -> {ok, State#state{begin_offset = NewBeginOffset}}; {error, Reason} -> @@ -658,21 +651,19 @@ resolve_begin_offset(#state{ begin_offset = BeginOffset resolve_begin_offset(State) -> {ok, State}. -%% @private -spec resolve_offset(pid(), topic(), partition(), offset_time()) -> {ok, offset()} | {error, any()}. -resolve_offset(SocketPid, Topic, Partition, BeginOffset) -> +resolve_offset(Connection, Topic, Partition, BeginOffset) -> try - brod_utils:resolve_offset(SocketPid, Topic, Partition, BeginOffset) + brod_utils:resolve_offset(Connection, Topic, Partition, BeginOffset) catch throw : Reason -> {error, Reason} end. -%% @private Reset fetch buffer, use the last unacked offset as the next begin +%% Reset fetch buffer, use the last unacked offset as the next begin %% offset to fetch data from. -%% Discard onwire fetch responses by setting last_corr_id to undefined. -%% @end +%% Discard onwire fetch responses by setting last_req_ref to undefined. -spec reset_buffer(state()) -> state(). reset_buffer(#state{ pending_acks = #pending_acks{queue = Queue} , begin_offset = BeginOffset0 @@ -683,10 +674,10 @@ reset_buffer(#state{ pending_acks = #pending_acks{queue = Queue} end, State#state{ begin_offset = BeginOffset , pending_acks = #pending_acks{} - , last_corr_id = ?undef + , last_req_ref = ?undef }. -%% @private Catch noproc exit exception when making gen_server:call. +%% Catch noproc exit exception when making gen_server:call. -spec safe_gen_call(pid() | atom(), Call, Timeout) -> Return when Call :: term(), Timeout :: infinity | integer(), @@ -698,37 +689,38 @@ safe_gen_call(Server, Call, Timeout) -> {error, consumer_down} end. -%% @private Init payload socket regardless of subscriber state. --spec maybe_init_socket(state()) -> {ok, state()} | {{error, any()}, state()}. -maybe_init_socket(#state{ client_pid = ClientPid +%% Init payload connection regardless of subscriber state. +-spec maybe_init_connection(state()) -> + {ok, state()} | {{error, any()}, state()}. +maybe_init_connection(#state{ client_pid = ClientPid , topic = Topic , partition = Partition - , socket_pid = ?undef + , connection = ?undef } = State0) -> %% Lookup, or maybe (re-)establish a connection to partition leader case brod_client:get_leader_connection(ClientPid, Topic, Partition) of - {ok, SocketPid} -> - _ = erlang:monitor(process, SocketPid), - %% Switching to a new socket - %% the response for last_coor_id will be lost forever - State = State0#state{ last_corr_id = ?undef - , socket_pid = SocketPid + {ok, Connection} -> + _ = erlang:monitor(process, Connection), + %% Switching to a new connection + %% the response for last_req_ref will be lost forever + State = State0#state{ last_req_ref = ?undef + , connection = Connection }, {ok, State}; {error, Reason} -> {{error, Reason}, State0} end; -maybe_init_socket(State) -> +maybe_init_connection(State) -> {ok, State}. -%% @private Send a ?INIT_SOCKET delayed loopback message to re-init socket. --spec maybe_send_init_socket(state()) -> ok. -maybe_send_init_socket(#state{subscriber = Subscriber}) -> - Timeout = ?SOCKET_RETRY_DELAY_MS, - %% re-init payload socket only when subscriber is alive +%% Send a ?INIT_CONNECTION delayed loopback message to re-init. +-spec maybe_send_init_connection(state()) -> ok. +maybe_send_init_connection(#state{subscriber = Subscriber}) -> + Timeout = ?CONNECTION_RETRY_DELAY_MS, + %% re-init payload connection only when subscriber is alive brod_utils:is_pid_alive(Subscriber) andalso - erlang:send_after(Timeout, self(), ?INIT_SOCKET), + erlang:send_after(Timeout, self(), ?INIT_CONNECTION), ok. %%%_* Tests ==================================================================== diff --git a/src/brod_consumers_sup.erl b/src/brod_consumers_sup.erl index 5264a553..6b0363b8 100644 --- a/src/brod_consumers_sup.erl +++ b/src/brod_consumers_sup.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2015-2017 Klarna AB +%%% Copyright (c) 2015-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -16,9 +16,6 @@ %%%============================================================================= %%% @doc brod consumers supervisor -%%% -%%% @copyright 2015 Klarna AB -%%% @end %%%============================================================================= -module(brod_consumers_sup). @@ -46,7 +43,6 @@ %%%_* APIs ===================================================================== %% @doc Start a root consumers supervisor. -%% @end -spec start_link() -> {ok, pid()}. start_link() -> supervisor3:start_link(?MODULE, ?TOPICS_SUP). @@ -65,7 +61,6 @@ stop_consumer(SupPid, TopicName) -> supervisor3:terminate_child(SupPid, TopicName). %% @doc Find a brod_consumer process pid running under ?PARTITIONS_SUP -%% @end -spec find_consumer(pid(), brod:topic(), brod:partition()) -> {ok, pid()} | {error, Reason} when Reason :: {consumer_not_found, brod:topic()} diff --git a/src/brod_group_coordinator.erl b/src/brod_group_coordinator.erl index c8b2b9de..5e508afb 100644 --- a/src/brod_group_coordinator.erl +++ b/src/brod_group_coordinator.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2016-2017 Klarna AB +%%% Copyright (c) 2016-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -46,8 +46,8 @@ -type partition_assignment_strategy() :: brod_partition_assignment_strategy(). %% default configs --define(SESSION_TIMEOUT_SECONDS, 10). --define(HEARTBEAT_RATE_SECONDS, 2). +-define(SESSION_TIMEOUT_SECONDS, 30). +-define(HEARTBEAT_RATE_SECONDS, 5). -define(PROTOCOL_TYPE, <<"consumer">>). -define(MAX_REJOIN_ATTEMPTS, 5). -define(REJOIN_DELAY_SECONDS, 1). @@ -98,20 +98,15 @@ , generationId = 0 :: integer() %% A set of topic names where the group members consumes from , topics = [] :: [brod:topic()] - %% This is the result of group coordinator discovery. - %% It may change when the coordinator is down then a new one - %% is elected among the kafka cluster members. - , coordinator :: ?undef | brod:endpoint() - %% The socket pid to the group coordinator broker. - %% This socket is dedicated for group management and - %% offset commit requests. - %% We can not just get a payload socket from client - %% because the socket might be shared with other group - %% members in the same client, however group members are - %% distinguished by connections to coordinator - , sock_pid :: ?undef | pid() + %% This socket is dedicated for group management requests for + %% join group, sync group, offset commit, and heartbeat. + %% We can not use a payload connection managed by brod_client + %% because connections in brod_client are shared resources, + %% but the connection to group coordinator has to be dedicated + %% to one group member. + , connection :: ?undef | kpro:connection() %% heartbeat reference, to discard stale responses - , hb_ref :: ?undef | {brod:corr_id(), ts()} + , hb_ref :: ?undef | {reference(), ts()} %% all group members received in the join group response , members = [] :: [member()] %% Set to false before joining the group @@ -213,7 +208,6 @@ %% commonly used protocol name for JAVA client. However, brod only supports %% roundrobin protocol out of the box, in order to mimic 'range' protocol %% one will have to do it via `callback_implemented' assignment strategy -%% @end -spec start_link(brod:client(), brod:group_id(), [brod:topic()], config(), module(), pid()) -> {ok, pid()} | {error, any()}. start_link(Client, GroupId, Topics, Config, CbModule, MemberPid) -> @@ -237,7 +231,6 @@ commit_offsets(CoordinatorPid) -> %% NOTE: A lists:usrot is applied on the given extra offsets to commit %% meaning if two or more offsets for the same topic-partition exist %% in the list, only the one that is closer the head of the list is kept -%% @end -spec commit_offsets(pid(), [{{brod:topic(), brod:partition()}, brod:offset()}]) -> ok | {error, any()}. @@ -317,9 +310,10 @@ handle_info(?LO_CMD_STABILIZE(N, Reason), State) -> {ok, NewState} = stabilize(State, N, Reason), {noreply, NewState}; -handle_info({'EXIT', Pid, Reason}, #state{sock_pid = Pid} = State) -> - {ok, NewState} = stabilize(State, 0, {sockent_down, Reason}), - {noreply, NewState}; +handle_info({'EXIT', Pid, Reason}, + #state{connection = Pid} = State) -> + {ok, NewState} = stabilize(State, 0, {connection_down, Reason}), + {noreply, NewState#state{connection = ?undef}}; handle_info({'EXIT', Pid, Reason}, #state{member_pid = Pid} = State) -> case Reason of shutdown -> {stop, shutdown, State}; @@ -336,24 +330,22 @@ handle_info(?LO_CMD_SEND_HB, ?undef -> {ok, NewState} = maybe_send_heartbeat(State), {noreply, NewState}; - {_HbCorrId, SentTime} -> + {_Ref, SentTime} -> Elapsed = timer:now_diff(os:timestamp(), SentTime), - case Elapsed < SessionTimeoutSec * 1000000 of + case Elapsed < SessionTimeoutSec * 1000000 of true -> %% keep waiting for heartbeat response {noreply, State}; false -> - %% Recovery from heartbeat timeout - %% does not work as expected - %% restart socket instead + %% try leave group and re-join when restarted by supervisor {stop, hb_timeout, State} end end; -handle_info({msg, _Pid, #kpro_rsp{ tag = heartbeat_response - , corr_id = HbCorrId - , msg = Body +handle_info({msg, _Pid, #kpro_rsp{ api = heartbeat + , ref = HbRef + , msg = Body }}, - #state{hb_ref = {HbCorrId, _SentTime}} = State0) -> + #state{hb_ref = {HbRef, _SentTime}} = State0) -> EC = kpro:find(error_code, Body), State = State0#state{hb_ref = ?undef}, case ?IS_ERROR(EC) of @@ -385,52 +377,49 @@ handle_cast(_Cast, #state{} = State) -> code_change(_OldVsn, #state{} = State, _Extra) -> {ok, State}. -terminate(Reason, #state{ sock_pid = SockPid +terminate(Reason, #state{ connection = Connection , groupId = GroupId , memberId = MemberId } = State) -> log(State, info, "Leaving group, reason: ~p\n", [Reason]), Body = [{group_id, GroupId}, {member_id, MemberId}], - Request = kpro:req(leave_group_request, _V = 0, Body), + Request = kpro:make_request(leave_group, _V = 0, Body), try - _ = send_sync(SockPid, Request, 1000), + _ = send_sync(Connection, Request, 1000), ok catch _ : _ -> ok - end, - ok = stop_socket(SockPid). + end. %%%_* Internal Functions ======================================================= -spec discover_coordinator(state()) -> {ok, state()}. -discover_coordinator(#state{ client = Client - , coordinator = Coordinator - , sock_pid = SockPid - , groupId = GroupId +discover_coordinator(#state{ client = Client + , connection = Connection0 + , groupId = GroupId } = State) -> - {{Host, Port}, Config} = + {Endpoint, ConnConfig0} = ?ESCALATE(brod_client:get_group_coordinator(Client, GroupId)), - HasConnectionToCoordinator = Coordinator =:= {Host, Port} - andalso brod_utils:is_pid_alive(SockPid), - case HasConnectionToCoordinator of + case is_already_connected(State, Endpoint) of true -> {ok, State}; false -> - %% close old socket - _ = brod_sock:stop(SockPid), + is_pid(Connection0) andalso kpro:close_connection(Connection0), ClientId = make_group_connection_client_id(), - NewSockPid = - ?ESCALATE(brod_sock:start_link(self(), Host, Port, ClientId, Config)), - log(State, info, "connected to group coordinator ~s:~p", - [Host, Port]), - NewState = - State#state{ coordinator = {Host, Port} - , sock_pid = NewSockPid - }, - {ok, NewState} + ConnConfig = ConnConfig0#{client_id => ClientId}, + Connection = ?ESCALATE(kpro:connect(Endpoint, ConnConfig)), + {ok, State#state{connection = Connection}} end. +%% Return true if there is already a connection to the given endpoint. +is_already_connected(#state{connection = Conn}, _) when not is_pid(Conn) -> + false; +is_already_connected(#state{connection = Conn}, {Host, Port}) -> + {Host0, Port0} = ?ESCALATE(kpro_connection:get_endpoint(Conn)), + iolist_to_binary(Host0) =:= iolist_to_binary(Host) andalso + Port0 =:= Port. + -spec stabilize(state(), integer(), any()) -> {ok, state()}. stabilize(#state{ rejoin_delay_seconds = RejoinDelaySeconds , member_module = MemberModule @@ -450,7 +439,7 @@ stabilize(#state{ rejoin_delay_seconds = RejoinDelaySeconds %% because it will fail on the same exception again State1 = case AttemptNo =:= 0 andalso - Reason =/= ?EC_ILLEGAL_GENERATION of + Reason =/= ?illegal_generation of true -> {ok, #state{} = State1_} = try_commit_offsets(State0), State1_; @@ -460,8 +449,7 @@ stabilize(#state{ rejoin_delay_seconds = RejoinDelaySeconds State2 = State1#state{is_in_group = false}, %$ 3. Clean up state based on the last failure reason - State3 = maybe_reset_member_id(State2, Reason), - State = maybe_reset_socket(State3, Reason), + State = maybe_reset_member_id(State2, Reason), %% 4. ensure we have a connection to the (maybe new) group coordinator F1 = fun discover_coordinator/1, @@ -472,7 +460,7 @@ stabilize(#state{ rejoin_delay_seconds = RejoinDelaySeconds RetryFun = fun(StateIn, NewReason) -> - log(StateIn, info, "failed to join group\nreason:~p", [NewReason]), + log(StateIn, info, "failed to join group\nreason: ~p", [NewReason]), _ = case AttemptNo =:= 0 of true -> %% do not delay before the first retry @@ -501,36 +489,26 @@ maybe_reset_member_id(State, Reason) -> false -> State end. -should_reset_member_id(?EC_UNKNOWN_MEMBER_ID) -> +should_reset_member_id(?unknown_member_id) -> %% we are likely kicked out from the group %% rejoin with empty member id true; -should_reset_member_id(?EC_NOT_COORDINATOR_FOR_GROUP) -> +should_reset_member_id(?not_coordinator) -> %% the coordinator have moved to another broker - %% set it to ?undef to trigger a socket restart + %% set it to ?undef to trigger a re-discover true; -should_reset_member_id({socket_down, _Reason}) -> - %% old socket was down, new connection will lead +should_reset_member_id({connection_down, _Reason}) -> + %% old connection was down, new connection will lead %% to a new member id true; should_reset_member_id(_) -> false. -maybe_reset_socket(State, ?EC_NOT_COORDINATOR_FOR_GROUP) -> - ok = stop_socket(State#state.sock_pid), - State#state{sock_pid = ?undef}; -maybe_reset_socket(State, _OtherReason) -> - State. - -stop_socket(SockPid) -> - catch unlink(SockPid), - ok = brod_sock:stop(SockPid). - -spec join_group(state()) -> {ok, state()}. join_group(#state{ groupId = GroupId , memberId = MemberId0 , topics = Topics - , sock_pid = SockPid + , connection = Connection , session_timeout_seconds = SessionTimeoutSec , protocol_name = ProtocolName } = State0) -> @@ -551,16 +529,10 @@ join_group(#state{ groupId = GroupId , {protocol_type, ?PROTOCOL_TYPE} , {group_protocols, [Protocol]} ], - %% TODO: support version 1 - Vsn = 0, - Req = kpro:req(join_group_request, Vsn, Body), + Req = brod_kafka_request:join_group(Connection, Body), %% send join group request and wait for response %% as long as the session timeout config - #kpro_rsp{ tag = join_group_response - , vsn = Vsn - , msg = RspBody - } = send_sync(SockPid, Req, SessionTimeout), - ?ESCALATE_EC(kpro:find(error_code, RspBody)), + RspBody = send_sync(Connection, Req, SessionTimeout), GenerationId = kpro:find(generation_id, RspBody), LeaderId = kpro:find(leader_id, RspBody), MemberId = kpro:find(member_id, RspBody), @@ -575,12 +547,11 @@ join_group(#state{ groupId = GroupId log(State, info, "elected=~p", [IsGroupLeader]), {ok, State}. -%% @private -spec sync_group(state()) -> {ok, state()}. sync_group(#state{ groupId = GroupId , generationId = GenerationId , memberId = MemberId - , sock_pid = SockPid + , connection = Connection , member_pid = MemberPid , member_module = MemberModule } = State) -> @@ -590,14 +561,9 @@ sync_group(#state{ groupId = GroupId , {member_id, MemberId} , {group_assignment, assign_partitions(State)} ], - Vsn = 0, - SyncReq = kpro:req(sync_group_request, Vsn, ReqBody), + SyncReq = brod_kafka_request:sync_group(Connection, ReqBody), %% send sync group request and wait for response - #kpro_rsp{ tag = sync_group_response - , vsn = Vsn - , msg = RspBody - } = send_sync(SockPid, SyncReq), - ?ESCALATE_EC(kpro:find(error_code, RspBody)), + RspBody = send_sync(Connection, SyncReq), %% get my partition assignments Assignment = kpro:find(member_assignment, RspBody), TopicAssignments = get_topic_assignments(State, Assignment), @@ -608,7 +574,6 @@ sync_group(#state{ groupId = GroupId [format_assignments(TopicAssignments)]), start_offset_commit_timer(NewState). -%% @private -spec handle_ack(state(), brod:topic(), brod:partition(), brod:offset()) -> {ok, state()}. handle_ack(#state{ acked_offsets = AckedOffsets @@ -617,7 +582,7 @@ handle_ack(#state{ acked_offsets = AckedOffsets merge_acked_offsets(AckedOffsets, [{{Topic, Partition}, Offset}]), {ok, State#state{acked_offsets = NewAckedOffsets}}. -%% @private Add new offsets to be acked into the acked offsets collection. +%% Add new offsets to be acked into the acked offsets collection. -spec merge_acked_offsets(Offsets, Offsets) -> Offsets when Offsets :: [{{brod:topic(), brod:partition()}, brod:offset()}]. merge_acked_offsets(AckedOffsets, OffsetsToAck) -> @@ -648,12 +613,11 @@ format_partition_assignments([{Partition, BeginOffset} | Rest]) -> , format_partition_assignments(Rest) ]. -%% @private Commit the current offsets before re-join the group. +%% Commit the current offsets before re-join the group. %% NOTE: this is a 'best-effort' attempt, failing to commit offset %% at this stage should be fine, after all, the consumers will %% refresh their start point offsets when new assignment is %% received. -%% @end -spec try_commit_offsets(state()) -> {ok, state()}. try_commit_offsets(#state{} = State) -> try @@ -662,7 +626,7 @@ try_commit_offsets(#state{} = State) -> {ok, State} end. -%% @private Commit collected offsets, stop old commit timer, start new timer. +%% Commit collected offsets, stop old commit timer, start new timer. -spec do_commit_offsets(state()) -> {ok, state()}. do_commit_offsets(State) -> {ok, NewState} = do_commit_offsets_(State), @@ -676,7 +640,7 @@ do_commit_offsets_(#state{offset_commit_policy = consumer_managed} = State) -> do_commit_offsets_(#state{ groupId = GroupId , memberId = MemberId , generationId = GenerationId - , sock_pid = SockPid + , connection = Connection , offset_retention_seconds = OffsetRetentionSecs , acked_offsets = AckedOffsets } = State) -> @@ -699,31 +663,26 @@ do_commit_offsets_(#state{ groupId = GroupId ] end, TopicOffsets0), Retention = - case OffsetRetentionSecs =/= ?undef of - true -> timer:seconds(OffsetRetentionSecs); - false -> ?OFFSET_RETENTION_DEFAULT + case is_default_offset_retention(OffsetRetentionSecs) of + true -> ?OFFSET_RETENTION_DEFAULT; + false -> timer:seconds(OffsetRetentionSecs) end, ReqBody = [ {group_id, GroupId} - , {group_generation_id, GenerationId} + , {generation_id, GenerationId} , {member_id, MemberId} , {retention_time, Retention} , {topics, TopicOffsets} ], - Vsn = 2, %% supports only version 2 (since kafka 0.9) - Req = kpro:req(offset_commit_request, Vsn, ReqBody), - #kpro_rsp{ tag = offset_commit_response - , vsn = Vsn - , msg = RspBody - } = send_sync(SockPid, Req), + Req = brod_kafka_request:offset_commit(Connection, ReqBody), + RspBody = send_sync(Connection, Req), Topics = kpro:find(responses, RspBody), ok = assert_commit_response(Topics), NewState = State#state{acked_offsets = []}, {ok, NewState}. -%% @private Check commit response. If no error returns ok, +%% Check commit response. If no error returns ok, %% if all error codes are the same, raise throw, otherwise error. -%% %% @end -spec assert_commit_response([kpro:struct()]) -> ok | no_return(). assert_commit_response(Topics) -> ErrorSet = collect_commit_response_error_codes(Topics), @@ -733,7 +692,6 @@ assert_commit_response(Topics) -> _ -> erlang:error({commit_offset_failed, Topics}) end. -%% @private -spec collect_commit_response_error_codes([kpro:struct()]) -> gb_sets:set(). collect_commit_response_error_codes(Topics) -> lists:foldl( @@ -749,7 +707,6 @@ collect_commit_response_error_codes(Topics) -> end, Acc1, Partitions) end, gb_sets:new(), Topics). -%% @private -spec assign_partitions(state()) -> [kpro:struct()]. assign_partitions(State) when ?IS_LEADER(State) -> #state{ client = Client @@ -791,7 +748,6 @@ assign_partitions(#state{}) -> %% only leader can assign partitions to members []. -%% @private -spec translate_members([kpro:struct()]) -> [member()]. translate_members(Members) -> lists:map( @@ -822,7 +778,6 @@ get_partitions(Client, Topic) -> Count = ?ESCALATE(brod_client:get_partitions_count(Client, Topic)), lists:seq(0, Count - 1). -%% @private -spec do_assign_partitions(roundrobin_v2, [member()], [{brod:topic(), brod:partition()}]) -> [{member_id(), [brod:partition_assignment()]}]. @@ -840,7 +795,6 @@ do_assign_partitions(roundrobin_v2, Members, AllPartitions) -> <- roundrobin_assign_loop(AllPartitions, MemberAssignment, []) ]. -%% @private roundrobin_assign_loop([], PendingMembers, AssignedMembers) -> lists:reverse(AssignedMembers) ++ PendingMembers; roundrobin_assign_loop(Partitions, [], AssignedMembers) -> @@ -861,9 +815,8 @@ roundrobin_assign_loop([{Topic, Partition} | Rest] = TopicPartitions, [Member0 | AssignedMembers]) end. -%% @private Extract the partition assignemts from SyncGroupResponse +%% Extract the partition assignemts from SyncGroupResponse %% then fetch the committed offsets of each partition. -%% @end -spec get_topic_assignments(state(), binary() | [kpro:struct()]) -> brod:received_assignments(). get_topic_assignments(#state{}, ?kpro_cg_no_assignment) -> []; %% no assignments @@ -881,9 +834,8 @@ get_topic_assignments(#state{} = State, Assignment) -> IsConsumerManaged = State#state.offset_commit_policy =:= consumer_managed, resolve_begin_offsets(TopicPartitions, CommittedOffsets, IsConsumerManaged). -%% @private Fetch committed offsets from kafka, +%% Fetch committed offsets from kafka, %% or call the consumer callback to read committed offsets. -%% @end -spec get_committed_offsets(state(), [{brod:topic(), brod:partition()}]) -> [{{brod:topic(), brod:partition()}, brod:offset()}]. get_committed_offsets(#state{ offset_commit_policy = consumer_managed @@ -894,16 +846,13 @@ get_committed_offsets(#state{ offset_commit_policy = consumer_managed R; get_committed_offsets(#state{ offset_commit_policy = commit_to_kafka_v2 , groupId = GroupId - , sock_pid = SockPid + , connection = Conn }, TopicPartitions) -> GrouppedPartitions = brod_utils:group_per_key(TopicPartitions), - Req = brod_kafka_request:offset_fetch_request(SockPid, GroupId, - GrouppedPartitions), - #kpro_rsp{ tag = offset_fetch_response - , msg = RspBody - } = send_sync(SockPid, Req), + Req = brod_kafka_request:offset_fetch(Conn, GroupId, GrouppedPartitions), + RspBody = send_sync(Conn, Req), %% error_code is introduced in version 2 - ?ESCALATE_EC(kpro:find(error_code, RspBody, ?EC_NONE)), + ?ESCALATE_EC(kpro:find(error_code, RspBody, ?no_error)), TopicOffsets = kpro:find(responses, RspBody), CommittedOffsets0 = lists:map( @@ -929,7 +878,6 @@ get_committed_offsets(#state{ offset_commit_policy = commit_to_kafka_v2 end, TopicOffsets), lists:append(CommittedOffsets0). -%% @private -spec resolve_begin_offsets([TP], [{TP, brod:offset()}], boolean()) -> brod:received_assignments() when TP :: {brod:topic(), brod:partition()}. @@ -961,19 +909,17 @@ resolve_begin_offsets([{Topic, Partition} | Rest], CommittedOffsets, | resolve_begin_offsets(Rest, CommittedOffsets, IsConsumerManaged) ]. -%% @private Start a timer to send a loopback command to self() to trigger +%% Start a timer to send a loopback command to self() to trigger %% a heartbeat request to the group coordinator. %% NOTE: the heartbeat requests are sent only when it is in group, %% but the timer is always restarted after expiration. -%% @end -spec start_heartbeat_timer(pos_integer()) -> ok. start_heartbeat_timer(HbRateSec) -> erlang:send_after(timer:seconds(HbRateSec), self(), ?LO_CMD_SEND_HB), ok. -%% @private Start a timer to send a loopback command to self() to trigger +%% Start a timer to send a loopback command to self() to trigger %% a offset commit request to group coordinator broker. -%% @end -spec start_offset_commit_timer(state()) -> {ok, state()}. start_offset_commit_timer(#state{offset_commit_timer = OldTimer} = State) -> #state{ offset_commit_policy = Policy @@ -996,32 +942,32 @@ start_offset_commit_timer(#state{offset_commit_timer = OldTimer} = State) -> {ok, State#state{offset_commit_timer = Timer}} end. -%% @private Send heartbeat request if it has joined the group. +%% Send heartbeat request if it has joined the group. -spec maybe_send_heartbeat(state()) -> {ok, state()}. maybe_send_heartbeat(#state{ is_in_group = true , groupId = GroupId , memberId = MemberId , generationId = GenerationId - , sock_pid = SockPid + , connection = Connection } = State) -> ReqBody = [ {group_id, GroupId} - , {group_generation_id, GenerationId} + , {generation_id, GenerationId} , {member_id, MemberId} ], - Req = kpro:req(heartbeat_request, 0, ReqBody), - {ok, CorrId} = brod_sock:request_async(SockPid, Req), - NewState = State#state{hb_ref = {CorrId, os:timestamp()}}, + Req = kpro:make_request(heartbeat, 0, ReqBody), + ok = kpro:request_async(Connection, Req), + NewState = State#state{hb_ref = {Req#kpro_req.ref, os:timestamp()}}, {ok, NewState}; maybe_send_heartbeat(#state{} = State) -> %% do not send heartbeat when not in group {ok, State#state{hb_ref = ?undef}}. -send_sync(SockPid, Request) -> - send_sync(SockPid, Request, 5000). +send_sync(Connection, Request) -> + send_sync(Connection, Request, 5000). -send_sync(SockPid, Request, Timeout) -> - ?ESCALATE(brod_sock:request_sync(SockPid, Request, Timeout)). +send_sync(Connection, Request, Timeout) -> + ?ESCALATE(brod_utils:request_sync(Connection, Request, Timeout)). log(#state{ groupId = GroupId , generationId = GenerationId @@ -1032,14 +978,14 @@ log(#state{ groupId = GroupId "Group member (~s,coor=~p,cb=~p,generation=~p):\n" ++ Fmt, [GroupId, self(), MemberPid, GenerationId | Args]). -%% @private Make metata to be committed together with offsets. +%% Make metata to be committed together with offsets. -spec make_offset_commit_metadata() -> binary(). make_offset_commit_metadata() -> %% Use a '+1/' prefix as a commit from group member which supports %% roundrobin_v2 protocol bin(["+1/", coordinator_id()]). -%% @private Make group member's user data in join_group_request +%% Make group member's user data in join_group_request %% %% user_data can be used to share state between group members. %% It is originally sent by group members in join_group_request:s, @@ -1050,36 +996,32 @@ make_offset_commit_metadata() -> %% module, when needed for advanced features, we can originate it from %% member's init callback, and pass it to members via %% `brod_received_assignments()' -%% @end -spec user_data(join | assign) -> binary(). user_data(Action) -> term_to_binary( [ coordinator_info(Action) ]). -%% @private -spec coordinator_info(join | assign) -> {atom(), pid()}. coordinator_info(join) -> {member_coordinator, self()}; coordinator_info(assign) -> {leader_coordinator, self()}. -%% @private Make a client_id() to be used in the requests sent over the group -%% coordinator's socket (group coordinator on the other end), this id will be -%% displayed when describing the group status with admin client/script. -%% e.g. brod@localhost/<0.45.0>_/172.18.0.1 -%% @end +%% Make a client ID to be used in the requests sent over the group +%% coordinator's connection (group coordinator broker on the other end), +%% this id will be displayed when describing the group status with admin +%% client/script. e.g. brod@localhost/<0.45.0>_/172.18.0.1 -spec make_group_connection_client_id() -> binary(). make_group_connection_client_id() -> coordinator_id(). -%% @private Use 'node()/pid()' as unique identifier of each group coordinator. +%% Use 'node()/pid()' as unique identifier of each group coordinator. -spec coordinator_id() -> binary(). coordinator_id() -> bin(io_lib:format("~p/~p", [node(), self()])). -%% @private -spec bin(iodata()) -> binary(). bin(X) -> iolist_to_binary(X). -%% @private Before roundrobin_v2, brod had two versions of commit metadata: +%% Before roundrobin_v2, brod had two versions of commit metadata: %% 1. "ts() node() pid()" %% e.g. "2017-10-24:18:20:55.475670 'nodename@host-name' <0.18980.6>" %% 2. "node()/pid()" @@ -1089,7 +1031,6 @@ bin(X) -> iolist_to_binary(X). %% e.g. "+1/'nodename@host-name'/<0.18980.6>" %% Here we try to recognize brod commits using a regexp, %% then check the +1 prefix to exclude roundrobin_v2. -%% @end -spec is_roundrobin_v1_commit(?kpro_null | binary()) -> boolean(). is_roundrobin_v1_commit(?kpro_null) -> false; is_roundrobin_v1_commit(<<"+1/", _/binary>>) -> false; @@ -1099,9 +1040,8 @@ is_roundrobin_v1_commit(Metadata) -> {match, _} -> true end. -%% @private Upgrade offset from old roundrobin protocol to new. +%% Upgrade offset from old roundrobin protocol to new. %% old (before roundrobin_v2) brod commits acked offsets not begin_offset -%% @end -spec maybe_upgrade_from_roundrobin_v1(brod:offset(), binary()) -> brod:offset(). maybe_upgrade_from_roundrobin_v1(Offset, Metadata) -> @@ -1110,6 +1050,11 @@ maybe_upgrade_from_roundrobin_v1(Offset, Metadata) -> false -> Offset end. +%% Return true if it should be default retention to be used in offset commit +is_default_offset_retention(-1) -> true; +is_default_offset_retention(?undef) -> true; +is_default_offset_retention(_) -> false. + -ifdef(TEST). -include_lib("eunit/include/eunit.hrl"). diff --git a/src/brod_group_member.erl b/src/brod_group_member.erl index e2f8d65c..2aad8819 100644 --- a/src/brod_group_member.erl +++ b/src/brod_group_member.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2016-2017 Klarna AB +%%% Copyright (c) 2016-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. diff --git a/src/brod_group_subscriber.erl b/src/brod_group_subscriber.erl index 92a8d06a..96397ca5 100644 --- a/src/brod_group_subscriber.erl +++ b/src/brod_group_subscriber.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2016-2017 Klarna AB +%%% Copyright (c) 2016-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -440,7 +440,6 @@ terminate(_Reason, #state{}) -> %%%_* Internal Functions ======================================================= -%% @private -spec start_subscribe_timer(?undef | reference(), timeout()) -> reference(). start_subscribe_timer(?undef, Delay) -> erlang:send_after(Delay, self(), ?LO_CMD_SUBSCRIBE_PARTITIONS); @@ -520,18 +519,17 @@ handle_ack(AckRef, #state{ generationId = GenerationId State end. -%% @private Tell consumer process to fetch more (if pre-fetch count allows). +%% Tell consumer process to fetch more (if pre-fetch count allows). consume_ack(Pid, Offset) when is_pid(Pid) -> ok = brod:consume_ack(Pid, Offset); consume_ack(_Down, _Offset) -> %% consumer is down, should be restarted by its supervisor ok. -%% @private Send an async message to group coordinator for offset commit. +%% Send an async message to group coordinator for offset commit. commit_ack(Pid, GenerationId, Topic, Partition, Offset) -> ok = brod_group_coordinator:ack(Pid, GenerationId, Topic, Partition, Offset). -%% @private subscribe_partitions(#state{ client = Client , consumers = Consumers0 } = State) -> @@ -539,7 +537,6 @@ subscribe_partitions(#state{ client = Client lists:map(fun(C) -> subscribe_partition(Client, C) end, Consumers0), {ok, State#state{consumers = Consumers}}. -%% @private subscribe_partition(Client, Consumer) -> #consumer{ topic_partition = {Topic, Partition} , consumer_pid = Pid diff --git a/src/brod_kafka_apis.erl b/src/brod_kafka_apis.erl index 461dcb26..55dc131b 100644 --- a/src/brod_kafka_apis.erl +++ b/src/brod_kafka_apis.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2017 Klarna AB +%%% Copyright (c) 2017-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -14,16 +14,14 @@ %%% limitations under the License. %%% -%% Version ranges are cached per host and per brod_sock pid in ets +%% Version ranges are cached per host and per connection pid in ets -module(brod_kafka_apis). -export([ default_version/1 - , maybe_add_sock_pid/2 , pick_version/2 , start_link/0 , stop/0 - , versions_received/4 ]). -export([ code_change/3 @@ -43,12 +41,10 @@ -record(state, {}). --type vsn() :: non_neg_integer(). +-type vsn() :: kpro:vsn(). -type range() :: {vsn(), vsn()}. --type api() :: kpro:req_tag(). --type client_id() :: binary(). %% not brod:client_id() --type host() :: brod:hostname(). --type versions() :: [{api(), range()}]. +-type api() :: kpro:api(). +-type conn() :: kpro:connection(). %% @doc Start process. -spec start_link() -> {ok, pid()}. @@ -59,46 +55,33 @@ start_link() -> stop() -> gen_server:call(?SERVER, stop, infinity). -%% @doc Report API version ranges for a given `brod_sock' pid. --spec versions_received(client_id(), pid(), versions(), host()) -> ok. -versions_received(ClientId, SockPid, Versions, Host) -> - Vsns = resolve_version_ranges(ClientId, Versions, []), - gen_server:call(?SERVER, {versions_received, SockPid, Vsns, Host}, infinity). - %% @doc Get default supported version for the given API. -spec default_version(api()) -> vsn(). default_version(API) -> {Min, _Max} = supported_versions(API), Min. -%% @doc Try add pid with existing version ranges. -%% Return `{error, unknow_host}' if the host is not cached already. -%% @end --spec maybe_add_sock_pid(host(), pid()) -> ok | {error, unknown_host}. -maybe_add_sock_pid(Host, SockPid) -> - case ets:lookup(?ETS, Host) of - [] -> - {error, unknown_host}; - [{_, ResolvedVersions}] -> - gen_server:call(?SERVER, {add_sock_pid, SockPid, ResolvedVersions}) - end. - %% @doc Pick API version for the given API. --spec pick_version(pid(), api()) -> vsn(). -pick_version(SockPid, API) -> - do_pick_version(SockPid, API, supported_versions(API)). +-spec pick_version(conn(), api()) -> vsn(). +pick_version(Conn, API) -> + do_pick_version(Conn, API, supported_versions(API)). + +%%%_* gen_server callbacks ===================================================== init([]) -> - ?ETS = ets:new(?ETS, [named_table, protected]), + ?ETS = ets:new(?ETS, [named_table, public]), {ok, #state{}}. -handle_info({'DOWN', _Mref, process, Pid, _Reason}, State) -> - ets:delete(?ETS, Pid), +handle_info({'DOWN', _Mref, process, Conn, _Reason}, State) -> + _ = ets:delete(?ETS, Conn), {noreply, State}; handle_info(Info, State) -> error_logger:error_msg("unknown info ~p", [Info]), {noreply, State}. +handle_cast({monitor_connection, Conn}, State) -> + erlang:monitor(process, Conn), + {noreply, State}; handle_cast(Cast, State) -> error_logger:error_msg("unknown cast ~p", [Cast]), {noreply, State}. @@ -106,15 +89,6 @@ handle_cast(Cast, State) -> handle_call(stop, From, State) -> gen_server:reply(From, ok), {stop, normal, State}; -handle_call({add_sock_pid, SockPid, ResolvedVersions}, _From, State) -> - _ = erlang:monitor(process, SockPid), - ets:insert(?ETS, {SockPid, ResolvedVersions}), - {reply, ok, State}; -handle_call({versions_received, SockPid, Versions, Host}, _From, State) -> - _ = erlang:monitor(process, SockPid), - ets:insert(?ETS, {Host, Versions}), - ets:insert(?ETS, {SockPid, Versions}), - {reply, ok, State}; handle_call(Call, _From, State) -> {reply, {error, {unknown_call, Call}}, State}. @@ -124,104 +98,63 @@ code_change(_OldVsn, State, _Extra) -> terminate(_Reason, _State) -> ok. -%% @private --spec do_pick_version(pid(), api(), range()) -> vsn(). -do_pick_version(_SockPid, _API, {Vsn, Vsn}) -> - %% only one version supported, no need to lookup - Vsn; -do_pick_version(SockPid, API, {Min, _Max}) -> - %% query the highest supported version - case lookup_version(SockPid, API) of - none -> Min; %% no version received from kafka, use min - Vsn -> Vsn %% use max supported version - end. - -%% @private Lookup API from cache, return default if not found. --spec lookup_version(pid(), api()) -> vsn() | none. -lookup_version(SockPid, API) -> - case ets:lookup(?ETS, SockPid) of - [] -> none; - [{SockPid, Versions}] -> - case lists:keyfind(API, 1, Versions) of - {API, Vsn} -> Vsn; - false -> none - end - end. - -%% @private --spec resolve_version_ranges(client_id(), [{api(), range()}], Acc) -> Acc - when Acc :: [{api(), vsn()}]. -resolve_version_ranges(_ClientId, [], Acc) -> lists:reverse(Acc); -resolve_version_ranges(ClientId, [{API, {MinKafka, MaxKafka}} | Rest], Acc) -> - case resolve_version_range(ClientId, API, MinKafka, MaxKafka, - supported_versions(API)) of - none -> resolve_version_ranges(ClientId, Rest, Acc); - Max -> resolve_version_ranges(ClientId, Rest, [{API, Max} | Acc]) +%%%_* Internals ================================================================ + +-spec do_pick_version(conn(), api(), range()) -> vsn(). +do_pick_version(_Conn, _API, {V, V}) -> V; +do_pick_version(Conn, API, {Min, Max} = MyRange) -> + case lookup_vsn_range(Conn, API) of + none -> + Min; %% no version received from kafka, use min + {KproMin, KproMax} = Range when KproMin > Max orelse KproMax < Min -> + erlang:error({unsupported_vsn_range, API, MyRange, Range}); + {_, KproMax} -> + min(KproMax, Max) %% try to use highest version end. -%% @private --spec resolve_version_range(client_id(), api(), vsn(), vsn(), - range() | none()) -> vsn() | none. -resolve_version_range(_ClientId, _API, _MinKafka, _MaxKafka, none) -> - %% API not implemented by brod - none; -resolve_version_range(ClientId, API, MinKafka, MaxKafka, {MinBrod, MaxBrod}) -> - Min = max(MinBrod, MinKafka), - Max = min(MaxBrod, MaxKafka), - case Min =< Max of - true when MinBrod =:= MaxBrod -> - %% if brod supports only one version - %% no need to store the range in ETS - none; - true -> - Max; - false -> - log_unsupported_api(ClientId, API, - {MinBrod, MaxBrod}, {MinKafka, MaxKafka}), - none +%% Lookup API from cache, return 'none' if not found. +-dialyzer([{nowarn_function, [lookup_vsn_range/2]}]). +-spec lookup_vsn_range(conn(), api()) -> {vsn(), vsn()} | none. +lookup_vsn_range(Conn, API) -> + case ets:lookup(?ETS, Conn) of + [] -> + case kpro:get_api_versions(Conn) of + {ok, Versions} when is_map(Versions) -> + %% public ets, insert it by caller + ets:insert(?ETS, {Conn, Versions}), + %% tell ?SERVER to monitor the connection + %% so to delete it from cache at when 'DOWN' is received + ok = monitor_connection(Conn), + maps:get(API, Versions, none); + {error, _Reason} -> + none %% connection died, ignore + end; + [{Conn, Vsns}] -> + maps:get(API, Vsns, none) end. -%% @private --spec log_unsupported_api(client_id(), api(), range(), range()) -> ok. -log_unsupported_api(ClientId, API, BrodRange, KafkaRange) -> - error_logger:error_msg("Can not support API ~p for client ~p, " - "brod versions: ~p, kafka versions: ~p", - [API, ClientId, BrodRange, KafkaRange]), - ok. - -%% @private Do not change range without verification. -%%% Fixed (hardcoded) version APIs -%% sasl_handshake_request: 0 -%% api_versions_request: 0 - -%%% Missing features -%% {create_topics_request, 0, 0} -%% {delete_topics_request, 0, 0} - -%%% Will not support -%% leader_and_isr_request -%% stop_replica_request -%% update_metadata_request -%% controlled_shutdown_request -%% @end +%% Do not change range without verification. supported_versions(API) -> case API of - produce_request -> {0, 2}; - fetch_request -> {0, 3}; - offsets_request -> {0, 1}; - metadata_request -> {0, 2}; - offset_commit_request -> {2, 2}; - offset_fetch_request -> {1, 2}; - group_coordinator_request -> {0, 0}; - join_group_request -> {0, 0}; - heartbeat_request -> {0, 0}; - leave_group_request -> {0, 0}; - sync_group_request -> {0, 0}; - describe_groups_request -> {0, 0}; - list_groups_request -> {0, 0}; - _ -> none + produce -> {0, 2}; + fetch -> {0, 3}; + list_offsets -> {0, 1}; + metadata -> {0, 2}; + offset_commit -> {2, 2}; + offset_fetch -> {1, 2}; + find_coordinator -> {0, 0}; + join_group -> {0, 0}; + heartbeat -> {0, 0}; + leave_group -> {0, 0}; + sync_group -> {0, 0}; + describe_groups -> {0, 0}; + list_groups -> {0, 0}; + _ -> erlang:error({unsupported_api, API}) end. +monitor_connection(Conn) -> + gen_server:cast(?SERVER, {monitor_connection, Conn}). + %%%_* Emacs ==================================================================== %%% Local Variables: %%% allout-layout: t diff --git a/src/brod_kafka_request.erl b/src/brod_kafka_request.erl index 67832de0..dfd30b83 100644 --- a/src/brod_kafka_request.erl +++ b/src/brod_kafka_request.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2017 Klarna AB +%%% Copyright (c) 2017-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -17,11 +17,15 @@ %% @doc Help functions to build request messages. -module(brod_kafka_request). --export([ fetch_request/7 - , metadata_request/2 - , offsets_request/4 - , produce_request/7 - , offset_fetch_request/3 +-export([ fetch/7 + , list_groups/1 + , list_offsets/4 + , join_group/2 + , metadata/2 + , offset_commit/2 + , offset_fetch/3 + , produce/7 + , sync_group/2 ]). -include("brod_int.hrl"). @@ -31,70 +35,61 @@ -type topic() :: brod:topic(). -type partition() :: brod:partition(). -type offset() :: brod:offset(). +-type conn() :: kpro:connection(). -%% @doc Make a produce request, If the first arg is a `brod_sock' pid, call +%% @doc Make a produce request, If the first arg is a connection pid, call %% `brod_kafka_apis:pick_version/2' to resolve version. -%% -%% NOTE: `pick_version' is essentially a ets lookup, for intensive callers -%% like `brod_producer', we should pick version before hand -%% and re-use it for each produce request. -%% @end --spec produce_request(pid() | vsn(), topic(), partition(), - brod:kv_list(), integer(), integer(), - brod:compression()) -> kpro:req(). -produce_request(MaybePid, Topic, Partition, KvList, - RequiredAcks, AckTimeout, Compression) -> - Vsn = pick_version(produce_request, MaybePid), - kpro:produce_request(Vsn, Topic, Partition, KvList, - RequiredAcks, AckTimeout, Compression). - -%% @doc Make a fetch request, If the first arg is a `brod_sock' pid, call +-spec produce(conn() | vsn(), topic(), partition(), + brod:kv_list(), integer(), integer(), + brod:compression()) -> kpro:req(). +produce(MaybePid, Topic, Partition, KvList, + RequiredAcks, AckTimeout, Compression) -> + Vsn = pick_version(produce, MaybePid), + kpro_req_lib:produce(Vsn, Topic, Partition, KvList, + #{ required_acks => RequiredAcks + , ack_timeout => AckTimeout + , compression => Compression + }). + +%% @doc Make a fetch request, If the first arg is a connection pid, call %% `brod_kafka_apis:pick_version/2' to resolve version. -%% -%% NOTE: `pick_version' is essentially a ets lookup, for intensive callers -%% like `brod_producer', we should pick version beforehand -%% and re-use it for each produce request. -%% @end --spec fetch_request(pid(), topic(), partition(), offset(), - kpro:wait(), kpro:count(), kpro:count()) -> kpro:req(). -fetch_request(Pid, Topic, Partition, Offset, - WaitTime, MinBytes, MaxBytes) -> - Vsn = pick_version(fetch_request, Pid), - kpro:fetch_request(Vsn, Topic, Partition, Offset, - WaitTime, MinBytes, MaxBytes). - -%% @doc Make a 'offsets_request' message for offset resolution. +-spec fetch(conn(), topic(), partition(), offset(), + kpro:wait(), kpro:count(), kpro:count()) -> kpro:req(). +fetch(Pid, Topic, Partition, Offset, + WaitTime, MinBytes, MaxBytes) -> + Vsn = pick_version(fetch, Pid), + kpro_req_lib:fetch(Vsn, Topic, Partition, Offset, + #{ max_wait_time => WaitTime + , min_bytes => MinBytes + , max_bytes => MaxBytes + }). + +%% @doc Make a `list_offsets' request message for offset resolution. %% In kafka protocol, -2 and -1 are semantic 'time' to request for %% 'earliest' and 'latest' offsets. %% In brod implementation, -2, -1, 'earliest' and 'latest' %% are semantic 'offset', this is why often a variable named %% Offset is used as the Time argument. -%% @end --spec offsets_request(pid(), topic(), partition(), brod:offset_time()) -> +-spec list_offsets(conn(), topic(), partition(), brod:offset_time()) -> kpro:req(). -offsets_request(SockPid, Topic, Partition, TimeOrSemanticOffset) -> +list_offsets(Connection, Topic, Partition, TimeOrSemanticOffset) -> Time = ensure_integer_offset_time(TimeOrSemanticOffset), - Vsn = pick_version(offsets_request, SockPid), - kpro:offsets_request(Vsn, Topic, Partition, Time). + Vsn = pick_version(list_offsets, Connection), + kpro_req_lib:list_offsets(Vsn, Topic, Partition, Time). %% @doc Make a metadata request. --spec metadata_request(pid(), [topic()]) -> kpro:req(). -metadata_request(SockPid, Topics) -> - Vsn = pick_version(metadata_request, SockPid), - TopicsForEncoding = - case Vsn of - 0 -> Topics; - _ when Topics =:= [] -> ?kpro_null; - _ -> Topics - end, - kpro:req(metadata_request, Vsn, [{topics, TopicsForEncoding}]). +-spec metadata(vsn() | conn(), all | [topic()]) -> kpro:req(). +metadata(Connection, Topics) when is_pid(Connection) -> + Vsn = brod_kafka_apis:pick_version(Connection, metadata), + metadata(Vsn, Topics); +metadata(Vsn, Topics) -> + kpro_req_lib:metadata(Vsn, Topics). %% @doc Make a offset fetch request. %% NOTE: empty topics list only works for kafka 0.10.2.0 or later -%% @end --spec offset_fetch_request(pid(), brod:group_id(), Topics) -> kpro:req() +-spec offset_fetch(conn(), brod:group_id(), Topics) -> kpro:req() when Topics :: [{topic(), [partition()]}]. -offset_fetch_request(SockPid, GroupId, Topics0) -> +offset_fetch(Connection, GroupId, Topics0) -> Topics = lists:map( fun({Topic, Partitions}) -> @@ -108,18 +103,45 @@ offset_fetch_request(SockPid, GroupId, Topics0) -> _ -> Topics end} ], - Vsn = pick_version(offset_fetch_request, SockPid), - kpro:req(offset_fetch_request, Vsn, Body). + Vsn = pick_version(offset_fetch, Connection), + kpro:make_request(offset_fetch, Vsn, Body). + +%% @doc Make a `list_groups' request. +-spec list_groups(conn()) -> kpro:req(). +list_groups(Connection) -> + Vsn = pick_version(list_groups, Connection), + kpro:make_request(list_groups, Vsn, []). + +%% @doc Make a `join_group' request. +-spec join_group(conn(), kpro:struct()) -> kpro:req(). +join_group(Conn, Fields) -> + make_req(join_group, Conn, Fields). + +%% @doc Make a `sync_group' request. +-spec sync_group(conn(), kpro:struct()) -> kpro:req(). +sync_group(Conn, Fields) -> + make_req(sync_group, Conn, Fields). + +%% @doc Make a `offset_commit' request. +-spec offset_commit(conn(), kpro:struct()) -> kpro:req(). +offset_commit(Conn, Fields) -> + make_req(offset_commit, Conn, Fields). + +%%%_* Internal Functions ======================================================= + +make_req(API, Conn, Fields) when is_pid(Conn) -> + Vsn = pick_version(API, Conn), + make_req(API, Vsn, Fields); +make_req(API, Vsn, Fields) -> + kpro:make_request(API, Vsn, Fields). -%% @private -spec pick_version(api(), pid()) -> vsn(). pick_version(_API, Vsn) when is_integer(Vsn) -> Vsn; -pick_version(API, SockPid) when is_pid(SockPid) -> - brod_kafka_apis:pick_version(SockPid, API); +pick_version(API, Connection) when is_pid(Connection) -> + brod_kafka_apis:pick_version(Connection, API); pick_version(API, _) -> brod_kafka_apis:default_version(API). -%% @private -spec ensure_integer_offset_time(brod:offset_time()) -> integer(). ensure_integer_offset_time(?OFFSET_EARLIEST) -> -2; ensure_integer_offset_time(?OFFSET_LATEST) -> -1; diff --git a/src/brod_kafka_requests.erl b/src/brod_kafka_requests.erl deleted file mode 100644 index 96a5477a..00000000 --- a/src/brod_kafka_requests.erl +++ /dev/null @@ -1,114 +0,0 @@ -%%% -%%% Copyright (c) 2014-2017, Klarna AB -%%% -%%% Licensed under the Apache License, Version 2.0 (the "License"); -%%% you may not use this file except in compliance with the License. -%%% You may obtain a copy of the License at -%%% -%%% http://www.apache.org/licenses/LICENSE-2.0 -%%% -%%% Unless required by applicable law or agreed to in writing, software -%%% distributed under the License is distributed on an "AS IS" BASIS, -%%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -%%% See the License for the specific language governing permissions and -%%% limitations under the License. -%%% - -%%%============================================================================= -%%% @doc -%%% This module manages an opaque of sent-request collection. -%%% @end -%%% ============================================================================ - -%% @private --module(brod_kafka_requests). - -%%%_* Exports ================================================================== - -%% API --export([ new/0 - , add/2 - , del/2 - , get_caller/2 - , get_corr_id/1 - , increment_corr_id/1 - , scan_for_max_age/1 - ]). - --export_type([requests/0]). - --record(requests, - { corr_id = 0 - , sent = gb_trees:empty() :: gb_trees:tree() - }). - --opaque requests() :: #requests{}. - --define(REQ(Caller, Ts), {Caller, Ts}). --define(MAX_CORR_ID_WINDOW_SIZE, (?MAX_CORR_ID div 2)). - -%%%_* Includes ================================================================= --include("brod_int.hrl"). - -%%%_* APIs ===================================================================== - --spec new() -> requests(). -new() -> #requests{}. - -%% @doc Add a new request to sent collection. -%% Return the last corrlation ID and the new opaque. -%% @end --spec add(requests(), pid()) -> {brod:corr_id(), requests()}. -add(#requests{ corr_id = CorrId - , sent = Sent - } = Requests, Caller) -> - NewSent = gb_trees:insert(CorrId, ?REQ(Caller, os:timestamp()), Sent), - NewRequests = Requests#requests{ corr_id = kpro:next_corr_id(CorrId) - , sent = NewSent - }, - {CorrId, NewRequests}. - -%% @doc Delete a request from the opaque collection. -%% Crash if correlation ID is not found. -%% @end --spec del(requests(), brod:corr_id()) -> requests(). -del(#requests{sent = Sent} = Requests, CorrId) -> - Requests#requests{sent = gb_trees:delete(CorrId, Sent)}. - -%% @doc Get caller of a request having the given correlation ID. -%% Crash if the request is not found. -%% @end --spec get_caller(requests(), brod:corr_id()) -> pid(). -get_caller(#requests{sent = Sent}, CorrId) -> - ?REQ(Caller, _Ts) = gb_trees:get(CorrId, Sent), - Caller. - -%% @doc Get the correction to be sent for the next request. --spec get_corr_id(requests()) -> brod:corr_id(). -get_corr_id(#requests{ corr_id = CorrId }) -> - CorrId. - -%% @doc Fetch and increment the correlation ID -%% This is used if we don't want a response from the broker -%% @end --spec increment_corr_id(requests()) -> {brod:corr_id(), requests()}. -increment_corr_id(#requests{corr_id = CorrId} = Requests) -> - {CorrId, Requests#requests{ corr_id = kpro:next_corr_id(CorrId) }}. - -%% @doc Scan the gb_tree to get oldest sent request. -%% Age is in milli-seconds. -%% 0 is returned if there is no pending response. -%% @end --spec scan_for_max_age(requests()) -> timeout(). -scan_for_max_age(#requests{sent = Sent}) -> - Now = os:timestamp(), - MinTs = lists:foldl(fun({_, ?REQ(_Caller, Ts)}, Min) -> - min(Ts, Min) - end, Now, gb_trees:to_list(Sent)), - timer:now_diff(Now, MinTs) div 1000. - -%%%_* Emacs ==================================================================== -%%% Local Variables: -%%% allout-layout: t -%%% erlang-indent-level: 2 -%%% End: diff --git a/src/brod_producer.erl b/src/brod_producer.erl index 68184d49..b954c94a 100644 --- a/src/brod_producer.erl +++ b/src/brod_producer.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2014-2017, Klarna AB +%%% Copyright (c) 2014-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -70,16 +70,16 @@ -type topic() :: brod:topic(). -type partition() :: brod:partition(). -type offset() :: brod:offset(). --type corr_id() :: brod:corr_id(). -type config() :: proplists:proplist(). -type call_ref() :: brod:call_ref(). +-type conn() :: kpro:connection(). -record(state, { client_pid :: pid() , topic :: topic() , partition :: partition() - , sock_pid :: ?undef | pid() - , sock_mref :: ?undef | reference() + , connection :: ?undef | conn() + , conn_mref :: ?undef | reference() , buffer :: brod_producer_buffer:buf() , retry_backoff_ms :: non_neg_integer() , retry_tref :: ?undef | reference() @@ -132,8 +132,8 @@ %% 'max.message.bytes' in kafka config (or topic config) %% max_retries (optional, default = 3): %% If {max_retries, N} is given, the producer retry produce request for -%% N times before crashing in case of failures like socket being shut down -%% or exceptions received in produce response from kafka. +%% N times before crashing in case of failures like connection being +%% shutdown by remote or exceptions received in produce response from kafka. %% The special value N = -1 means 'retry indefinitely' %% retry_backoff_ms (optional, default = 500); %% Time in milli-seconds to sleep before retry the failed produce request. @@ -155,7 +155,6 @@ %% threshold is hit. %% NOTE: It does not make sense to have this value set larger than %% `partition_buffer_limit' -%% @end -spec start_link(pid(), topic(), partition(), config()) -> {ok, pid()}. start_link(ClientPid, Topic, Partition, Config) -> gen_server:start_link(?MODULE, {ClientPid, Topic, Partition, Config}, []). @@ -166,7 +165,6 @@ start_link(ClientPid, Topic, Partition, Config) -> %% caller so the caller can used it to expect (match) a brod_produce_req_acked %% message after the produce request has been acked by configured number of %% replicas in kafka cluster. -%% @end -spec produce(pid(), brod:key(), brod:value()) -> {ok, call_ref()} | {error, any()}. produce(Pid, Key, Value) -> @@ -186,11 +184,11 @@ produce(Pid, Key, Value) -> end. %% @doc Block calling process until it receives an acked reply for the CallRef. -%% The caller pid of this function must be the caller of produce/3 -%% in which the call reference was created. -%% @end +%% The caller pid of this function must be the caller of produce/3 +%% in which the call reference was created. -spec sync_produce_request(call_ref(), timeout()) -> - {ok, offset()} | {error, {producer_down, any()}}. + {ok, offset()} | {error, Reason} + when Reason :: timeout | {producer_down, any()}. sync_produce_request(CallRef, Timeout) -> #brod_call_ref{ caller = Caller , callee = Callee @@ -239,23 +237,30 @@ init({ClientPid, Topic, Partition, Config}) -> end end, SendFun = - fun(SockPid, KafkaKvList, Vsn) -> + fun(Conn, KafkaKvList, Vsn) -> ProduceRequest = - kpro:produce_request(Vsn, Topic, Partition, KafkaKvList, - RequiredAcks, AckTimeout, - MaybeCompress(KafkaKvList)), - sock_send(SockPid, ProduceRequest) + brod_kafka_request:produce(Vsn, Topic, Partition, KafkaKvList, + RequiredAcks, AckTimeout, + MaybeCompress(KafkaKvList)), + case send(Conn, ProduceRequest) of + ok when ProduceRequest#kpro_req.no_ack -> + ok; + ok -> + {ok, ProduceRequest#kpro_req.ref}; + {error, Reason} -> + {error, Reason} + end end, Buffer = brod_producer_buffer:new(BufferLimit, OnWireLimit, MaxBatchSize, MaxRetries, MaxLingerMs, MaxLingerCount, SendFun), - DefaultReqVersion = brod_kafka_apis:default_version(produce_request), + DefaultReqVersion = brod_kafka_apis:default_version(produce), State = #state{ client_pid = ClientPid , topic = Topic , partition = Partition , buffer = Buffer , retry_backoff_ms = RetryBackoffMs - , sock_pid = ?undef + , connection = ?undef , produce_req_vsn = DefaultReqVersion }, %% Register self() to client. @@ -272,33 +277,33 @@ handle_info(?DELAYED_SEND_MSG(_MsgRef), #state{} = State) -> {noreply, State}; handle_info(?RETRY_MSG, #state{} = State0) -> State1 = State0#state{retry_tref = ?undef}, - {ok, State2} = maybe_reinit_socket(State1), - %% For retry-interval deterministic, produce regardless of socket state. - %% In case it has failed to find a new socket in maybe_reinit_socket/1 + {ok, State2} = maybe_reinit_connection(State1), + %% For retry-interval deterministic, produce regardless of connection state. + %% In case it has failed to find a new connection in maybe_reinit_connection/1 %% the produce call should fail immediately with {error, no_leader} %% and a new retry should be scheduled (if not reached max_retries yet) {ok, State} = maybe_produce(State2), {noreply, State}; handle_info({'DOWN', _MonitorRef, process, Pid, Reason}, - #state{sock_pid = Pid, buffer = Buffer0} = State) -> + #state{connection = Pid, buffer = Buffer0} = State) -> case brod_producer_buffer:is_empty(Buffer0) of true -> - %% no socket restart in case of empty request buffer - {noreply, State#state{sock_pid = ?undef}}; + %% no connection restart in case of empty request buffer + {noreply, State#state{connection = ?undef, conn_mref = ?undef}}; false -> - %% put sent requests back to buffer immediately after socket down + %% put sent requests back to buffer immediately after connection down %% to fail fast if retry is not allowed (reaching max_retries). - {ok, Buffer} = brod_producer_buffer:nack_all(Buffer0, Reason), + Buffer = brod_producer_buffer:nack_all(Buffer0, Reason), {ok, NewState} = schedule_retry(State#state{buffer = Buffer}), - {noreply, NewState#state{sock_pid = ?undef}} + {noreply, NewState#state{connection = ?undef, conn_mref = ?undef}} end; handle_info({produce, CallRef, Key, Value}, #state{} = State) -> handle_produce(CallRef, Key, Value, State); -handle_info({msg, Pid, #kpro_rsp{ tag = produce_response - , corr_id = CorrId - , msg = Rsp +handle_info({msg, Pid, #kpro_rsp{ api = produce + , ref = Ref + , msg = Rsp }}, - #state{ sock_pid = Pid + #state{ connection = Pid , buffer = Buffer } = State) -> [TopicRsp] = kpro:find(responses, Rsp), @@ -316,21 +321,11 @@ handle_info({msg, Pid, #kpro_rsp{ tag = produce_response Error = {produce_response_error, Topic, Partition, Offset, ErrorCode}, is_retriable(ErrorCode) orelse exit({not_retriable, Error}), - case brod_producer_buffer:nack(Buffer, CorrId, Error) of - {ok, NewBuffer} -> - schedule_retry(State#state{buffer = NewBuffer}); - {error, CorrIdExpected} -> - _ = log_discarded_corr_id(CorrId, CorrIdExpected), - maybe_produce(State) - end; + NewBuffer = brod_producer_buffer:nack(Buffer, Ref, Error), + schedule_retry(State#state{buffer = NewBuffer}); false -> - case brod_producer_buffer:ack(Buffer, CorrId, Offset) of - {ok, NewBuffer} -> - maybe_produce(State#state{buffer = NewBuffer}); - {error, CorrIdExpected} -> - _ = log_discarded_corr_id(CorrId, CorrIdExpected), - maybe_produce(State) - end + NewBuffer = brod_producer_buffer:ack(Buffer, Ref, Offset), + maybe_produce(State#state{buffer = NewBuffer}) end, {noreply, NewState}; handle_info(_Info, #state{} = State) -> @@ -352,7 +347,6 @@ terminate(_Reason, _State) -> %%%_* Internal Functions ======================================================= -%% @private -spec log_error_code(topic(), partition(), offset(), brod:error_code()) -> _. log_error_code(Topic, Partition, Offset, ErrorCode) -> brod_utils:log(error, @@ -360,82 +354,71 @@ log_error_code(Topic, Partition, Offset, ErrorCode) -> "Topic: ~s Partition: ~B Offset: ~B Error: ~p", [Topic, Partition, Offset, ErrorCode]). -%% @private --spec log_discarded_corr_id(corr_id(), none | corr_id()) -> _. -log_discarded_corr_id(CorrIdReceived, CorrIdExpected) -> - brod_utils:log(warning, - "Correlation ID discarded:~p, expecting: ~p", - [CorrIdReceived, CorrIdExpected]). - -%% @private handle_produce(CallRef, Key, Value, #state{retry_tref = Ref} = State) when is_reference(Ref) -> - %% pending on retry, add to buffer regardless of socket state + %% pending on retry, add to buffer regardless of connection state do_handle_produce(CallRef, Key, Value, State); handle_produce(CallRef, Key, Value, - #state{sock_pid = Pid} = State) when is_pid(Pid) -> - %% Socket is alive, add to buffer, and try send produce request + #state{connection = Pid} = State) when is_pid(Pid) -> + %% Connection is alive, add to buffer, and try send produce request do_handle_produce(CallRef, Key, Value, State); handle_produce(CallRef, Key, Value, #state{} = State) -> - %% this is the first request after fresh start/restart or socket death - {ok, NewState} = maybe_reinit_socket(State), + %% this is the first request after fresh start/restart or conection death + {ok, NewState} = maybe_reinit_connection(State), do_handle_produce(CallRef, Key, Value, NewState). -%% @private do_handle_produce(CallRef, Key, Value, #state{buffer = Buffer} = State) -> - {ok, NewBuffer} = brod_producer_buffer:add(Buffer, CallRef, Key, Value), + NewBuffer = brod_producer_buffer:add(Buffer, CallRef, Key, Value), State1 = State#state{buffer = NewBuffer}, {ok, NewState} = maybe_produce(State1), {noreply, NewState}. -%% @private --spec maybe_reinit_socket(state()) -> {ok, state()}. -maybe_reinit_socket(#state{ client_pid = ClientPid - , sock_pid = OldSockPid - , sock_mref = OldSockMref +-spec maybe_reinit_connection(state()) -> {ok, state()}. +maybe_reinit_connection(#state{ client_pid = ClientPid + , connection = OldConnection + , conn_mref = OldConnMref , topic = Topic , partition = Partition , buffer = Buffer0 } = State) -> %% Lookup, or maybe (re-)establish a connection to partition leader case brod_client:get_leader_connection(ClientPid, Topic, Partition) of - {ok, OldSockPid} -> - %% Still the old socket + {ok, OldConnection} -> + %% Still the old connection {ok, State}; - {ok, SockPid} -> - ok = maybe_demonitor(OldSockMref), - SockMref = erlang:monitor(process, SockPid), + {ok, Connection} -> + ok = maybe_demonitor(OldConnMref), + ConnMref = erlang:monitor(process, Connection), %% Make sure the sent but not acked ones are put back to buffer - {ok, Buffer} = brod_producer_buffer:nack_all(Buffer0, new_leader), - ReqVersion = brod_kafka_apis:pick_version(SockPid, produce_request), - {ok, State#state{ sock_pid = SockPid - , sock_mref = SockMref + Buffer = brod_producer_buffer:nack_all(Buffer0, new_leader), + ReqVersion = brod_kafka_apis:pick_version(Connection, produce), + {ok, State#state{ connection = Connection + , conn_mref = ConnMref , buffer = Buffer , produce_req_vsn = ReqVersion }}; {error, Reason} -> - ok = maybe_demonitor(OldSockMref), + ok = maybe_demonitor(OldConnMref), %% Make sure the sent but not acked ones are put back to buffer - {ok, Buffer} = brod_producer_buffer:nack_all(Buffer0, no_leader), - brod_utils:log(warning, "Failed to (re)init socket, reason:\n~p", + Buffer = brod_producer_buffer:nack_all(Buffer0, no_leader), + brod_utils:log(warning, "Failed to (re)init connection, reason:\n~p", [Reason]), - {ok, State#state{ sock_pid = ?undef - , sock_mref = ?undef - , buffer = Buffer + {ok, State#state{ connection = ?undef + , conn_mref = ?undef + , buffer = Buffer }} end. -%% @private maybe_produce(#state{retry_tref = Ref} = State) when is_reference(Ref) -> %% pending on retry after failure {ok, State}; maybe_produce(#state{ buffer = Buffer0 - , sock_pid = SockPid + , connection = Connection , delay_send_ref = DelaySendRef0 , produce_req_vsn = Vsn } = State) -> _ = cancel_delay_send_timer(DelaySendRef0), - case brod_producer_buffer:maybe_send(Buffer0, SockPid, Vsn) of + case brod_producer_buffer:maybe_send(Buffer0, Connection, Vsn) of {ok, Buffer} -> %% One or more produce requests are sent; %% Or no more message left to send; @@ -449,33 +432,30 @@ maybe_produce(#state{ buffer = Buffer0 }, {ok, NewState}; {retry, Buffer} -> - %% Failed to send, e.g. due to socket error, retry later + %% Failed to send, e.g. due to connection error, retry later schedule_retry(State#state{buffer = Buffer}) end. -%% @private Start delay send timer. +%% Start delay send timer. -spec start_delay_send_timer(milli_sec()) -> delay_send_ref(). start_delay_send_timer(Timeout) -> MsgRef = make_ref(), TRef = erlang:send_after(Timeout, self(), ?DELAYED_SEND_MSG(MsgRef)), {TRef, MsgRef}. -%% @private Ensure delay send timer is canceled. +%% Ensure delay send timer is canceled. %% But not flushing the possibly already sent (stale) message %% Stale message should be discarded in handle_info -%% @end -spec cancel_delay_send_timer(delay_send_ref()) -> _. cancel_delay_send_timer(?undef) -> ok; cancel_delay_send_timer({Tref, _Msg}) -> _ = erlang:cancel_timer(Tref). -%% @private maybe_demonitor(?undef) -> ok; maybe_demonitor(Mref) -> true = erlang:demonitor(Mref, [flush]), ok. -%% @private schedule_retry(#state{ retry_tref = ?undef , retry_backoff_ms = Timeout } = State) -> @@ -485,23 +465,19 @@ schedule_retry(State) -> %% retry timer has been already activated {ok, State}. -%% @private -is_retriable(EC) when EC =:= ?EC_CORRUPT_MESSAGE; - EC =:= ?EC_UNKNOWN_TOPIC_OR_PARTITION; - EC =:= ?EC_LEADER_NOT_AVAILABLE; - EC =:= ?EC_NOT_LEADER_FOR_PARTITION; - EC =:= ?EC_REQUEST_TIMED_OUT; - EC =:= ?EC_NOT_ENOUGH_REPLICAS; - EC =:= ?EC_NOT_ENOUGH_REPLICAS_AFTER_APPEND -> +is_retriable(EC) when EC =:= ?unknown_topic_or_partition; + EC =:= ?leader_not_available; + EC =:= ?not_leader_for_partition; + EC =:= ?request_timed_out; + EC =:= ?not_enough_replicas; + EC =:= ?not_enough_replicas_after_append -> true; is_retriable(_) -> false. -%% @private --spec sock_send(?undef | pid(), kpro:req()) -> - ok | {ok, corr_id()} | {error, any()}. -sock_send(?undef, _KafkaReq) -> {error, no_leader}; -sock_send(SockPid, KafkaReq) -> brod_sock:request_async(SockPid, KafkaReq). +-spec send(?undef | pid(), kpro:req()) -> ok | {error, any()}. +send(?undef, _KafkaReq) -> {error, no_leader}; +send(Connection, KafkaReq) -> kpro:request_async(Connection, KafkaReq). %%%_* Emacs ==================================================================== %%% Local Variables: diff --git a/src/brod_producer_buffer.erl b/src/brod_producer_buffer.erl index 42f1dead..8bcc111d 100644 --- a/src/brod_producer_buffer.erl +++ b/src/brod_producer_buffer.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2015-2017, Klarna AB +%%% Copyright (c) 2015-2018, Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -52,7 +52,7 @@ -type vsn() :: brod_kafka_apis:vsn(). -type send_fun() :: fun((pid(), [{brod:key(), brod:value()}], vsn()) -> ok | - {ok, brod:corr_id()} | + {ok, reference()} | {error, any()}). -define(ERR_FUN, fun() -> erlang:error(bad_init) end). @@ -70,19 +70,17 @@ , onwire_count = 0 :: non_neg_integer() , pending = ?NEW_QUEUE :: queue:queue(req()) , buffer = ?NEW_QUEUE :: queue:queue(req()) - , onwire = [] :: [{brod:corr_id(), [req()]}] + , onwire = [] :: [{reference(), [req()]}] }). -opaque buf() :: #buf{}. --type corr_id() :: brod:corr_id(). -type offset() :: brod:offset(). %%%_* APIs ===================================================================== %% @doc Create a new buffer %% For more details: @see brod_producer:start_link/4 -%% @end -spec new(pos_integer(), pos_integer(), pos_integer(), integer(), milli_sec(), count(), send_fun()) -> buf(). new(BufferLimit, OnWireLimit, MaxBatchSize, MaxRetry, @@ -104,8 +102,7 @@ new(BufferLimit, OnWireLimit, MaxBatchSize, MaxRetry, %% @doc Buffer a produce request. %% Respond to caller immediately if the buffer limit is not yet reached. -%% @end --spec add(buf(), brod:call_ref(), brod:key(), brod:value()) -> {ok, buf()}. +-spec add(buf(), brod:call_ref(), brod:key(), brod:value()) -> buf(). add(#buf{pending = Pending} = Buf, CallRef, Key, Value) -> Req = #req{ call_ref = CallRef , data = fun() -> {Key, Value} end @@ -129,62 +126,50 @@ add(#buf{pending = Pending} = Buf, CallRef, Key, Value) -> %% Caller should retry after the returned milli-seconds. %% retry: %% Failed to send a batch, caller should schedule a delayed retry. -%% @end -spec maybe_send(buf(), pid(), vsn()) -> {Action, buf()} when Action :: ok | retry | {delay, milli_sec()}. -maybe_send(#buf{} = Buf, SockPid, Vsn) -> +maybe_send(#buf{} = Buf, Conn, Vsn) -> case take_reqs(Buf) of false -> {ok, Buf}; {delay, T} -> {{delay, T}, Buf}; - {Reqs, NewBuf} -> do_send(Reqs, NewBuf, SockPid, Vsn) + {Reqs, NewBuf} -> do_send(Reqs, NewBuf, Conn, Vsn) end. %% @doc Reply 'acked' to callers. --spec ack(buf(), corr_id()) -> {ok, buf()} | {error, none | corr_id()}. -ack(Buf, CorrId) -> - ack(Buf, CorrId, ?BROD_PRODUCE_UNKNOWN_OFFSET). +-spec ack(buf(), reference()) -> buf(). +ack(Buf, Ref) -> + ack(Buf, Ref, ?BROD_PRODUCE_UNKNOWN_OFFSET). --spec ack(buf(), corr_id(), offset()) -> - {ok, buf()} | {error, none | corr_id()}. +%% @doc Reply 'acked' with base offset to callers. +-spec ack(buf(), reference(), offset()) -> buf(). ack(#buf{ onwire_count = OnWireCount - , onwire = [{CorrId, Reqs} | Rest] - } = Buf, CorrId, BaseOffset) -> + , onwire = [{Ref, Reqs} | Rest] + } = Buf, Ref, BaseOffset) -> _ = lists:foldl(fun reply_acked/2, BaseOffset, Reqs), - {ok, Buf#buf{ onwire_count = OnWireCount - 1 - , onwire = Rest - }}; -ack(#buf{onwire = OnWire}, CorrIdReceived, _Offset) -> - %% unkonwn corr-id, ignore - CorrIdExpected = assert_corr_id(OnWire, CorrIdReceived), - {error, CorrIdExpected}. + Buf#buf{ onwire_count = OnWireCount - 1 + , onwire = Rest + }. %% @doc 'Negative' ack, put all sent requests back to the head of buffer. %% An 'exit' exception is raised if any of the negative-acked requests %% reached maximum retry limit. -%% Unknown correlation IDs are discarded. -%% @end --spec nack(buf(), corr_id(), any()) -> {ok, buf()} | {error, none | corr_id()}. -nack(#buf{onwire = [{CorrId, _Reqs} | _]} = Buf, CorrId, Reason) -> - nack_all(Buf, Reason); -nack(#buf{onwire = OnWire}, CorrIdReceived, _Reason) -> - CorrIdExpected = assert_corr_id(OnWire, CorrIdReceived), - {error, CorrIdExpected}. +-spec nack(buf(), reference(), any()) -> buf(). +nack(#buf{onwire = [{Ref, _Reqs} | _]} = Buf, Ref, Reason) -> + nack_all(Buf, Reason). %% @doc 'Negative' ack, put all sent requests back to the head of buffer. %% An 'exit' exception is raised if any of the negative-acked requests %% reached maximum retry limit. -%% @end --spec nack_all(buf(), any()) -> {ok, buf()}. +-spec nack_all(buf(), any()) -> buf(). nack_all(#buf{onwire = OnWire} = Buf, Reason) -> - AllOnWireReqs = lists:map(fun({_CorrId, Reqs}) -> Reqs end, OnWire), + AllOnWireReqs = lists:map(fun({_Ref, Reqs}) -> Reqs end, OnWire), NewBuf = Buf#buf{ onwire_count = 0 , onwire = [] }, - {ok, rebuffer_or_crash(lists:append(AllOnWireReqs), NewBuf, Reason)}. + rebuffer_or_crash(lists:append(AllOnWireReqs), NewBuf, Reason). %% @doc Return true if there is no message pending, %% buffered or waiting for ack. -%% @end -spec is_empty(buf()) -> boolean(). is_empty(#buf{ pending = Pending , buffer = Buffer @@ -196,34 +181,6 @@ is_empty(#buf{ pending = Pending %%%_* Internal functions ======================================================= -%% @private This is a validation on the received correlation IDs for produce -%% responses, the assumption made in brod implementation is that kafka broker -%% guarantees the produce responses are replied in the order the corresponding -%% produce requests were received from clients. -%% Return expected correlation ID, or otherwise raise an 'exit' exception. -%% @end --spec assert_corr_id([{corr_id(), [req()]}], corr_id()) -> none | corr_id(). -assert_corr_id(_OnWireRequests = [], _CorrIdReceived) -> - none; -assert_corr_id([{CorrId, _Req} | _], CorrIdReceived) -> - case is_later_corr_id(CorrId, CorrIdReceived) of - true -> exit({bad_order, CorrId, CorrIdReceived}); - false -> CorrId - end. - -%% @private Compare two corr-ids, return true if ID-2 is considered a 'later' -%% one comparing to ID1. -%% Assuming that no clients would send up to 2^26 messages asynchronously. -%% @end --spec is_later_corr_id(corr_id(), corr_id()) -> boolean(). -is_later_corr_id(Id1, Id2) -> - Diff = abs(Id1 - Id2), - case Diff < (kpro:max_corr_id() div 2) of - true -> Id1 < Id2; - false -> Id1 > Id2 - end. - -%% @private -spec take_reqs(buf()) -> false | {delay, milli_sec()} | {[req()], buf()}. take_reqs(#buf{ onwire_count = OnWireCount , onwire_limit = OnWireLimit @@ -237,11 +194,10 @@ take_reqs(#buf{ buffer = Buffer, pending = Pending} = Buf) -> false; false -> %% ensure buffer is not empty before calling do_take_reqs/1 - {ok, NewBuf} = maybe_buffer(Buf), + NewBuf = maybe_buffer(Buf), do_take_reqs(NewBuf) end. -%% @private -spec do_take_reqs(buf()) -> {delay, milli_sec()} | {[req()], buf()}. do_take_reqs(#buf{ max_linger_count = MaxLingerCount , buffer_count = BufferCount @@ -263,7 +219,6 @@ do_take_reqs(#buf{ max_linger_ms = MaxLingerMs take_reqs_loop(Buf, _Acc = [], _AccBytes = 0) end. -%% @private -spec take_reqs_loop(buf(), [req()], integer()) -> {[req()], buf()}. take_reqs_loop(#buf{ buffer_count = 0 , pending = Pending @@ -276,14 +231,13 @@ take_reqs_loop(#buf{ buffer_count = 0 {lists:reverse(Acc), Buf}; false -> %% Take requests from pending to buffer - {ok, NewBuf} = maybe_buffer(Buf), + NewBuf = maybe_buffer(Buf), %% and continue to accumulate the batch take_reqs_loop_2(NewBuf, Acc, AccBytes) end; take_reqs_loop(Buf, Acc, AccBytes) -> take_reqs_loop_2(Buf, Acc, AccBytes). -%% @private -spec take_reqs_loop_2(buf(), [req()], non_neg_integer()) -> {[req()], buf()}. take_reqs_loop_2(#buf{ buffer_count = BufferCount , buffer = Buffer @@ -297,7 +251,7 @@ take_reqs_loop_2(#buf{ buffer_count = BufferCount true -> %% finished accumulating the batch %% take more pending ones into buffer - {ok, NewBuf} = maybe_buffer(Buf), + NewBuf = maybe_buffer(Buf), {lists:reverse(Acc), NewBuf}; false -> {_, Rest} = queue:out(Buffer), @@ -307,42 +261,41 @@ take_reqs_loop_2(#buf{ buffer_count = BufferCount take_reqs_loop(NewBuf, [Req | Acc], BatchSize) end. -%% @private Send produce request to kafka. +%% Send produce request to kafka. -spec do_send([req()], buf(), pid(), brod_kafka_apis:vsn()) -> {Action, buf()} when Action :: ok | retry | {delay, milli_sec()}. do_send(Reqs, #buf{ onwire_count = OnWireCount , onwire = OnWire , send_fun = SendFun - } = Buf, SockPid, Vsn) -> + } = Buf, Conn, Vsn) -> MessageSet = lists:map(fun(#req{data = F}) -> F() end, Reqs), - case SendFun(SockPid, MessageSet, Vsn) of + case SendFun(Conn, MessageSet, Vsn) of ok -> %% fire and forget, do not add onwire counter ok = lists:foreach(fun reply_acked/1, Reqs), %% continue to try next batch - maybe_send(Buf, SockPid, Vsn); - {ok, CorrId} -> + maybe_send(Buf, Conn, Vsn); + {ok, Ref} -> %% Keep onwire message reference to match acks later on NewBuf = Buf#buf{ onwire_count = OnWireCount + 1 - , onwire = OnWire ++ [{CorrId, Reqs}] + , onwire = OnWire ++ [{Ref, Reqs}] }, %% continue try next batch - maybe_send(NewBuf, SockPid, Vsn); + maybe_send(NewBuf, Conn, Vsn); {error, Reason} -> %% The requests sent on-wire are not re-buffered here %% because there are still chances to receive acks for them. %% brod_producer should call nack_all to put all sent requests %% back to buffer for retry in any of the cases below: - %% 1. Socket pid monitoring 'DOWN' message is received - %% 2. Discovered a new leader (a new socket pid) + %% 1. Connection pid monitoring 'DOWN' message is received + %% 2. Discovered a new leader (a new connection pid) NewBuf = rebuffer_or_crash(Reqs, Buf, Reason), {retry, NewBuf} end. -%% @private Put the produce requests back to buffer. +%% Put the produce requests back to buffer. %% raise an 'exit' exception if the first request to send has reached %% retry limit -%% @end -spec rebuffer_or_crash([req()], buf(), any()) -> buf() | no_return(). rebuffer_or_crash([#req{failures = Failures} | _], #buf{max_retries = MaxRetries}, Reason) @@ -361,8 +314,8 @@ rebuffer_or_crash(Reqs0, #buf{ buffer = Buffer , buffer_count = length(Reqs) + BufferCount }. -%% @private Take pending requests into buffer and reply 'buffered' to caller. --spec maybe_buffer(buf()) -> {ok, buf()}. +%% Take pending requests into buffer and reply 'buffered' to caller. +-spec maybe_buffer(buf()) -> buf(). maybe_buffer(#buf{ buffer_limit = BufferLimit , buffer_count = BufferCount , pending = Pending @@ -377,10 +330,10 @@ maybe_buffer(#buf{ buffer_limit = BufferLimit }, maybe_buffer(NewBuf); {empty, _} -> - {ok, Buf} + Buf end; maybe_buffer(#buf{} = Buf) -> - {ok, Buf}. + Buf. -spec reply_buffered(req()) -> ok. reply_buffered(#req{call_ref = CallRef}) -> @@ -418,7 +371,6 @@ cast(Pid, Msg) -> -spec data_size(brod:key() | brod:value()) -> non_neg_integer(). data_size(Data) -> brod_utils:bytes(Data). -%% @private -spec now_ms() -> milli_ts(). now_ms() -> {M, S, Micro} = os:timestamp(), @@ -437,19 +389,6 @@ cast_test() -> end, ok = cast(?undef, Ref). -assert_corr_id_test() -> - Max = kpro:max_corr_id(), - {error, none} = ack(#buf{}, 0), - {error, none} = nack(#buf{}, 0, ignored), - {error, 1} = ack(#buf{onwire = [{1, req}]}, 0), - {error, 1} = nack(#buf{onwire = [{1, req}]}, 0, ignored), - {error, 1} = ack(#buf{onwire = [{1, req}]}, Max), - ?assertException(exit, {bad_order, 0, 1}, - ack(#buf{onwire = [{0, req}]}, 1)), - ?assertException(exit, {bad_order, Max, 0}, - ack(#buf{onwire = [{Max, req}]}, 0)), - ok. - -endif. % TEST %%%_* Emacs ==================================================================== diff --git a/src/brod_producers_sup.erl b/src/brod_producers_sup.erl index 94fb4dec..6734e584 100644 --- a/src/brod_producers_sup.erl +++ b/src/brod_producers_sup.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2015-2017 Klarna AB +%%% Copyright (c) 2015-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -16,9 +16,6 @@ %%%============================================================================= %%% @doc brod producers supervisor -%%% -%%% @copyright 2015 Klarna AB -%%% @end %%%============================================================================= -module(brod_producers_sup). diff --git a/src/brod_sock.erl b/src/brod_sock.erl deleted file mode 100644 index 55909390..00000000 --- a/src/brod_sock.erl +++ /dev/null @@ -1,678 +0,0 @@ -%%% -%%% Copyright (c) 2014-2017, Klarna AB -%%% -%%% Licensed under the Apache License, Version 2.0 (the "License"); -%%% you may not use this file except in compliance with the License. -%%% You may obtain a copy of the License at -%%% -%%% http://www.apache.org/licenses/LICENSE-2.0 -%%% -%%% Unless required by applicable law or agreed to in writing, software -%%% distributed under the License is distributed on an "AS IS" BASIS, -%%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -%%% See the License for the specific language governing permissions and -%%% limitations under the License. -%%% - -%% @private --module(brod_sock). - -%%%_* Exports ================================================================== - -%% API --export([ get_tcp_sock/1 - , init/5 - , loop/2 - , request_sync/3 - , request_async/2 - , start/4 - , start/5 - , start_link/4 - , start_link/5 - , stop/1 - , debug/2 - ]). - -%% system calls support for worker process --export([ system_continue/3 - , system_terminate/4 - , system_code_change/4 - , format_status/2 - ]). - --export_type([ options/0 - ]). - --define(DEFAULT_CONNECT_TIMEOUT, timer:seconds(5)). --define(DEFAULT_REQUEST_TIMEOUT, timer:minutes(4)). --define(SIZE_HEAD_BYTES, 4). - -%% try not to use 0 corr ID for the first few requests -%% as they are usually used by upper level callers --define(SASL_AUTH_REQ_CORRID, kpro:max_corr_id()). --define(API_VERSIONS_REQ_CORRID, (kpro:max_corr_id() - 1)). - --include("brod_int.hrl"). - --type opt_key() :: connect_timeout - | request_timeout - | ssl - | sasl. --type opt_val() :: term(). --type options() :: [{opt_key(), opt_val()}]. --type requests() :: brod_kafka_requests:requests(). --type byte_count() :: non_neg_integer(). - --record(acc, { expected_size = error(bad_init) :: byte_count() - , acc_size = 0 :: byte_count() - , acc_buffer = [] :: [binary()] %% received bytes in reversed order - }). - --type acc() :: binary() | #acc{}. - --record(state, { client_id :: binary() - , parent :: pid() - , sock :: ?undef | port() - , acc = <<>> :: acc() - , requests :: ?undef | requests() - , mod :: ?undef | gen_tcp | ssl - , req_timeout :: ?undef | timeout() - }). - --type state() :: #state{}. --type client_id() :: brod:client_id() | binary(). - -%%%_* API ====================================================================== - -%% @equiv start_link(Parent, Host, Port, ClientId, []) -start_link(Parent, Host, Port, ClientId) -> - start_link(Parent, Host, Port, ClientId, []). - --spec start_link(pid(), brod:hostname(), brod:portnum(), - client_id() | binary(), term()) -> - {ok, pid()} | {error, any()}. -start_link(Parent, Host, Port, ClientId, Options) when is_atom(ClientId) -> - BinClientId = atom_to_binary(ClientId, utf8), - start_link(Parent, Host, Port, BinClientId, Options); -start_link(Parent, Host, Port, ClientId, Options) when is_binary(ClientId) -> - proc_lib:start_link(?MODULE, init, [Parent, Host, Port, ClientId, Options]). - -%% @equiv start(Parent, Host, Port, ClientId, []) -start(Parent, Host, Port, ClientId) -> - start(Parent, Host, Port, ClientId, []). - --spec start(pid(), brod:hostname(), brod:portnum(), - client_id() | binary(), term()) -> - {ok, pid()} | {error, any()}. -start(Parent, Host, Port, ClientId, Options) when is_atom(ClientId) -> - BinClientId = atom_to_binary(ClientId, utf8), - start(Parent, Host, Port, BinClientId, Options); -start(Parent, Host, Port, ClientId, Options) when is_binary(ClientId) -> - proc_lib:start(?MODULE, init, [Parent, Host, Port, ClientId, Options]). - -%% @doc Send a request and wait (indefinitely) for response. --spec request_async(pid(), kpro:req()) -> - {ok, brod:corr_id()} | ok | {error, any()}. -request_async(Pid, Request) -> - case call(Pid, {send, Request}) of - {ok, CorrId} -> - case Request of - #kpro_req{no_ack = true} -> ok; - _ -> {ok, CorrId} - end; - {error, Reason} -> - {error, Reason} - end. - -%% @doc Send a request and wait for response for at most Timeout milliseconds. --spec request_sync(pid(), kpro:req(), timeout()) -> - {ok, term()} | ok | {error, any()}. -request_sync(Pid, Request, Timeout) -> - case request_async(Pid, Request) of - ok -> ok; - {ok, CorrId} -> wait_for_resp(Pid, Request, CorrId, Timeout); - {error, Reason} -> {error, Reason} - end. - -%% @doc Stop socket process. --spec stop(pid()) -> ok | {error, any()}. -stop(Pid) when is_pid(Pid) -> - call(Pid, stop); -stop(_) -> - ok. - -%% @hidden --spec get_tcp_sock(pid()) -> {ok, port()}. -get_tcp_sock(Pid) -> - call(Pid, get_tcp_sock). - -%% @doc Enable/disable debugging on the socket process. -%% debug(Pid, pring) prints debug info on stdout -%% debug(Pid, File) prints debug info into a File -%% debug(Pid, none) stops debugging -%% @end --spec debug(pid(), print | string() | none) -> ok. -debug(Pid, none) -> - system_call(Pid, {debug, no_debug}); -debug(Pid, print) -> - system_call(Pid, {debug, {trace, true}}); -debug(Pid, File) when is_list(File) -> - system_call(Pid, {debug, {log_to_file, File}}). - -%%%_* Internal functions ======================================================= - -%% @private --spec init(pid(), brod:hostname(), brod:portnum(), - binary(), options()) -> no_return(). -init(Parent, Host, Port, ClientId, Options) -> - Timeout = get_connect_timeout(Options), - ExtraSockOpts = proplists:get_value(extra_sock_opts, Options, []), - SockOpts = [{active, once}, {packet, raw}, binary] ++ ExtraSockOpts, - case gen_tcp:connect(Host, Port, SockOpts, Timeout) of - {ok, Sock} -> - State = #state{ client_id = ClientId - , parent = Parent - }, - try - do_init(State, Sock, Host, Options) - catch - error : Reason -> - IsSsl = proplists:get_value(ssl, Options, false), - SaslOpt = brod_utils:get_sasl_opt(Options), - ok = maybe_log_hint(Host, Port, Reason, IsSsl, SaslOpt), - erlang:exit({Reason, erlang:get_stacktrace()}) - end; - {error, Reason} -> - %% exit instead of {error, Reason} - %% otherwise exit reason will be 'normal' - exit({connection_failure, Reason}) - end. - -%% @private --spec do_init(state(), port(), brod:hostname(), options()) -> no_return(). -do_init(State0, Sock, Host, Options) -> - #state{parent = Parent, client_id = ClientId} = State0, - Debug = sys:debug_options(proplists:get_value(debug, Options, [])), - Timeout = get_connect_timeout(Options), - %% adjusting buffer size as per recommendation at - %% http://erlang.org/doc/man/inet.html#setopts-2 - %% idea is from github.com/epgsql/epgsql - {ok, [{recbuf, RecBufSize}, {sndbuf, SndBufSize}]} = - inet:getopts(Sock, [recbuf, sndbuf]), - ok = inet:setopts(Sock, [{buffer, max(RecBufSize, SndBufSize)}]), - SslOpts = proplists:get_value(ssl, Options, false), - Mod = get_tcp_mod(SslOpts), - NewSock = maybe_upgrade_to_ssl(Sock, Mod, SslOpts, Timeout), - ok = sasl_auth(Host, NewSock, Mod, ClientId, Timeout, - brod_utils:get_sasl_opt(Options)), - MaybeQuery = proplists:get_value(query_api_versions, Options), - ok = maybe_query_api_versions(MaybeQuery, Host, ClientId, - NewSock, Mod, Timeout), - State = State0#state{mod = Mod, sock = NewSock}, - proc_lib:init_ack(Parent, {ok, self()}), - ReqTimeout = get_request_timeout(Options), - ok = send_assert_max_req_age(self(), ReqTimeout), - Requests = brod_kafka_requests:new(), - loop(State#state{requests = Requests, req_timeout = ReqTimeout}, Debug). - -%% @private -maybe_query_api_versions(false, _Host, _ClientId, _Sock, _Mod, _Timeout) -> - %% do not query - ok; -maybe_query_api_versions(_, Host, ClientId, Sock, Mod, Timeout) -> - case brod_kafka_apis:maybe_add_sock_pid(Host, self()) of - ok -> - ok; - {error, unknown_host} -> - Versions = query_api_versions(ClientId, Sock, Mod, Timeout), - ok = brod_kafka_apis:versions_received(ClientId, self(), Versions, Host) - end. - -%% @private Query API version ranges. -query_api_versions(ClientId, Sock, Mod, Timeout) -> - ok = setopts(Sock, Mod, [{active, false}]), - Req = kpro:req(api_versions_request, _Vsn = 0, []), - ReqBin = kpro:encode_request(ClientId, ?API_VERSIONS_REQ_CORRID, Req), - Rsp = inactive_request_sync(Sock, Mod, ReqBin, Timeout, - query_api_versions_error), - #kpro_rsp{tag = api_versions_response, vsn = 0, msg = Body} = Rsp, - ErrorCode = kpro:find(error_code, Body), - case ?IS_ERROR(ErrorCode) of - true -> - erlang:error({failed_to_query_api_versions, ErrorCode}); - false -> - Versions = kpro:find(api_versions, Body), - F = fun(S) -> - ReqName = kpro:find(api_key, S), - MinVsn = kpro:find(min_version, S), - MaxVsn = kpro:find(max_version, S), - {ReqName, {MinVsn, MaxVsn}} - end, - lists:map(F, Versions) - end. - -%% @private Send request to active = false socket, and wait for response. -inactive_request_sync(Sock, Mod, ReqBin, Timeout, ErrorTag) -> - try - ok = Mod:send(Sock, ReqBin), - {ok, <>} = Mod:recv(Sock, 4, Timeout), - {ok, RspBin} = Mod:recv(Sock, Len, Timeout), - {[Rsp], <<>>} = kpro:decode_response(<>), - Rsp - catch - error : Reason -> - Stack = erlang:get_stacktrace(), - erlang:raise(error, {ErrorTag, Reason}, Stack) - end. - -get_tcp_mod(_SslOpts = true) -> ssl; -get_tcp_mod(_SslOpts = [_|_]) -> ssl; -get_tcp_mod(_) -> gen_tcp. - -maybe_upgrade_to_ssl(Sock, _Mod = ssl, SslOpts0, Timeout) -> - SslOpts = case SslOpts0 of - true -> []; - [_|_] -> SslOpts0 - end, - case ssl:connect(Sock, SslOpts, Timeout) of - {ok, NewSock} -> NewSock; - {error, Reason} -> erlang:error({failed_to_upgrade_to_ssl, Reason}) - end; -maybe_upgrade_to_ssl(Sock, _Mod, _SslOpts, _Timeout) -> - Sock. - -%% @private -sasl_auth(_Host, _Sock, _Mod, _ClientId, _Timeout, ?undef) -> - %% no auth - ok; -sasl_auth(_Host, Sock, Mod, ClientId, Timeout, - {_Method = plain, SaslUser, SaslPassword}) -> - ok = setopts(Sock, Mod, [{active, false}]), - Req = kpro:req(sasl_handshake_request, _V = 0, [{mechanism, <<"PLAIN">>}]), - HandshakeRequestBin = - kpro:encode_request(ClientId, ?SASL_AUTH_REQ_CORRID, Req), - Rsp = inactive_request_sync(Sock, Mod, HandshakeRequestBin, Timeout, - sasl_auth_error), - #kpro_rsp{tag = sasl_handshake_response, vsn = 0, msg = Body} = Rsp, - ErrorCode = kpro:find(error_code, Body), - case ?IS_ERROR(ErrorCode) of - true -> - erlang:error({sasl_auth_error, ErrorCode}); - false -> - ok = Mod:send(Sock, sasl_plain_token(SaslUser, SaslPassword)), - case Mod:recv(Sock, 4, Timeout) of - {ok, <<0:32>>} -> - ok; - {error, closed} -> - erlang:error({sasl_auth_error, bad_credentials}); - Unexpected -> - erlang:error({sasl_auth_error, Unexpected}) - end - end; -sasl_auth(Host, Sock, Mod, ClientId, Timeout, - {callback, ModuleName, Opts}) -> - case brod_auth_backend:auth(ModuleName, Host, Sock, Mod, - ClientId, Timeout, Opts) of - ok -> - ok; - {error, Reason} -> - erlang:error({sasl_auth_error, Reason}) - end. - -%% @private -sasl_plain_token(User, Password) -> - Message = list_to_binary([0, unicode:characters_to_binary(User), - 0, unicode:characters_to_binary(Password)]), - <<(byte_size(Message)):32, Message/binary>>. - -setopts(Sock, _Mod = gen_tcp, Opts) -> inet:setopts(Sock, Opts); -setopts(Sock, _Mod = ssl, Opts) -> ssl:setopts(Sock, Opts). - -%% @private --spec wait_for_resp(pid(), term(), brod:corr_id(), timeout()) -> - {ok, term()} | {error, any()}. -wait_for_resp(Pid, _, CorrId, Timeout) -> - Mref = erlang:monitor(process, Pid), - receive - {msg, Pid, #kpro_rsp{corr_id = CorrId} = Rsp} -> - erlang:demonitor(Mref, [flush]), - {ok, Rsp}; - {'DOWN', Mref, _, _, Reason} -> - {error, {sock_down, Reason}} - after - Timeout -> - erlang:demonitor(Mref, [flush]), - {error, timeout} - end. - -%% @private -system_call(Pid, Request) -> - Mref = erlang:monitor(process, Pid), - erlang:send(Pid, {system, {self(), Mref}, Request}), - receive - {Mref, Reply} -> - erlang:demonitor(Mref, [flush]), - Reply; - {'DOWN', Mref, _, _, Reason} -> - {error, {sock_down, Reason}} - end. - -%% @private -call(Pid, Request) -> - Mref = erlang:monitor(process, Pid), - erlang:send(Pid, {{self(), Mref}, Request}), - receive - {Mref, Reply} -> - erlang:demonitor(Mref, [flush]), - Reply; - {'DOWN', Mref, _, _, Reason} -> - {error, {sock_down, Reason}} - end. - -%% @private -reply({To, Tag}, Reply) -> - To ! {Tag, Reply}. - -%% @private -loop(#state{sock = Sock, mod = Mod} = State, Debug) -> - ok = setopts(Sock, Mod, [{active, once}]), - Msg = receive Input -> Input end, - decode_msg(Msg, State, Debug). - -%% @private -decode_msg({system, From, Msg}, #state{parent = Parent} = State, Debug) -> - sys:handle_system_msg(Msg, From, Parent, ?MODULE, Debug, State); -decode_msg(Msg, State, [] = Debug) -> - handle_msg(Msg, State, Debug); -decode_msg(Msg, State, Debug0) -> - Debug = sys:handle_debug(Debug0, fun print_msg/3, State, Msg), - handle_msg(Msg, State, Debug). - -%% @private -handle_msg({_, Sock, Bin}, #state{ sock = Sock - , acc = Acc0 - , requests = Requests - , mod = Mod - } = State, Debug) when is_binary(Bin) -> - case Mod of - gen_tcp -> ok = inet:setopts(Sock, [{active, once}]); - ssl -> ok = ssl:setopts(Sock, [{active, once}]) - end, - Acc1 = acc_recv_bytes(Acc0, Bin), - {Responses, Acc} = decode_response(Acc1), - NewRequests = - lists:foldl( - fun(#kpro_rsp{corr_id = CorrId} = Rsp, Reqs) -> - Caller = brod_kafka_requests:get_caller(Reqs, CorrId), - cast(Caller, {msg, self(), Rsp}), - brod_kafka_requests:del(Reqs, CorrId) - end, Requests, Responses), - ?MODULE:loop(State#state{acc = Acc, requests = NewRequests}, Debug); -handle_msg(assert_max_req_age, #state{ requests = Requests - , req_timeout = ReqTimeout - } = State, Debug) -> - SockPid = self(), - erlang:spawn_link(fun() -> - ok = assert_max_req_age(Requests, ReqTimeout), - ok = send_assert_max_req_age(SockPid, ReqTimeout) - end), - ?MODULE:loop(State, Debug); -handle_msg({tcp_closed, Sock}, #state{sock = Sock}, _) -> - exit({shutdown, tcp_closed}); -handle_msg({ssl_closed, Sock}, #state{sock = Sock}, _) -> - exit({shutdown, ssl_closed}); -handle_msg({tcp_error, Sock, Reason}, #state{sock = Sock}, _) -> - exit({tcp_error, Reason}); -handle_msg({ssl_error, Sock, Reason}, #state{sock = Sock}, _) -> - exit({ssl_error, Reason}); -handle_msg({From, {send, Request}}, - #state{ client_id = ClientId - , mod = Mod - , sock = Sock - , requests = Requests - } = State, Debug) -> - {Caller, _Ref} = From, - {CorrId, NewRequests} = - case Request of - #kpro_req{no_ack = true} -> - brod_kafka_requests:increment_corr_id(Requests); - _ -> - brod_kafka_requests:add(Requests, Caller) - end, - RequestBin = kpro:encode_request(ClientId, CorrId, Request), - Res = case Mod of - gen_tcp -> gen_tcp:send(Sock, RequestBin); - ssl -> ssl:send(Sock, RequestBin) - end, - case Res of - ok -> - _ = reply(From, {ok, CorrId}), - ok; - {error, Reason} -> - exit({send_error, Reason}) - end, - ?MODULE:loop(State#state{requests = NewRequests}, Debug); -handle_msg({From, get_tcp_sock}, State, Debug) -> - _ = reply(From, {ok, State#state.sock}), - ?MODULE:loop(State, Debug); -handle_msg({From, stop}, #state{mod = Mod, sock = Sock}, _Debug) -> - Mod:close(Sock), - _ = reply(From, ok), - ok; -handle_msg(Msg, #state{} = State, Debug) -> - error_logger:warning_msg("[~p] ~p got unrecognized message: ~p", - [?MODULE, self(), Msg]), - ?MODULE:loop(State, Debug). - -%% @private -cast(Pid, Msg) -> - try - Pid ! Msg, - ok - catch _ : _ -> - ok - end. - -%% @private -system_continue(_Parent, Debug, State) -> - ?MODULE:loop(State, Debug). - -%% @private --spec system_terminate(any(), _, _, _) -> no_return(). -system_terminate(Reason, _Parent, Debug, _Misc) -> - sys:print_log(Debug), - exit(Reason). - -%% @private -system_code_change(State, _Module, _Vsn, _Extra) -> - {ok, State}. - -%% @private -format_status(Opt, Status) -> - {Opt, Status}. - -%% @private -print_msg(Device, {_From, {send, Request}}, State) -> - do_print_msg(Device, "send: ~p", [Request], State); -print_msg(Device, {tcp, _Sock, Bin}, State) -> - do_print_msg(Device, "tcp: ~p", [Bin], State); -print_msg(Device, {tcp_closed, _Sock}, State) -> - do_print_msg(Device, "tcp_closed", [], State); -print_msg(Device, {tcp_error, _Sock, Reason}, State) -> - do_print_msg(Device, "tcp_error: ~p", [Reason], State); -print_msg(Device, {_From, stop}, State) -> - do_print_msg(Device, "stop", [], State); -print_msg(Device, Msg, State) -> - do_print_msg(Device, "unknown msg: ~p", [Msg], State). - -%% @private -do_print_msg(Device, Fmt, Args, State) -> - CorrId = brod_kafka_requests:get_corr_id(State#state.requests), - io:format(Device, "[~s] ~p [~10..0b] " ++ Fmt ++ "~n", - [ts(), self(), CorrId] ++ Args). - -%% @private -ts() -> - Now = os:timestamp(), - {_, _, MicroSec} = Now, - {{Y, M, D}, {HH, MM, SS}} = calendar:now_to_local_time(Now), - lists:flatten(io_lib:format("~.4.0w-~.2.0w-~.2.0w ~.2.0w:~.2.0w:~.2.0w.~w", - [Y, M, D, HH, MM, SS, MicroSec])). - -%% @private This is to be backward compatible for -%% 'timeout' as connect timeout option name -%% TODO: change to support 'connect_timeout' only for 2.3 -%% @end --spec get_connect_timeout(options()) -> timeout(). -get_connect_timeout(Options) -> - case {proplists:get_value(connect_timeout, Options), - proplists:get_value(timeout, Options)} of - {T, _} when is_integer(T) -> T; - {_, T} when is_integer(T) -> T; - _ -> ?DEFAULT_CONNECT_TIMEOUT - end. - -%% @private Get request timeout from options. --spec get_request_timeout(options()) -> timeout(). -get_request_timeout(Options) -> - proplists:get_value(request_timeout, Options, ?DEFAULT_REQUEST_TIMEOUT). - -%% @private --spec assert_max_req_age(requests(), timeout()) -> ok | no_return(). -assert_max_req_age(Requests, Timeout) -> - case brod_kafka_requests:scan_for_max_age(Requests) of - Age when Age > Timeout -> - erlang:exit(request_timeout); - _ -> - ok - end. - -%% @private Send the 'assert_max_req_age' message to brod_sock process. -%% The send interval is set to a half of configured timeout. -%% @end --spec send_assert_max_req_age(pid(), timeout()) -> ok. -send_assert_max_req_age(Pid, Timeout) when Timeout >= 1000 -> - %% Check every 1 minute - %% or every half of the timeout value if it's less than 2 minute - SendAfter = erlang:min(Timeout div 2, timer:minutes(1)), - _ = erlang:send_after(SendAfter, Pid, assert_max_req_age), - ok. - -%% @private Accumulate newly received bytes. --spec acc_recv_bytes(acc(), binary()) -> acc(). -acc_recv_bytes(Acc, NewBytes) when is_binary(Acc) -> - case <> of - <> = AccBytes -> - do_acc(#acc{expected_size = Size + ?SIZE_HEAD_BYTES}, AccBytes); - AccBytes -> - AccBytes - end; -acc_recv_bytes(#acc{} = Acc, NewBytes) -> - do_acc(Acc, NewBytes). - -%% @private Add newly received bytes to buffer. --spec do_acc(acc(), binary()) -> acc(). -do_acc(#acc{acc_size = AccSize, acc_buffer = AccBuffer} = Acc, NewBytes) -> - Acc#acc{acc_size = AccSize + size(NewBytes), - acc_buffer = [NewBytes | AccBuffer] - }. - -%% @private Decode response when accumulated enough bytes. --spec decode_response(acc()) -> {[kpro:rsp()], acc()}. -decode_response(#acc{expected_size = ExpectedSize, - acc_size = AccSize, - acc_buffer = AccBuffer}) when AccSize >= ExpectedSize -> - %% iolist_to_binary here to simplify kafka_protocol implementation - %% maybe make it smarter in the next version - kpro:decode_response(iolist_to_binary(lists:reverse(AccBuffer))); -decode_response(Acc) -> - {[], Acc}. - -%% @private brod supported endpoint is tuple {Hostname, Port} -%% which lacks of hint on which protocol to use. -%% It would be a bit nicer if we support endpoint formats like below: -%% PLAINTEX://hostname:port -%% SSL://hostname:port -%% SASL_PLAINTEXT://hostname:port -%% SASL_SSL://hostname:port -%% which may give some hint for early config validation before trying to -%% connect to the endpoint. -%% -%% However, even with the hint, it is still quite easy to misconfig and endup -%% with a clueless crash report. Here we try to make a guess on what went -%% wrong in case there was an error during connection estabilishment. -%% @end -maybe_log_hint(Host, Port, Reason, IsSsl, SaslOpt) -> - case hint_msg(Reason, IsSsl, SaslOpt) of - ?undef -> - ok; - Msg -> - error_logger:error_msg("Failed to connect to ~s:~p\n~s\n", - [Host, Port, Msg]) - end. - -%% @private -hint_msg({failed_to_upgrade_to_ssl, R}, _IsSsl, SaslOpt) when R =:= closed; - R =:= timeout -> - case SaslOpt of - ?undef -> "Make sure connecting to a 'SSL://' listener"; - _ -> "Make sure connecting to 'SASL_SSL://' listener" - end; -hint_msg({sasl_auth_error, 'IllegalSaslState'}, true, _SaslOpt) -> - "Make sure connecting to 'SASL_SSL://' listener"; -hint_msg({sasl_auth_error, 'IllegalSaslState'}, false, _SaslOpt) -> - "Make sure connecting to 'SASL_PLAINTEXT://' listener"; -hint_msg({sasl_auth_error, {badmatch, {error, enomem}}}, false, _SaslOpts) -> - %% This happens when KAFKA is expecting SSL handshake - %% but client started SASL handshake instead - "Make sure 'ssl' option is in client config, \n" - "or make sure connecting to 'SASL_PLAINTEXT://' listener"; -hint_msg(_, _, _) -> - %% Sorry, I have no clue, please read the crash log - ?undef. - -%%%_* Eunit ==================================================================== - --ifdef(TEST). - --include_lib("eunit/include/eunit.hrl"). - -acc_test_() -> - [{"clean start flow", - fun() -> - Acc0 = acc_recv_bytes(<<>>, <<0, 0>>), - ?assertEqual(Acc0, <<0, 0>>), - Acc1 = acc_recv_bytes(Acc0, <<0, 1, 0, 0>>), - ?assertEqual(#acc{expected_size = 5, - acc_size = 6, - acc_buffer = [<<0, 0, 0, 1, 0, 0>>] - }, Acc1) - end}, - {"old tail leftover", - fun() -> - Acc0 = acc_recv_bytes(<<0, 0>>, <<0, 4>>), - ?assertEqual(#acc{expected_size = 8, - acc_size = 4, - acc_buffer = [<<0, 0, 0, 4>>] - }, Acc0), - Acc1 = acc_recv_bytes(Acc0, <<0, 0>>), - ?assertEqual(#acc{expected_size = 8, - acc_size = 6, - acc_buffer = [<<0, 0>>, <<0, 0, 0, 4>>] - }, Acc1), - Acc2 = acc_recv_bytes(Acc1, <<1, 1>>), - ?assertEqual(#acc{expected_size = 8, - acc_size = 8, - acc_buffer = [<<1, 1>>, <<0, 0>>, <<0, 0, 0, 4>>] - }, Acc2) - end - } - ]. - --endif. - -%%%_* Emacs ==================================================================== -%%% Local Variables: -%%% allout-layout: t -%%% erlang-indent-level: 2 -%%% End: diff --git a/src/brod_sup.erl b/src/brod_sup.erl index bc9acd42..94200183 100644 --- a/src/brod_sup.erl +++ b/src/brod_sup.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2015-2017 Klarna AB +%%% Copyright (c) 2015-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -52,7 +52,6 @@ %%% | |... %%% |... %%% -%%% @copyright 2015 Klarna AB %%% @end %%%============================================================================= diff --git a/src/brod_topic_subscriber.erl b/src/brod_topic_subscriber.erl index d3af34fc..893f5d3a 100644 --- a/src/brod_topic_subscriber.erl +++ b/src/brod_topic_subscriber.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2016-2017 Klarna AB +%%% Copyright (c) 2016-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -117,7 +117,6 @@ %% @doc Start (link) a topic subscriber which receives and processes the %% messages from the given partition set. Use atom 'all' to subscribe to all %% partitions. Messages are handled by calling CbModule:handle_message -%% @end -spec start_link(brod:client(), brod:topic(), all | [brod:partition()], brod:consumer_config(), module(), term()) -> {ok, pid()} | {error, any()}. @@ -131,7 +130,6 @@ start_link(Client, Topic, Partitions, ConsumerConfig, %% messages or message sets from the given partition set. Use atom 'all' %% to subscribe to all partitions. Messages are handled by calling %% CbModule:handle_message -%% @end -spec start_link(brod:client(), brod:topic(), all | [brod:partition()], brod:consumer_config(), message | message_set, module(), term()) -> @@ -148,7 +146,6 @@ start_link(Client, Topic, Partitions, ConsumerConfig, %% %% NOTE: CommittedOffsets are the offsets for the messages that are successfully %% processed (acknoledged), not the begin-offset ot start fetching from. -%% @end -spec start_link(brod:client(), brod:topic(), all | [brod:partition()], brod:consumer_config(), committed_offsets(), message | message_set, cb_fun(), cb_state()) -> @@ -392,7 +389,7 @@ handle_ack(AckRef, #state{consumers = Consumers} = State) -> State end. -%% @private Tell consumer process to fetch more (if pre-fetch count allows). +%% Tell consumer process to fetch more (if pre-fetch count allows). consume_ack(Pid, Offset) when is_pid(Pid) -> ok = brod:consume_ack(Pid, Offset); consume_ack(_Down, _Offset) -> diff --git a/src/brod_utils.erl b/src/brod_utils.erl index 8491c089..fa25c840 100644 --- a/src/brod_utils.erl +++ b/src/brod_utils.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2014-2017, Klarna AB +%%% Copyright (c) 2014-2018, Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -22,19 +22,15 @@ , assert_topics/1 , assert_topic/1 , bytes/1 - , connect_group_coordinator/3 - , decode_messages/2 , describe_groups/3 , epoch_ms/0 - , fetch/8 + , fetch/5 , fetch_committed_offsets/3 , fetch_committed_offsets/4 - , find_leader_in_metadata/3 - , find_struct/3 + , flatten_batches/2 , get_metadata/1 , get_metadata/2 , get_metadata/3 - , get_sasl_opt/1 , group_per_key/1 , group_per_key/2 , init_sasl_opt/1 @@ -44,21 +40,21 @@ , list_all_groups/2 , list_groups/2 , log/3 - , make_fetch_fun/6 + , make_fetch_fun/4 , os_time_utc_str/0 + , parse_rsp/1 + , request_sync/2 + , request_sync/3 , resolve_offset/4 , resolve_offset/5 - , shutdown_pid/1 - , try_connect/1 - , try_connect/2 - , resolve_group_coordinator/3 ]). -include("brod_int.hrl"). -type req_fun() :: fun((offset(), kpro:count()) -> kpro:req()). -type fetch_fun() :: fun((offset()) -> {ok, [brod:message()]} | {error, any()}). --type sock_opts() :: brod:sock_opts(). +-type connection() :: kpro:connection(). +-type conn_config() :: brod:conn_config(). -type topic() :: brod:topic(). -type partition() :: brod:partition(). -type offset() :: brod:offset(). @@ -70,80 +66,52 @@ %% @doc Try to connect to any of the bootstrap nodes and fetch metadata %% for all topics -%% @end -spec get_metadata([endpoint()]) -> {ok, kpro:struct()} | {error, any()}. get_metadata(Hosts) -> - get_metadata(Hosts, []). + get_metadata(Hosts, all). %% @doc Try to connect to any of the bootstrap nodes and fetch metadata %% for the given topics -%% @end --spec get_metadata([endpoint()], [topic()]) -> +-spec get_metadata([endpoint()], all | [topic()]) -> {ok, kpro:struct()} | {error, any()}. get_metadata(Hosts, Topics) -> - get_metadata(Hosts, Topics, _Options = []). + get_metadata(Hosts, Topics, _ConnCfg = []). %% @doc Try to connect to any of the bootstrap nodes using the given %% connection options and fetch metadata for the given topics. -%% @end --spec get_metadata([endpoint()], [topic()], sock_opts()) -> +-spec get_metadata([endpoint()], all | [topic()], conn_config()) -> {ok, kpro:struct()} | {error, any()}. -get_metadata(Hosts, Topics, Options) -> - with_sock( - try_connect(Hosts, Options), - fun(Pid) -> - Request = brod_kafka_request:metadata_request(Pid, Topics), - #kpro_rsp{ tag = metadata_response - , msg = Msg - } = request_sync(Pid, Request), - {ok, Msg} - end). +get_metadata(Hosts, Topics, ConnCfg) -> + with_conn(Hosts, ConnCfg, + fun(Pid) -> + Request = brod_kafka_request:metadata(Pid, Topics), + request_sync(Pid, Request) + end). %% @doc Resolve timestamp to real offset. -spec resolve_offset([endpoint()], topic(), partition(), - offset_time(), sock_opts()) -> + offset_time(), conn_config()) -> {ok, offset()} | {error, any()}. -resolve_offset(Hosts, Topic, Partition, Time, Options) when is_list(Options) -> - with_sock( - brod:connect_leader(Hosts, Topic, Partition, Options), - fun(Pid) -> - resolve_offset(Pid, Topic, Partition, Time) - end). +resolve_offset(Hosts, Topic, Partition, Time, ConnCfg) -> + with_conn( + kpro:connect_partition_leader(Hosts, ConnCfg, Topic, Partition), + fun(Pid) -> resolve_offset(Pid, Topic, Partition, Time) end). -%% @doc Resolve timestamp to real offset. +%% @doc Resolve timestamp or semantic offset to real offset. +%% The give pid should be the connection to partition leader broker. -spec resolve_offset(pid(), topic(), partition(), offset_time()) -> {ok, offset()} | {error, any()}. resolve_offset(Pid, Topic, Partition, Time) -> - Request = brod_kafka_request:offsets_request(Pid, Topic, Partition, Time), - #kpro_rsp{tag = offsets_response - , vsn = Vsn - , msg = Msg - } = request_sync(Pid, Request), - [Response] = kf(responses, Msg), - [PartitionRespons] = kf(partition_responses, Response), - Ec = kf(error_code, PartitionRespons), - ?IS_ERROR(Ec) andalso erlang:throw(Ec), - case Vsn of - 0 -> - case kf(offsets, PartitionRespons) of - [Offset] -> {ok, Offset}; - [] -> {error, not_found} - end; - 1 -> - {ok, kf(offset, PartitionRespons)} + Req = brod_kafka_request:list_offsets(Pid, Topic, Partition, Time), + case request_sync(Pid, Req) of + {ok, #{error_code := EC}} when ?IS_ERROR(EC) -> + {error, EC}; + {ok, #{offset := Offset}} -> + {ok, Offset}; + {error, Reason} -> + {error, Reason} end. -%% @doc Try connect to any of the given bootstrap nodes. --spec try_connect([endpoint()]) -> {ok, pid()} | {error, any()}. -try_connect(Hosts) -> - try_connect(Hosts, [], ?undef). - -%% @doc Try connect to any of the given bootstrap nodes using -%% the given connect options. -%% @end -try_connect(Hosts, Options) -> - try_connect(Hosts, Options, ?undef). - %% @doc Check terminate reason for a gen_server implementation is_normal_reason(normal) -> true; is_normal_reason(shutdown) -> true; @@ -153,24 +121,6 @@ is_normal_reason(_) -> false. is_pid_alive(Pid) -> is_pid(Pid) andalso is_process_alive(Pid). -shutdown_pid(Pid) -> - case is_pid_alive(Pid) of - true -> exit(Pid, shutdown); - false -> ok - end. - -%% @doc Find leader broker ID for the given topic-partiton in -%% the metadata response received from socket. -%% @end --spec find_leader_in_metadata(kpro:struct(), topic(), partition()) -> - {ok, endpoint()} | {error, any()}. -find_leader_in_metadata(Metadata, Topic, Partition) -> - try - {ok, do_find_leader_in_metadata(Metadata, Topic, Partition)} - catch throw : Reason -> - {error, Reason} - end. - %% @doc Get now timestamp, and format as UTC string. -spec os_time_utc_str() -> string(). os_time_utc_str() -> @@ -191,7 +141,6 @@ epoch_ms() -> %% NOTE: keep making MFA calls to error_logger to %% 1. allow logging libraries such as larger parse_transform %% 2. be more xref friendly -%% @end -spec log(info | warning | error, string(), [any()]) -> ok. log(info, Fmt, Args) -> error_logger:info_msg(Fmt, Args); log(warning, Fmt, Args) -> error_logger:warning_msg(Fmt, Args); @@ -219,162 +168,128 @@ assert_topics(Topics) -> %% @doc Assert topic is a binary(). -spec assert_topic(topic()) -> ok | no_return(). assert_topic(Topic) -> - ok_when(is_binary(Topic) andalso size(Topic) > 0, - {bad_topic, Topic}). - -%% @doc Map message to brod's format. -%% incomplete message indicator is kept when the only one message is incomplete. -%% Messages having offset earlier than the requested offset are discarded. -%% this might happen for compressed message sets -%% @end --spec decode_messages(offset(), kpro:incomplete_message() | [brod:message()]) -> - kpro:incomplete_message() | [brod:message()]. -decode_messages(BeginOffset, Messages) when is_binary(Messages) -> - decode_messages(BeginOffset, kpro:decode_message_set(Messages)); -decode_messages(_BeginOffset, ?incomplete_message(_) = Incomplete) -> - Incomplete; -decode_messages(BeginOffset, Messages) when is_list(Messages) -> - drop_old_messages(BeginOffset, Messages). + ok_when(is_binary(Topic) andalso size(Topic) > 0, {bad_topic, Topic}). + +%% @doc Make a flat message list from decoded batch list. +-spec flatten_batches(offset(), [kpro:batch()]) -> [kpro:message()]. +flatten_batches(BeginOffset, Batches) -> + MsgList = lists:append([Msgs || {_Meta, Msgs} <- Batches]), + drop_old_messages(BeginOffset, MsgList). %% @doc Fetch a single message set from the given topic-partition. --spec fetch([endpoint()], topic(), partition(), offset(), - non_neg_integer(), non_neg_integer(), pos_integer(), - sock_opts()) -> {ok, [brod:message()]} | {error, any()}. -fetch(Hosts, Topic, Partition, Offset, WaitTime, - MinBytes, MaxBytes, Options) -> - with_sock( - brod:connect_leader(Hosts, Topic, Partition, Options), - fun(Pid) -> - Fetch = make_fetch_fun(Pid, Topic, Partition, - WaitTime, MinBytes, MaxBytes), - Fetch(Offset) - end). +-spec fetch(connection() | {[endpoint()], conn_config()}, + topic(), partition(), offset(), brod:fetch_opts()) -> + {ok, [brod:message()]} | {error, any()}. +fetch({Hosts, ConnCfg}, Topic, Partition, Offset, Opts) -> + with_conn( + kpro:connect_partition_leader(Hosts, ConnCfg, Topic, Partition), + fun(Conn) -> fetch(Conn, Topic, Partition, Offset, Opts) end); +fetch(Conn, Topic, Partition, Offset, Opts) -> + Fetch = make_fetch_fun(Conn, Topic, Partition, Opts), + Fetch(Offset). %% @doc Make a fetch function which should expand `max_bytes' when %% it is not big enough to fetch one signle message. -%% @end --spec make_fetch_fun(pid(), topic(), partition(), kpro:wait(), - kpro:count(), kpro:count()) -> fetch_fun(). -make_fetch_fun(SockPid, Topic, Partition, WaitTime, MinBytes, MaxBytes) -> - ReqFun = make_req_fun(SockPid, Topic, Partition, WaitTime, MinBytes), - fun(Offset) -> fetch(SockPid, ReqFun, Offset, MaxBytes) end. - -%% @doc Get sasl options from client config. --spec get_sasl_opt(brod:client_config()) -> sasl_opt(). -get_sasl_opt(Config) -> - case proplists:get_value(sasl, Config) of - {plain, User, PassFun} when is_function(PassFun) -> - {plain, User, PassFun()}; - {plain, File} -> - {User, Pass} = read_sasl_file(File), - {plain, User, Pass}; - Other -> - Other - end. +-spec make_fetch_fun(pid(), topic(), partition(), brod:fetch_opts()) -> + fetch_fun(). +make_fetch_fun(Conn, Topic, Partition, FetchOpts) -> + WaitTime = maps:get(max_wait_time, FetchOpts, 1000), + MinBytes = maps:get(min_bytes, FetchOpts, 1), + MaxBytes = maps:get(max_bytes, FetchOpts, 1 bsl 20), + ReqFun = make_req_fun(Conn, Topic, Partition, WaitTime, MinBytes), + fun(Offset) -> fetch(Conn, ReqFun, Offset, MaxBytes) end. %% @doc Hide sasl plain password in an anonymous function to avoid %% the plain text being dumped to crash logs -%% @end -spec init_sasl_opt(brod:client_config()) -> brod:client_config(). init_sasl_opt(Config) -> case get_sasl_opt(Config) of - {plain, User, Pass} when not is_function(Pass) -> - replace_prop(sasl, {plain, User, fun() -> Pass end}, Config); + {Mechanism, User, Pass} when Mechanism =/= callback -> + replace_prop(sasl, {Mechanism, User, fun() -> Pass end}, Config); _Other -> Config end. -%% @doc Fetch ommitted offsets for the given topics in a consumer group. +%% @doc Fetch committed offsets for the given topics in a consumer group. %% 1. try find out the group coordinator broker from the bootstrap hosts -%% 2. send `offset_fetch_request' and wait for `offset_fetch_response' +%% 2. send `offset_fetch' request and wait for response. %% If Topics is an empty list, fetch offsets for all topics in the group -%% @end --spec fetch_committed_offsets([endpoint()], sock_opts(), +-spec fetch_committed_offsets([endpoint()], conn_config(), group_id(), [topic()]) -> {ok, [kpro:struct()]} | {error, any()}. -fetch_committed_offsets(BootstrapEndpoints, SockOpts, GroupId, Topics) -> - with_sock( - connect_group_coordinator(BootstrapEndpoints, SockOpts, GroupId), +fetch_committed_offsets(BootstrapEndpoints, ConnCfg, GroupId, Topics) -> + Args = #{type => group, id => GroupId}, + with_conn( + kpro:connect_coordinator(BootstrapEndpoints, ConnCfg, Args), fun(Pid) -> do_fetch_committed_offsets(Pid, GroupId, Topics) end). %% @doc Fetch commited offsts for the given topics in a consumer group. -%% 1. locate the group coordinator broker by calling +%% 1. Get broker endpoint by calling %% `brod_client:get_group_coordinator' -%% 2. connect group coordinator broker -%% 3. send `offset_fetch_request' and wait for `offset_fetch_response' +%% 2. Establish a connecton to the discovered endpoint. +%% 3. send `offset_fetch' request and wait for response. %% If Topics is an empty list, fetch offsets for all topics in the group -%% @end -spec fetch_committed_offsets(brod:client(), group_id(), [topic()]) -> {ok, [kpro:struct()]} | {error, any()}. fetch_committed_offsets(Client, GroupId, Topics) -> - with_sock( - connect_group_coordinator(Client, GroupId), - fun(Pid) -> do_fetch_committed_offsets(Pid, GroupId, Topics) end). + case brod_client:get_group_coordinator(Client, GroupId) of + {ok, {Endpoint, ConnCfg}} -> + case kpro:connect(Endpoint, ConnCfg) of + {ok, Conn} -> + do_fetch_committed_offsets(Conn, GroupId, Topics); + {error, Reason} -> + {error, Reason} + end; + {error, Reason} -> + {error, Reason} + end. -%%%_* Internal Functions ======================================================= +-spec get_sasl_opt(brod:client_config()) -> sasl_opt(). +get_sasl_opt(Config) -> + case proplists:get_value(sasl, Config) of + ?undef -> {sasl, ?undef}; + {callback, Module, Args} -> + %% Module should implement kpro_auth_backend behaviour + {callback, Module, Args}; + {Mechanism, File} when is_list(File) orelse is_binary(File) -> + {User, Pass} = read_sasl_file(File), + {Mechanism, User, Pass}; + Other -> + Other + end. -%% @private With a socket connected to the group coordinator broker, send -%% `offset_fetch_request' and wait for `offset_fetch_response' -%% @end +%% With a connection to the group coordinator broker, +%% send `offset_fetch' and wait for response. -spec do_fetch_committed_offsets(brod:client_id() | pid(), group_id(), [topic()]) -> {ok, [kpro:struct()]} | {error, any()}. -do_fetch_committed_offsets(SockPid, GroupId, Topics) when is_pid(SockPid) -> - Req = brod_kafka_request:offset_fetch_request(SockPid, GroupId, Topics), - try - #kpro_rsp{ tag = offset_fetch_response - , msg = Msg - } = request_sync(SockPid, Req), - TopicsArray = kf(responses, Msg), - {ok, TopicsArray} - catch - throw : Reason -> +do_fetch_committed_offsets(Conn, GroupId, Topics) when is_pid(Conn) -> + Req = brod_kafka_request:offset_fetch(Conn, GroupId, Topics), + case request_sync(Conn, Req) of + {ok, Msg} -> + {ok, kf(responses, Msg)}; + {error, Reason} -> {error, Reason} end. -%% @private Make a function to build fetch requests. -%% The function takes offset and max_bytes as input as these two parameters -%% are varient when continuously polling a specific topic-partition. -%% @end --spec make_req_fun(pid(), topic(), partition(), - kpro:wait(), kpro:count()) -> req_fun(). -make_req_fun(SockPid, Topic, Partition, WaitTime, MinBytes) -> - fun(Offset, MaxBytes) -> - brod_kafka_request:fetch_request(SockPid, Topic, Partition, Offset, - WaitTime, MinBytes, MaxBytes) - end. - %% @doc Fetch a message-set. If the given MaxBytes is not enough to fetch a %% single message, expand it to fetch exactly one message -%% @end --spec fetch(pid(), req_fun(), offset(), kpro:count()) -> +-spec fetch(connection(), req_fun(), offset(), kpro:count()) -> {ok, [brod:message()]} | {error, any()}. -fetch(SockPid, ReqFun, Offset, MaxBytes) when is_pid(SockPid) -> +fetch(Conn, ReqFun, Offset, MaxBytes) -> Request = ReqFun(Offset, MaxBytes), - #kpro_rsp{ tag = fetch_response - , msg = Msg - } = request_sync(SockPid, Request, infinity), - [Response] = kf(responses, Msg), - [PartitionResponse] = kf(partition_responses, Response), - Header = kf(partition_header, PartitionResponse), - Messages0 = kf(record_set, PartitionResponse), - ErrorCode = kf(error_code, Header), - case ?IS_ERROR(ErrorCode) of - true -> - {error, kpro_error_code:desc(ErrorCode)}; - false -> - case decode_messages(Offset, Messages0) of - ?incomplete_message(Size) -> - fetch(SockPid, ReqFun, Offset, Size); - Messages -> - {ok, Messages} - end + case request_sync(Conn, Request, infinity) of + {ok, #{error_code := ErrorCode}} when ?IS_ERROR(ErrorCode) -> + {error, ErrorCode}; + {ok, #{batches := Batches}} -> + {ok, flatten_batches(Offset, Batches)}; + {error, Reason} -> + {error, Reason} end. %% @doc List all groups in the given cluster. %% NOTE: Exception if failed against any of the coordinator brokers. -%% @end --spec list_all_groups([endpoint()], sock_opts()) -> +-spec list_all_groups([endpoint()], conn_config()) -> [{endpoint(), [brod:cg()] | {error, any()}}]. list_all_groups(Endpoints, Options) -> {ok, Metadata} = get_metadata(Endpoints, [], Options), @@ -389,24 +304,14 @@ list_all_groups(Endpoints, Options) -> end, [], Brokers). %% @doc List all groups in the given coordinator broker. --spec list_groups(endpoint(), sock_opts()) -> +-spec list_groups(endpoint(), conn_config()) -> {ok, [brod:cg()]} | {error, any()}. -list_groups(Endpoint, Options) -> - with_sock( - try_connect([Endpoint], Options), +list_groups(Endpoint, ConnCfg) -> + with_conn([Endpoint], ConnCfg, fun(Pid) -> - Vsn = 0, %% only one version - Body = [], %% this request has no struct field - Request = kpro:req(list_groups_request, Vsn, Body), - #kpro_rsp{ tag = list_groups_response - , vsn = Vsn - , msg = Msg - } = request_sync(Pid, Request), - ErrorCode = kf(error_code, Msg), - case ?IS_ERROR(ErrorCode) of - true -> - {error, ErrorCode}; - false -> + Request = brod_kafka_request:list_groups(Pid), + case request_sync(Pid, Request) of + {ok, Groups0} -> Groups = lists:map( fun(Struct) -> @@ -415,80 +320,28 @@ list_groups(Endpoint, Options) -> #brod_cg{ id = Id , protocol_type = Type } - end, kf(groups, Msg)), - {ok, Groups} + end, Groups0), + {ok, Groups}; + {error, Reason} -> + {error, Reason} end end). %% @doc Send describe_groups_request and wait for describe_groups_response. --spec describe_groups(endpoint(), sock_opts(), [brod:group_id()]) -> +-spec describe_groups(endpoint(), conn_config(), [brod:group_id()]) -> {ok, kpro:struct()} | {error, any()}. -describe_groups(Coordinator, SockOpts, IDs) -> - with_sock( - try_connect([Coordinator], SockOpts), +describe_groups(CoordinatorEndpoint, ConnCfg, IDs) -> + with_conn([CoordinatorEndpoint], ConnCfg, fun(Pid) -> - Req = kpro:req(describe_groups_request, 0, [{group_ids, IDs}]), - #kpro_rsp{ tag = describe_groups_response - , vsn = 0 - , msg = Msg - } = request_sync(Pid, Req), - Groups = kf(groups, Msg), - {ok, Groups} - end). - -%% @doc Connect to consumer group coordinator broker. -%% Done in steps: 1) connect to any of the given bootstrap ednpoints; -%% 2) send `group_coordinator_request' to resolve group coordinator endpoint; -%% 3) connect to the resolved endpoint and return the `brod_sock' pid -%% @end --spec connect_group_coordinator([endpoint()], sock_opts(), group_id()) -> - {ok, pid()} | {error, any()}. -connect_group_coordinator(BootstrapEndpoints, SockOpts, GroupId) -> - case resolve_group_coordinator(BootstrapEndpoints, SockOpts, GroupId) of - {ok, Endpoint} -> try_connect([Endpoint], SockOpts); - {error, Reason} -> {error, Reason} - end. - -%% @doc Connect to consumer group coordinator broker. -%% Done in steps: 1) make use of `brod_client' metadata socket to resolve -%% group coordinator broker endpoint, 2) connect to the resolved endpoint -%% and return the `brod_sock' pid --spec connect_group_coordinator(brod:client(), brod:group_id()) -> - {ok, pid()} | {error, any()}. -connect_group_coordinator(Client, GroupId) -> - case brod_client:get_group_coordinator(Client, GroupId) of - {ok, {Endpoint, SockOpts}} -> try_connect([Endpoint], SockOpts); - {error, Reason} -> {error, Reason} - end. - -%% @doc Send group_coordinator_request to any of the bootstrap endpoints. -%% return resolved coordinator broker endpoint. -%% @end --spec resolve_group_coordinator([endpoint()], sock_opts(), group_id()) -> - {ok, endpoint()} | {error, any()}. -resolve_group_coordinator(BootstrapEndpoints, SockOpts, GroupId) -> - with_sock( - try_connect(BootstrapEndpoints, SockOpts), - fun(BootstrapSockPid) -> - Req = kpro:req(group_coordinator_request, 0, [{group_id, GroupId}]), - #kpro_rsp{ tag = group_coordinator_response - , vsn = 0 - , msg = Struct - } = request_sync(BootstrapSockPid, Req), - EC = kf(error_code, Struct), - ?IS_ERROR(EC) andalso erlang:throw(EC), - Coordinator = kf(coordinator, Struct), - Host = kf(host, Coordinator), - Port = kf(port, Coordinator), - {ok, {binary_to_list(Host), Port}} + Req = kpro:make_request(describe_groups, 0, [{group_ids, IDs}]), + request_sync(Pid, Req) end). -define(IS_BYTE(I), (I>=0 andalso I<256)). %% @doc Return message set size in number of bytes. %% NOTE: This does not include the overheads of encoding protocol. -%% such as magic bytes, attributes, and length tags etc. -%% @end +%% such as magic bytes, attributes, and length tags etc. -spec bytes(brod:key() | brod:value() | brod:kv_list()) -> non_neg_integer(). bytes([]) -> 0; bytes(?undef) -> 0; @@ -504,12 +357,12 @@ kv_count(KVList) -> Fold = fun(_Msg, Acc) -> Acc + 1 end, foldl_kvlist(Fold, 0, KVList). -%% @private Get nested kv-list. +%% Get nested kv-list. nested({_K, [Msg | _] = Nested}) when is_tuple(Msg) -> Nested; nested({_T, _K, [Msg | _] = Nested}) when is_tuple(Msg) -> Nested; nested(_) -> false. -%% @private Foldl kv-list. +%% Foldl kv-list. foldl_kvlist(_Fun, Acc, []) -> Acc; foldl_kvlist(Fun, Acc, [Msg | Rest]) -> NewAcc = case nested(Msg) of @@ -533,32 +386,99 @@ group_per_key(List) -> group_per_key(MapFun, List) -> group_per_key(lists:map(MapFun, List)). +%% @doc Parse decoded kafka response (`#kpro_rsp{}') into a more generic +%% representation. +%% Return `ok' if it is a trivial 'ok or not' response without data fields +%% Return `{ok, Result}' for some of the APIs when no error-code found in +%% response. Result could be a transformed representation of response message +%% body `#kpro_rsp.msg' or the response body itself. +%% For some APIs, it returns `{error, CodeOrMessage}' when error-code is not +%% `no_error' in the message body. +%% NOTE: Not all error codes are interpreted as `{error, CodeOrMessage}' tuple. +%% for some of the complex response bodies, error-codes are retained +%% for caller to parse. +-spec parse_rsp(kpro:rsp()) -> ok | {ok, term()} | {error, any()}. +parse_rsp(#kpro_rsp{api = API, vsn = Vsn, msg = Msg}) -> + try parse(API, Vsn, Msg) of + ok -> ok; + Result -> {ok, Result} + catch + throw : ErrorCodeOrMessage -> + {error, ErrorCodeOrMessage} + end. + +-spec request_sync(connection(), kpro:req()) -> + ok | {ok, term()} | {error, any()}. +request_sync(Conn, Req) -> + request_sync(Conn, Req, infinity). + +-spec request_sync(connection(), kpro:req(), infinity | timeout()) -> + ok | {ok, term()} | {error, any()}. +request_sync(Conn, #kpro_req{ref = Ref} = Req, Timeout) -> + % kpro_connection has a global 'request_timeout' option + % the connection pid will exit if that one times out + case kpro:request_sync(Conn, Req, Timeout) of + {ok, #kpro_rsp{ref = Ref} = Rsp} -> parse_rsp(Rsp); + {error, Reason} -> {error, Reason} + end. + %%%_* Internal functions ======================================================= -%% @private -with_sock({ok, Pid}, Fun) -> - try - Fun(Pid) - catch - throw : Reason -> - {error, Reason} - after - _ = brod_sock:stop(Pid) - end; -with_sock({error, Reason}, _Fun) -> - {error, Reason}. +%% Make a function to build fetch requests. +%% The function takes offset and max_bytes as input as these two parameters +%% are varient when continuously polling a specific topic-partition. +-spec make_req_fun(connection(), topic(), partition(), + kpro:wait(), kpro:count()) -> req_fun(). +make_req_fun(Conn, Topic, Partition, WaitTime, MinBytes) -> + fun(Offset, MaxBytes) -> + brod_kafka_request:fetch(Conn, Topic, Partition, Offset, + WaitTime, MinBytes, MaxBytes) + end. + +%% Parse fetch response into a more user-friendly representation. +-spec parse_fetch_rsp(kpro:struct()) -> map(). +parse_fetch_rsp(Msg) -> + EC1 = kpro:find(error_code, Msg, ?no_error), + SessionID = kpro:find(session_id, Msg, 0), + {Header, Batches, EC2} = + case kpro:find(responses, Msg) of + [] -> + %% a session init without data + {undefined, [], ?no_error}; + _ -> + PartitionRsp = get_partition_rsp(Msg), + HeaderX = kpro:find(partition_header, PartitionRsp), + throw_error_code([HeaderX]), + Records = kpro:find(record_set, PartitionRsp), + ECx = kpro:find(error_code, HeaderX), + {HeaderX, kpro:decode_batches(Records), ECx} + end, + ErrorCode = case EC2 =:= ?no_error of + true -> EC1; + false -> EC2 + end, + case ?IS_ERROR(ErrorCode) of + true -> erlang:throw(ErrorCode); + false -> #{ session_id => SessionID + , header => Header + , batches => Batches + } + end. + +get_partition_rsp(Struct) -> + [TopicRsp] = kpro:find(responses, Struct), + [PartitionRsp] = kpro:find(partition_responses, TopicRsp), + PartitionRsp. -%% @private -spec replace_prop(term(), term(), proplists:proplist()) -> proplists:proplist(). replace_prop(Key, Value, PropL0) -> PropL = proplists:delete(Key, PropL0), [{Key, Value} | PropL]. -%% @private Read a regular file, assume it has two lines: +%% Read a regular file, assume it has two lines: %% First line is the sasl-plain username %% Second line is the password -%% @end -spec read_sasl_file(file:name_all()) -> {binary(), binary()}. read_sasl_file(File) -> {ok, Bin} = file:read_file(File), @@ -566,28 +486,9 @@ read_sasl_file(File) -> [User, Pass] = lists:filter(fun(Line) -> Line =/= <<>> end, Lines), {User, Pass}. -%% @private Try to connect to one of the given endpoints. -%% Try next in the list if failed. Return the last failure reason -%% if failed on all endpoints. -%% @end --spec try_connect([endpoint()], sock_opts(), any()) -> - {ok, pid()} | {error, any()}. -try_connect([], _Options, LastError) -> - LastError; -try_connect([{Host, Port} | Hosts], Options, _) -> - %% Do not 'start_link' to avoid unexpected 'EXIT' message. - %% Should be ok since we're using a single blocking request which - %% monitors the process anyway. - case brod_sock:start(self(), Host, Port, - ?BROD_DEFAULT_CLIENT_ID, Options) of - {ok, Pid} -> {ok, Pid}; - Error -> try_connect(Hosts, Options, Error) - end. - -%% @private A fetched batch may contain offsets earlier than the +%% A fetched batch may contain offsets earlier than the %% requested begin-offset because the batch might be compressed on %% kafka side. Here we drop the leading messages. -%% @end drop_old_messages(_BeginOffset, []) -> []; drop_old_messages(BeginOffset, [Message | Rest] = All) -> case Message#kafka_message.offset < BeginOffset of @@ -595,63 +496,116 @@ drop_old_messages(BeginOffset, [Message | Rest] = All) -> false -> All end. -%% @private Raise an 'error' exception when first argument is not 'true'. +%% Raise an 'error' exception when first argument is not 'true'. %% The second argument is used as error reason. -%% @end -spec ok_when(boolean(), any()) -> ok | no_return(). ok_when(true, _) -> ok; ok_when(_, Reason) -> erlang:error(Reason). -%% @private --spec do_find_leader_in_metadata(kpro:struct(), brod:topic(), - brod:partition()) -> brod:endpoint(). -do_find_leader_in_metadata(Msg, Topic, Partition) -> - Brokers = kf(brokers, Msg), - [TopicMetadata] = kf(topic_metadata, Msg), - TopicEC = kf(topic_error_code, TopicMetadata), - RealTopic = kf(topic, TopicMetadata), - Partitions = kf(partition_metadata, TopicMetadata), - RealTopic /= Topic andalso erlang:throw(?EC_UNKNOWN_TOPIC_OR_PARTITION), - ?IS_ERROR(TopicEC) andalso erlang:throw(TopicEC), - Id = case find_struct(partition_id, Partition, Partitions) of - false -> erlang:throw(?EC_UNKNOWN_TOPIC_OR_PARTITION); - PartitionMetadata -> kf(leader, PartitionMetadata) - end, - Id >= 0 orelse erlang:throw(?EC_LEADER_NOT_AVAILABLE), - Broker = find_struct(node_id, Id, Brokers), - Host = kf(host, Broker), - Port = kf(port, Broker), - {binary_to_list(Host), Port}. - -%% @private -spec kf(kpro:field_name(), kpro:struct()) -> kpro:field_value(). kf(FieldName, Struct) -> kpro:find(FieldName, Struct). -%% @private Find kpro struct in array. -%% Return false if no struct matches the given field name and value -%% @end --spec find_struct(kpro:field_name(), kpro:field_value(), [kpro:struct()]) -> - false | kpro:struct(). -find_struct(_FieldName, _Value, []) -> false; -find_struct(FieldName, Value, [Struct | Rest]) -> - case kf(FieldName, Struct) =:= Value of - true -> Struct; - false -> find_struct(FieldName, Value, Rest) - end. +with_conn({ok, Pid}, Fun) -> + try + Fun(Pid) + after + kpro:close_connection(Pid) + end; +with_conn({error, Reason}, _Fun) -> + {error, Reason}. -%% @private --spec request_sync(pid(), kpro:req()) -> kpro:rsp(). -request_sync(Pid, Req) -> - request_sync(Pid, Req, infinity). - -%% @private --spec request_sync(pid(), kpro:req(), timeout()) -> kpro:rsp(). -request_sync(Pid, Req, Timeout) -> - % brod_sock has a global 'request_timeout' option - % the socket pid will exit if that one times out - case brod_sock:request_sync(Pid, Req, Timeout) of - {ok, Rsp} -> Rsp; - {error, Reason} -> erlang:throw(Reason) +with_conn(Endpoints, ConnCfg, Fun) when is_list(ConnCfg) -> + with_conn(Endpoints, maps:from_list(ConnCfg), Fun); +with_conn(Endpoints, ConnCfg, Fun) -> + kpro_brokers:with_connection(Endpoints, ConnCfg, Fun). + +parse(produce, _Vsn, Msg) -> + kpro:find(base_offset, get_partition_rsp(Msg)); +parse(fetch, _Vsn, Msg) -> + parse_fetch_rsp(Msg); +parse(list_offsets, _, Msg) -> + case get_partition_rsp(Msg) of + #{offsets := [Offset]} = M -> M#{offset => Offset}; + #{offset := _} = M -> M + end; +parse(metadata, _, Msg) -> + ok = throw_error_code(kpro:find(topic_metadata, Msg)), + Msg; +parse(find_coordinator, _, Msg) -> + ok = throw_error_code([Msg]), + Msg; +parse(join_group, _, Msg) -> + ok = throw_error_code([Msg]), + Msg; +parse(heartbeat, _, Msg) -> + ok = throw_error_code([Msg]), + Msg; +parse(leave_group, _, Msg) -> + ok = throw_error_code([Msg]); +parse(sync_group, _, Msg) -> + ok = throw_error_code([Msg]), + Msg; +parse(describe_groups, _, Msg) -> + %% return groups + Groups = kpro:find(groups, Msg), + ok = throw_error_code(Groups), + Groups; +parse(list_groups, _, Msg) -> + %% return groups + ok = throw_error_code([Msg]), + kpro:find(groups, Msg); +parse(create_topics, _, Msg) -> + ok = throw_error_code(kpro:find(topic_errors, Msg)); +parse(delete_topics, _, Msg) -> + ok = throw_error_code(kpro:find(topic_error_codes, Msg)); +parse(init_producer_id, _, Msg) -> + ok = throw_error_code([Msg]), + Msg; +parse(create_partitions, _, Msg) -> + ok = throw_error_code(kpro:find(topic_errors, Msg)); +parse(end_txn, _, Msg) -> + ok = throw_error_code([Msg]); +parse(describe_acls, _, Msg) -> + ok = throw_error_code([Msg]), + Msg; +parse(create_acls, _, Msg) -> + ok = throw_error_code(kpro:find(creation_responses, Msg)); +parse(_API, _Vsn, Msg) -> + %% leave it to the caller to parse: + %% offset_commit + %% offset_fetch + %% sasl_handshake + %% api_versions + %% delete_records + %% add_partitions_to_txn + %% txn_offset_commit + %% delete_acls + %% describe_acls + %% describe_configs + %% alter_configs + %% alter_replica_log_dirs + %% describe_log_dirs + %% sasl_authenticate + %% create_partitions + %% create_delegation_token + %% renew_delegation_token + %% expire_delegation_token + %% describe_delegation_token + %% delete_groups + Msg. + +%% This function takes a list of kpro structs, +%% return ok if all structs have 'no_error' as error code. +%% Otherwise throw an exception with the first error. +throw_error_code([]) -> ok; +throw_error_code([Struct | Structs]) -> + EC = kpro:find(error_code, Struct), + case ?IS_ERROR(EC) of + true -> + Err = kpro:find(error_message, Struct, EC), + erlang:throw(Err); + false -> + throw_error_code(Structs) end. %%%_* Emacs ==================================================================== diff --git a/test/brod_cg_commits_SUITE.erl b/test/brod_cg_commits_SUITE.erl index bd482590..1a1c22b7 100644 --- a/test/brod_cg_commits_SUITE.erl +++ b/test/brod_cg_commits_SUITE.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2017, Klarna AB +%%% Copyright (c) 2017-2018, Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -76,13 +76,13 @@ t_set_then_reset(Config) when is_list(Config) -> Offsets0 = [{0, 0}, {1, 0}, {2, 0}], ok = do_commit(Topic, Offsets0), {ok, Rsp0} = - brod_utils:fetch_committed_offsets(?BROKERS, [], ?GROUP_ID, + brod_utils:fetch_committed_offsets(?BROKERS, #{}, ?GROUP_ID, [{Topic, Partitions}]), ok = assert_offsets([{Topic, Offsets0}], Rsp0), Offsets1 = [{0, 1}, {1, 1}, {2, 1}], ok = do_commit(Topic, Offsets1), {ok, Rsp1} = - brod_utils:fetch_committed_offsets(?BROKERS, [], ?GROUP_ID, + brod_utils:fetch_committed_offsets(?BROKERS, #{}, ?GROUP_ID, [{Topic, Partitions}]), ok = assert_offsets([{Topic, Offsets1}], Rsp1), ok. diff --git a/test/brod_cli_pipe_tests.erl b/test/brod_cli_pipe_tests.erl index 3bb0bcc2..a8800c2c 100644 --- a/test/brod_cli_pipe_tests.erl +++ b/test/brod_cli_pipe_tests.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2017, Klarna AB +%%% Copyright (c) 2017-2018, Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. diff --git a/test/brod_cli_tests.erl b/test/brod_cli_tests.erl index 83a89b80..2135ccc9 100644 --- a/test/brod_cli_tests.erl +++ b/test/brod_cli_tests.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2017, Klarna AB +%%% Copyright (c) 2017-2018, Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. diff --git a/test/brod_client_SUITE.erl b/test/brod_client_SUITE.erl index df49e3cc..ed00772f 100644 --- a/test/brod_client_SUITE.erl +++ b/test/brod_client_SUITE.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2015-2017, Klarna AB +%%% Copyright (c) 2015-2018, Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -14,12 +14,6 @@ %%% limitations under the License. %%% -%%%============================================================================= -%%% @doc -%%% @copyright 2015 Klarna AB -%%% @end -%%% ============================================================================ - %% @private -module(brod_client_SUITE). @@ -38,8 +32,8 @@ -export([ t_skip_unreachable_endpoint/1 , t_no_reachable_endpoint/1 , t_call_bad_client_id/1 - , t_metadata_socket_restart/1 - , t_payload_socket_restart/1 + , t_metadata_connection_restart/1 + , t_payload_connection_restart/1 , t_auto_start_producers/1 , t_auto_start_producer_for_unknown_topic/1 , t_ssl/1 @@ -54,8 +48,8 @@ -define(HOST, "localhost"). -define(HOSTS, [{?HOST, 9092}]). --define(HOSTS_SSL, [{?HOST, 9192}]). --define(HOSTS_SASL_SSL, [{?HOST, 9292}]). +-define(HOSTS_SSL, [{?HOST, 9093}]). +-define(HOSTS_SASL_SSL, [{?HOST, 9094}]). -define(TOPIC, <<"brod-client-SUITE-topic">>). -define(WAIT(PATTERN, RESULT, TIMEOUT), @@ -113,7 +107,7 @@ t_skip_unreachable_endpoint(Config) when is_list(Config) -> Client = t_skip_unreachable_endpoint, ok = brod:start_client([{"localhost", 8192} | ?HOSTS], Client), _Res = brod_client:get_partitions_count(Client, <<"some-unknown-topic">>), - ?assertMatch({error, 'UnknownTopicOrPartition'}, _Res), + ?assertMatch({error, unknown_topic_or_partition}, _Res), ClientPid = whereis(Client), Ref = erlang:monitor(process, ClientPid), ok = brod:stop_client(Client), @@ -124,61 +118,62 @@ t_no_reachable_endpoint({'end', _Config}) -> brod:stop_client(t_no_reachable_endpoint); t_no_reachable_endpoint(Config) when is_list(Config) -> Client = t_no_reachable_endpoint, - ok = brod:start_client([{"badhost", 9092}], Client), + Endpoint = {"badhost", 9092}, + ok = brod:start_client([Endpoint], Client), ClientPid = whereis(Client), Mref = erlang:monitor(process, ClientPid), - Reason = ?WAIT({'DOWN', Mref, process, ClientPid, Reason_}, Reason_, 1000), - ?assertMatch({{{connection_failure, nxdomain}, _Hosts}, _Stacktrace}, Reason). + Reason = ?WAIT({'DOWN', Mref, process, ClientPid, ReasonX}, ReasonX, 1000), + ?assertMatch([{Endpoint, {nxdomain, _Stack}}], Reason). t_call_bad_client_id(Config) when is_list(Config) -> %% call a bad client ID Res = brod:produce(?undef, <<"topic">>, _Partition = 0, <<"k">>, <<"v">>), ?assertEqual({error, client_down}, Res). -t_metadata_socket_restart({init, Config}) -> - meck:new(brod_sock, [passthrough, no_passthrough_cover, no_history]), +t_metadata_connection_restart({init, Config}) -> + meck:new(kpro_connection, [passthrough, no_passthrough_cover, no_history]), Config; -t_metadata_socket_restart({'end', Config}) -> - brod:stop_client(t_metadata_socket_restart), - meck:validate(brod_sock), - meck:unload(brod_sock), +t_metadata_connection_restart({'end', Config}) -> + brod:stop_client(t_metadata_connection_restart), + meck:validate(kpro_connection), + meck:unload(kpro_connection), Config; -t_metadata_socket_restart(Config) when is_list(Config) -> - Ref = mock_brod_sock(), - Client = t_metadata_socket_restart, +t_metadata_connection_restart(Config) when is_list(Config) -> + Ref = mock_connection(hd(?HOSTS)), + Client = t_metadata_connection_restart, ok = brod:start_client(?HOSTS, Client), ClientPid = whereis(Client), - SocketPid = ?WAIT({socket_started, Ref, Pid}, Pid, 5000), + Connection = ?WAIT({connection_pid, Ref, Pid}, Pid, 5000), ?assert(is_process_alive(ClientPid)), - ?assert(is_process_alive(SocketPid)), - %% kill the brod_sock pid - MRef = erlang:monitor(process, SocketPid), - exit(SocketPid, kill), - ?WAIT({'DOWN', MRef, process, SocketPid, Reason_}, Reason_, 5000), - %% query metadata to trigger reconnect - {ok, _} = brod_client:get_metadata(Client, ?TOPIC), - %% expect the socket pid get restarted - SocketPid2 = ?WAIT({socket_started, Ref, Pid}, Pid, 5000), + ?assert(is_process_alive(Connection)), + %% kill the connection pid + MRef = erlang:monitor(process, Connection), + exit(Connection, kill), + ?WAIT({'DOWN', MRef, process, Connection, Reason_}, Reason_, 5000), + %% trigger a metadata query + brod_client:get_metadata(Client, all), + %% expect the connection pid get restarted + Connection2 = ?WAIT({connection_pid, Ref, Pid}, Pid, 5000), ?assert(is_process_alive(ClientPid)), - ?assert(is_process_alive(SocketPid2)), + ?assert(is_process_alive(Connection2)), ok. -t_payload_socket_restart({init, Config}) -> - meck:new(brod_sock, [passthrough, no_passthrough_cover, no_history]), +t_payload_connection_restart({init, Config}) -> + meck:new(kpro_connection, [passthrough, no_passthrough_cover, no_history]), Config; -t_payload_socket_restart({'end', Config}) -> - brod:stop_client(t_payload_socket_restart), - meck:validate(brod_sock), - meck:unload(brod_sock), +t_payload_connection_restart({'end', Config}) -> + brod:stop_client(t_payload_connection_restart), + meck:validate(kpro_connection), + meck:unload(kpro_connection), Config; -t_payload_socket_restart(Config) when is_list(Config) -> - Ref = mock_brod_sock(), +t_payload_connection_restart(Config) when is_list(Config) -> + Ref = mock_connection(hd(?HOSTS)), CooldownSecs = 2, ProducerRestartDelay = 1, ClientConfig = [{reconnect_cool_down_seconds, CooldownSecs}], - Client = t_payload_socket_restart, + Client = t_payload_connection_restart, ok = brod:start_client(?HOSTS, Client, ClientConfig), - ?WAIT({socket_started, Ref, _MetadataSocket}, ok, 5000), + ?WAIT({connection_pid, Ref, _MetadataConnection}, ok, 5000), ProducerConfig = [{partition_restart_delay_seconds, ProducerRestartDelay}, {max_retries, 0}], ok = brod:start_producer(Client, ?TOPIC, ProducerConfig), @@ -188,8 +183,8 @@ t_payload_socket_restart(Config) when is_list(Config) -> end, %% producing data should trigger a payload connection to be established ok = ProduceFun(), - %% the socket pid should have already delivered to self() mail box - PayloadSock = ?WAIT({socket_started, Ref, Pid}, Pid, 0), + %% the connection pid should have already delivered to self() mail box + PayloadSock = ?WAIT({connection_pid, Ref, Pid}, Pid, 0), %% spawn a writer which keeps retrying to produce data to partition 0 %% and report the produce_sync return value changes @@ -204,7 +199,7 @@ t_payload_socket_restart(Config) when is_list(Config) -> exit(PayloadSock, kill), %% now the writer should have {error, _} returned from produce API ?WAIT({WriterPid, {produce_result, {error, _}}}, ok, 1000), - ?WAIT({socket_started, Ref, Pid_}, Pid_, 4000), + ?WAIT({connection_pid, Ref, Pid_}, Pid_, 4000), %% then wait for the producer to get restarted by supervisor %% and the writer process should continue working normally again. %% socket should be restarted after cooldown timeout @@ -248,10 +243,10 @@ t_auto_start_producer_for_unknown_topic(Config) when is_list(Config) -> ?assertEqual({error, {producer_not_found, Topic0, Partition}}, brod:produce_sync(Client, Topic0, Partition, <<>>, <<"v">>)), Topic1 = <<"unknown-topic">>, - ?assertEqual({error, 'UnknownTopicOrPartition'}, + ?assertEqual({error, 'unknown_topic_or_partition'}, brod:produce_sync(Client, Topic1, 0, <<>>, <<"v">>)), %% this error should hit the cache - ?assertEqual({error, 'UnknownTopicOrPartition'}, + ?assertEqual({error, 'unknown_topic_or_partition'}, brod:produce_sync(Client, Topic1, 0, <<>>, <<"v">>)), ok. @@ -273,12 +268,12 @@ t_sasl_plain_ssl({'end', Config}) -> t_sasl_plain_ssl(Config) when is_list(Config) -> ClientConfig = [ {ssl, ssl_options()} , {get_metadata_timeout_seconds, 10} - , {sasl, {plain, "alice", "alice-secret"}} + , {sasl, {plain, "alice", "ecila"}} ], produce_and_consume_message(?HOSTS_SASL_SSL, t_sasl_plain_ssl, ClientConfig). t_sasl_plain_file_ssl({init, Config}) -> - ok = file:write_file("sasl-plain-user-pass-file", "alice\nalice-secret\n"), + ok = file:write_file("sasl-plain-user-pass-file", "alice\necila\n"), Config; t_sasl_plain_file_ssl({'end', Config}) -> brod:stop_client(t_sasl_plain_file_ssl), @@ -349,23 +344,21 @@ retry_writer_loop(Parent, ProduceFun, LastResult) -> retry_writer_loop(Parent, ProduceFun, Result) end. -%% tap the call to brod_sock:start_link/5, +%% tap the call to kpro_connection:start/3 %% intercept the returned socket pid %% and send it to the test process: self() -mock_brod_sock() -> +mock_connection(EP) -> Ref = make_ref(), Tester = self(), - SocketStartLinkFun = - fun(Parent, Host, Port, ClientId, Dbg) -> - {ok, Pid} = meck:passthrough([Parent, Host, Port, ClientId, Dbg]), - %% assert the caller - ?assertEqual(Parent, whereis(ClientId)), - ct:pal("client ~p: socket to ~s:~p intercepted. pid=~p", - [ClientId, Host, Port, Pid]), - Tester ! {socket_started, Ref, Pid}, + StartFun = + fun(Host, Port, Config) -> + {ok, Pid} = meck:passthrough([Host, Port, Config]), + Tester ! {connection_pid, Ref, Pid}, {ok, Pid} end, - ok = meck:expect(brod_sock, start_link, SocketStartLinkFun), + ok = meck:expect(kpro_connection, start, StartFun), + ok = meck:expect(kpro_connection, get_endpoint, 1, {ok, EP}), + ok = meck:expect(kpro_connection, get_api_vsns, 1, {ok, ?undef}), Ref. %%%_* Emacs ==================================================================== diff --git a/test/brod_compression_SUITE.erl b/test/brod_compression_SUITE.erl index 8e04a64a..930775f9 100644 --- a/test/brod_compression_SUITE.erl +++ b/test/brod_compression_SUITE.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2016-2017, Klarna AB +%%% Copyright (c) 2016-2018, Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -14,12 +14,6 @@ %%% limitations under the License. %%% -%%%============================================================================= -%%% @doc -%%% @copyright 2015 Klarna AB -%%% @end -%%% ============================================================================ - %% @private -module(brod_compression_SUITE). diff --git a/test/brod_consumer_SUITE.erl b/test/brod_consumer_SUITE.erl index 1f00d6c6..9747db32 100644 --- a/test/brod_consumer_SUITE.erl +++ b/test/brod_consumer_SUITE.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2015 - 2017, Klarna AB +%%% Copyright (c) 2015-2018, Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -14,12 +14,6 @@ %%% limitations under the License. %%% -%%%============================================================================= -%%% @doc -%%% @copyright 20150-2016 Klarna AB -%%% @end -%%% ============================================================================ - %% @private -module(brod_consumer_SUITE). @@ -37,7 +31,7 @@ , t_direct_fetch_with_small_max_bytes/1 , t_direct_fetch_expand_max_bytes/1 , t_consumer_max_bytes_too_small/1 - , t_consumer_socket_restart/1 + , t_consumer_connection_restart/1 , t_consumer_resubscribe/1 , t_subscriber_restart/1 , t_subscribe_with_unknown_offset/1 @@ -177,15 +171,14 @@ t_direct_fetch_expand_max_bytes(Config) when is_list(Config) -> %% @doc Consumer should be smart enough to try greater max_bytes %% when it's not great enough to fetch one single message -%% @end t_consumer_max_bytes_too_small({init, Config}) -> - meck:new(kpro, [passthrough, no_passthrough_cover, no_history]), + meck:new(brod_kafka_request, [passthrough, no_passthrough_cover, no_history]), %% kafka returns empty message set when it's 0.9 %% or when fetch request sent was version 0 %% Avoid querying api version will make brod send v0 requests [{client_config, [{query_api_versions, false}]} | Config]; t_consumer_max_bytes_too_small({'end', _Config}) -> - meck:unload(kpro); + meck:unload(brod_kafka_request); t_consumer_max_bytes_too_small(Config) -> Client = ?config(client), Partition = 0, @@ -196,13 +189,13 @@ t_consumer_max_bytes_too_small(Config) -> MaxBytes2 = 12, %% too small but message size is fetched MaxBytes3 = size(Key) + ValueBytes, Tester = self(), - F = fun(Vsn, Topic, Partition1, BeginOffset, MaxWait, MinBytes, MaxBytes) -> + F = fun(Conn, Topic, Partition1, BeginOffset, MaxWait, MinBytes, MaxBytes) -> Tester ! {max_bytes, MaxBytes}, - meck:passthrough([Vsn, Topic, Partition1, BeginOffset, + meck:passthrough([Conn, Topic, Partition1, BeginOffset, MaxWait, MinBytes, MaxBytes]) end, %% Expect the fetch_request construction function called twice - meck:expect(kpro, fetch_request, F), + meck:expect(brod_kafka_request, fetch, F), Value = make_bytes(ValueBytes), Options = [{max_bytes, MaxBytes1}], {ok, ConsumerPid} = @@ -219,10 +212,9 @@ t_consumer_max_bytes_too_small(Config) -> ?assertEqual(Value, ValueReceived) end). -%% @doc Consumer shoud auto recover from socket down, subscriber should not +%% @doc Consumer shoud auto recover from connection down, subscriber should not %% notice a thing except for a few seconds of break in data streaming -%% @end -t_consumer_socket_restart(Config) -> +t_consumer_connection_restart(Config) -> Client = ?config(client), Topic = ?TOPIC, Partition = 0, @@ -254,9 +246,9 @@ t_consumer_socket_restart(Config) -> after 1000 -> ct:fail("timed out waiting for seqno producer loop to start") end, - {ok, SocketPid} = + {ok, ConnPid} = brod_client:get_leader_connection(Client, Topic, Partition), - exit(SocketPid, kill), + exit(ConnPid, kill), receive {produce_result_change, ProducerPid, error} -> ok @@ -290,7 +282,6 @@ t_consumer_socket_restart(Config) -> %% @doc Data stream should resume after re-subscribe starting from the %% the last acked offset -%% @end t_consumer_resubscribe(Config) when is_list(Config) -> Client = ?config(client), Topic = ?TOPIC, @@ -431,9 +422,8 @@ t_offset_reset_policy(Config) when is_list(Config) -> %%%_* Help functions =========================================================== -%% @private Expecting sequence numbers delivered from kafka +%% Expecting sequence numbers delivered from kafka %% not expecting any error messages. -%% @end seqno_consumer_loop(ExitSeqNo, ExitSeqNo) -> %% we have verified all sequence numbers, time to exit exit(normal); @@ -455,10 +445,9 @@ seqno_consumer_loop(ExpectedSeqNo, ExitSeqNo) -> exit({"unexpected message received", Msg}) end. -%% @private Verify if a received sequence number list is as expected +%% Verify if a received sequence number list is as expected %% sequence numbers are allowed to get redelivered, %% but should not be re-ordered. -%% @end verify_seqno(SeqNo, []) -> SeqNo + 1; verify_seqno(SeqNo, [X | _] = SeqNoList) when X < SeqNo -> @@ -468,9 +457,8 @@ verify_seqno(SeqNo, [SeqNo | Rest]) -> verify_seqno(SeqNo, SeqNoList) -> exit({"sequence number received is not as expected", SeqNo, SeqNoList}). -%% @private Produce sequence numbers in a retry loop. +%% Produce sequence numbers in a retry loop. %% Report produce API return value pattern changes to parent pid -%% @end seqno_producer_loop(ProduceFun, SeqNo, LastResult, Parent) -> {Result, NextSeqNo} = case ProduceFun(SeqNo) of diff --git a/test/brod_demo_cg_collector.erl b/test/brod_demo_cg_collector.erl index 471bbc91..fa108da5 100644 --- a/test/brod_demo_cg_collector.erl +++ b/test/brod_demo_cg_collector.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2016-2017 Klarna AB +%%% Copyright (c) 2016-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -24,7 +24,6 @@ %%% %%% This can be useful to build your own consumer lagging monitoring or %%% dashboarding tools. -%%% @copyright 2016 Klarna AB %%% @end %%%============================================================================= diff --git a/test/brod_demo_group_subscriber_koc.erl b/test/brod_demo_group_subscriber_koc.erl index 12bcf6e1..abf35756 100644 --- a/test/brod_demo_group_subscriber_koc.erl +++ b/test/brod_demo_group_subscriber_koc.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2016-2017 Klarna AB +%%% Copyright (c) 2016-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -21,7 +21,6 @@ %%% it demos a all-configs-by-default minimal implenemtation of a %%% consumer group subscriber which commits offsets to kafka. %%% See bootstrap/0 for more details about all prerequisite. -%%% @copyright 2016 Klarna AB %%% @end %%%============================================================================= @@ -74,7 +73,6 @@ %% synchronously, or dispatch it to any number of worker processes for %% concurrent processing, acks can be sent from the worker processes %% by calling brod_group_subscriber:ack/4 -%% @end -spec bootstrap() -> ok. bootstrap() -> bootstrap(?PRODUCE_DELAY_SECONDS). @@ -179,12 +177,11 @@ producer_loop(ClientId, Topic, Partition, DelaySeconds, Seqno) -> timer:sleep(timer:seconds(DelaySeconds)), producer_loop(ClientId, Topic, Partition, DelaySeconds, Seqno+1). -%% @private Spawn one message handler per partition. Some of them may sit +%% Spawn one message handler per partition. Some of them may sit %% idle if the partition is assigned to another group member. %% Perhaps hibernate if idle for certain minutes. %% Or even spawn dynamically in `handle_message` callback and %% `exit(normal)` when idle for long. -%% @end -spec spawn_message_handlers(brod:client_id(), [brod:topic()]) -> [{{brod:topic(), brod:partition()}, pid()}]. spawn_message_handlers(_ClientId, []) -> []; diff --git a/test/brod_demo_group_subscriber_loc.erl b/test/brod_demo_group_subscriber_loc.erl index c58e42cd..4762c305 100644 --- a/test/brod_demo_group_subscriber_loc.erl +++ b/test/brod_demo_group_subscriber_loc.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2016-2017 Klarna AB +%%% Copyright (c) 2016-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -20,7 +20,6 @@ %%% This is called 'loc' as in 'Local Offset Commit'. i.e. it demos an %%% implementation of group subscriber that writes offsets locally (to file %%% in this module), but does not commit offsets to Kafka. -%%% @copyright 2016 Klarna AB %%% @end %%%============================================================================= @@ -195,12 +194,11 @@ bootstrap_subscribers([ClientId | Rest], BootstrapHosts, GroupId, filename(Dir, GroupId, Topic, Partition) -> filename:join([Dir, GroupId, Topic, integer_to_list(Partition)]). -%% @private Offsets are committed locally in files for demo. +%% Offsets are committed locally in files for demo. %% Due to the fact that a partition can be assigned to any group member, %% in a real use case, when group members are distributed among Erlang nodes %% (or even hosts), the offsets should be committed to a place where all %% members have access to. e.g. a database. -%% @end commit_offset(Dir, GroupId, Topic, Partition, Offset) -> Filename = filename(Dir, GroupId, Topic, Partition), ok = filelib:ensure_dir(Filename), diff --git a/test/brod_demo_topic_subscriber.erl b/test/brod_demo_topic_subscriber.erl index 38668ee7..8c24a4b6 100644 --- a/test/brod_demo_topic_subscriber.erl +++ b/test/brod_demo_topic_subscriber.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2016-2017 Klarna AB +%%% Copyright (c) 2016-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -52,7 +52,6 @@ %% * consumed sequence numbers are printed to console %% * consumed offsets are written to file /tmp/T/P.offset %% where T is the topic name and X is the partition number -%% @end -spec bootstrap() -> ok. bootstrap() -> bootstrap(?PRODUCE_DELAY_SECONDS, message). diff --git a/test/brod_group_subscriber_SUITE.erl b/test/brod_group_subscriber_SUITE.erl index d0087e02..1138c66e 100644 --- a/test/brod_group_subscriber_SUITE.erl +++ b/test/brod_group_subscriber_SUITE.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2015-2017, Klarna AB +%%% Copyright (c) 2015-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -394,7 +394,7 @@ do_wait_for_subscribers(States) -> NewStates = lists:keystore(TP, 1, States, {TP, undefined}), do_wait_for_subscribers(NewStates) after - 4000 -> + 10000 -> erlang:error({timeout, States}) end end. diff --git a/test/brod_kafka_apis_tests.erl b/test/brod_kafka_apis_tests.erl index 70dc8c27..f87cafa1 100644 --- a/test/brod_kafka_apis_tests.erl +++ b/test/brod_kafka_apis_tests.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2017, Klarna AB +%%% Copyright (c) 2017-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -18,6 +18,16 @@ -include_lib("eunit/include/eunit.hrl"). +-define(WITH_MECK(Versions, EXPR), + fun() -> + try + ok = setup(Versions), + EXPR + after + ok = clear() + end + end()). + start_stop_test() -> _ = application:stop(brod), %% other tests might have it started {ok, _Pid} = brod_kafka_apis:start_link(), @@ -25,68 +35,46 @@ start_stop_test() -> ok = brod_kafka_apis:stop(). only_one_version_test() -> - ?assertEqual(0, brod_kafka_apis:pick_version(pid, list_groups_request)). - -pick_min_version_test() -> - %% use min version when ther is no versions received from broker - {ok, Pid} = brod_kafka_apis:start_link(), - ?assertEqual(0, brod_kafka_apis:pick_version(Pid, produce_request)), - ok = brod_kafka_apis:stop(). + %% we support only one version, no need to lookup + ?assertEqual(0, brod_kafka_apis:pick_version(conn, list_groups)). pick_brod_max_version_test() -> %% brod supports max = 2, kafka supports max = 7 - {ok, _Pid} = brod_kafka_apis:start_link(), - Sock = self(), %% faking it - versions_received(client, Sock, [{produce_request, {0, 7}}]), - ?assertEqual(2, brod_kafka_apis:pick_version(Sock, produce_request)), - ok = brod_kafka_apis:stop(). + ?WITH_MECK(#{produce => {0, 7}}, + ?assertEqual(2, brod_kafka_apis:pick_version(self(), produce))). pick_kafka_max_version_test() -> %% brod supports max = 2, kafka supports max = 1 - {ok, _Pid} = brod_kafka_apis:start_link(), - Sock = self(), %% faking it - versions_received(client, Sock, [{produce_request, {0, 1}}]), - ?assertEqual(1, brod_kafka_apis:pick_version(Sock, produce_request)), - ok = brod_kafka_apis:stop(). + ?WITH_MECK(#{produce => {0, 1}}, + ?assertEqual(1, brod_kafka_apis:pick_version(self(), produce))). pick_min_brod_version_test() -> %% no versions received from kafka - {ok, _Pid} = brod_kafka_apis:start_link(), - Sock = self(), %% faking it - ?assertEqual(0, brod_kafka_apis:pick_version(Sock, produce_request)), - ok = brod_kafka_apis:stop(). + ?WITH_MECK(#{}, + ?assertEqual(0, brod_kafka_apis:pick_version(self(), produce))). pick_min_brod_version_2_test() -> - %% no versions received from kafka - {ok, _Pid} = brod_kafka_apis:start_link(), - Sock = self(), %% faking it - versions_received(client, Sock, [{fetch_request, {0, 0}}]), - ?assertEqual(0, brod_kafka_apis:pick_version(Sock, produce_request)), - ok = brod_kafka_apis:stop(). + %% received 'fetch' API version, lookup 'produce' + ?WITH_MECK(#{fetch => {0, 0}}, + ?assertEqual(0, brod_kafka_apis:pick_version(self(), produce))). -no_overlapping_version_range_test() -> +no_version_range_intersection_test() -> %% brod supports 0 - 2, kafka supports 6 - 7 - {ok, _Pid} = brod_kafka_apis:start_link(), - Sock = self(), %% faking it - versions_received(client, Sock, [{produce_request, {6, 7}}]), - ?assertEqual(0, brod_kafka_apis:pick_version(Sock, produce_request)), - ok = brod_kafka_apis:stop(). + ?WITH_MECK(#{produce => {6, 7}}, + ?assertError({unsupported_vsn_range, _, _, _}, + brod_kafka_apis:pick_version(self(), produce))). -add_sock_pid_test() -> - {ok, _Pid} = brod_kafka_apis:start_link(), - Sock1 = spawn(fun() -> receive after infinity -> ok end end), - Sock2 = spawn(fun() -> receive after infinity -> ok end end), - Versions = [{produce_request, {2, 3}}], - ?assertEqual({error, unknown_host}, - brod_kafka_apis:maybe_add_sock_pid(host, Sock1)), - brod_kafka_apis:versions_received(client, Sock1, Versions, host), - ok = brod_kafka_apis:maybe_add_sock_pid(host, Sock2), - ?assertEqual(2, brod_kafka_apis:pick_version(Sock1, produce_request)), - ?assertEqual(2, brod_kafka_apis:pick_version(Sock2, produce_request)), - ok = brod_kafka_apis:stop(). +setup(Versions) -> + _ = application:stop(brod), %% other tests might have it started + _ = brod_kafka_apis:start_link(), + meck:new(kpro, [passthrough, no_passthrough_cover, no_history]), + meck:expect(kpro, get_api_versions, fun(_) -> {ok, Versions} end), + ok. -versions_received(Client, SockPid, Versions) -> - brod_kafka_apis:versions_received(Client, SockPid, Versions, host). +clear() -> + brod_kafka_apis:stop(), + meck:unload(kpro), + ok. %%%_* Emacs ==================================================================== %%% Local Variables: diff --git a/test/brod_producer_SUITE.erl b/test/brod_producer_SUITE.erl index 626317da..af9ef644 100644 --- a/test/brod_producer_SUITE.erl +++ b/test/brod_producer_SUITE.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2015-2017, Klarna AB +%%% Copyright (c) 2015-2018, Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -14,12 +14,6 @@ %%% limitations under the License. %%% -%%%============================================================================= -%%% @doc -%%% @copyright 2015-2017 Klarna AB -%%% @end -%%% ============================================================================ - %% @private -module(brod_producer_SUITE). diff --git a/test/brod_producer_buffer_SUITE.erl b/test/brod_producer_buffer_SUITE.erl index c1970e6d..d74a203f 100644 --- a/test/brod_producer_buffer_SUITE.erl +++ b/test/brod_producer_buffer_SUITE.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2015-2017, Klarna AB +%%% Copyright (c) 2015-2018 Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -14,12 +14,6 @@ %%% limitations under the License. %%% -%%%============================================================================= -%%% @doc -%%% @copyright 2015 Klarna AB -%%% @end -%%% ============================================================================ - %% @private -module(brod_producer_buffer_SUITE). @@ -81,19 +75,14 @@ t_random_latency_ack(Config) when is_list(Config) -> ?assert(proper:quickcheck(prop_random_latency_ack_run(), Opts)). t_nack(Config) when is_list(Config) -> - UnknownCorrId = 0, SendFun = - fun(SockPid, KvList, _Vsn) -> - CorrId = case get(<<"t_nack_corr_id">>) of - undefined -> UnknownCorrId + 1; - N -> N + 1 - end, - put(<<"t_nack_corr_id">>, CorrId), + fun(Conn, KvList, _Vsn) -> + Ref = make_ref(), NumList = lists:map(fun({Bin, Bin}) -> list_to_integer(binary_to_list(Bin)) end, KvList), - SockPid ! {produce, CorrId, NumList}, - {ok, CorrId} + Conn ! {produce, Ref, NumList}, + {ok, Ref} end, Buf0 = brod_producer_buffer:new(_BufferLimit = 2, _OnWireLimit = 2, @@ -109,34 +98,32 @@ t_nack(Config) when is_list(Config) -> , ref = Num }, Bin = list_to_binary(integer_to_list(Num)), - {ok, BufOut} = brod_producer_buffer:add(BufIn, CallRef, Bin, Bin), - BufOut + brod_producer_buffer:add(BufIn, CallRef, Bin, Bin) end, MaybeSend = fun(BufIn) -> - {ok, BufOut} = brod_producer_buffer:maybe_send(BufIn, self(), 0), - BufOut + {ok, Buf} = brod_producer_buffer:maybe_send(BufIn, self(), 0), + Buf end, AckFun = - fun(BufIn, CorrId) -> - {ok, BufOut} = brod_producer_buffer:ack(BufIn, CorrId), - BufOut + fun(BufIn, Ref) -> + brod_producer_buffer:ack(BufIn, Ref) end, NackFun = - fun(BufIn, CorrId) -> - brod_producer_buffer:nack(BufIn, CorrId, test) + fun(BufIn, Ref) -> + brod_producer_buffer:nack(BufIn, Ref, test) end, ReceiveFun = fun(Line, ExpectedNums) -> receive - {produce, CorrId_, NumList} -> + {produce, RefX, NumList} -> case ExpectedNums =:= NumList of true -> ok; false -> ct:fail("~p\nexp=~p\ngot=~p\n", [Line, ExpectedNums, NumList]) end, - CorrId_ + RefX after 1000 -> erlang:error({Line, "timed out receiving produce message"}) end @@ -145,16 +132,14 @@ t_nack(Config) when is_list(Config) -> Buf2 = AddFun(Buf1, 1), Buf3 = AddFun(AddFun(Buf2, 2), 3), Buf4 = MaybeSend(Buf3), - CorrId1 = ReceiveFun(?LINE, [0, 1]), %% max batch size - CorrId2 = ReceiveFun(?LINE, [2, 3]), %% max onwire is 2 - ?assertEqual({error, CorrId1}, NackFun(Buf4, UnknownCorrId)), - ?assertException(exit, {bad_order, _, _}, NackFun(Buf4, CorrId2)), - {ok, Buf5} = NackFun(Buf4, CorrId1), %% re-queue all - Buf6 = MaybeSend(Buf5), %% as if a sheduled retry - CorrId3 = ReceiveFun(?LINE, [0, 1]), %% receive a max batch - CorrId4 = ReceiveFun(?LINE, [2, 3]), %% another max batch (max onwire is 2) - Buf7 = AckFun(Buf6, CorrId3), - Buf8 = AckFun(Buf7, CorrId4), + Ref1 = ReceiveFun(?LINE, [0, 1]), %% max batch size + _Ref = ReceiveFun(?LINE, [2, 3]), %% max onwire is 2 + Buf5 = NackFun(Buf4, Ref1), %% re-queue all + Buf6 = MaybeSend(Buf5), %% as if a sheduled retry + Ref3 = ReceiveFun(?LINE, [0, 1]), %% receive a max batch + Ref4 = ReceiveFun(?LINE, [2, 3]), %% another max batch (max onwire is 2) + Buf7 = AckFun(Buf6, Ref3), + Buf8 = AckFun(Buf7, Ref4), ?assert(brod_producer_buffer:is_empty(Buf8)). t_send_fun_error(Config) when is_list(Config) -> @@ -176,8 +161,7 @@ t_send_fun_error(Config) when is_list(Config) -> , ref = Num }, Bin = list_to_binary(integer_to_list(Num)), - {ok, BufOut} = brod_producer_buffer:add(BufIn, CallRef, Bin, Bin), - BufOut + brod_producer_buffer:add(BufIn, CallRef, Bin, Bin) end, MaybeSend = fun(BufIn) -> @@ -227,13 +211,13 @@ prop_random_latency_ack_run() -> SendFun0 = fun(FakeKafka, KvList, _Vsn) -> %% use reference as correlation to simplify test - CorrId = make_ref(), + Ref = make_ref(), %% send the message to fake kafka %% the pre-generated latency values are in KvList %% fake kafka should receive the KvList, sleep a while %% and reply ack - FakeKafka ! {produce, self(), CorrId, KvList}, - {ok, CorrId} + FakeKafka ! {produce, self(), Ref, KvList}, + {ok, Ref} end, ?FORALL( {BufferLimit, OnWireLimit, MsgSetBytes, @@ -263,7 +247,7 @@ no_ack_produce(Buf, [{Key, Value} | Rest]) -> , ref = Key }, BinKey = list_to_binary(integer_to_list(Key)), - {ok, Buf1} = brod_producer_buffer:add(Buf, CallRef, BinKey, Value), + Buf1 = brod_producer_buffer:add(Buf, CallRef, BinKey, Value), FakeSockPid = self(), {ok, NewBuf} = brod_producer_buffer:maybe_send(Buf1, FakeSockPid, 0), %% in case of no ack required, expect 'buffered' immediately @@ -311,7 +295,7 @@ produce_loop(FakeKafka, [{Key, Value} | Rest], State0) -> , ref = Key }, BinKey = list_to_binary(integer_to_list(Key)), - {ok, Buf1} = brod_producer_buffer:add(Buf0, CallRef, BinKey, Value), + Buf1 = brod_producer_buffer:add(Buf0, CallRef, BinKey, Value), State1 = State0#state{buf = Buf1}, State2 = maybe_send(State1), State = collect_replies(State2, _Delay = 0), @@ -335,8 +319,8 @@ collect_replies(#state{ buffered = Buffered } -> State = State0#state{buffered = [Key | Buffered]}, collect_replies(State, Timeout); - {ack_from_kafka, CorrId} -> - {ok, Buf1} = brod_producer_buffer:ack(Buf0, CorrId), + {ack_from_kafka, Ref} -> + Buf1 = brod_producer_buffer:ack(Buf0, Ref), State1 = State0#state{buf = Buf1}, State = maybe_send(State1), collect_replies(State, Timeout); @@ -362,16 +346,15 @@ maybe_send(#state{buf = Buf0, delay_ref = DelayRef} = State) -> State#state{buf = Buf, delay_ref = NewDelayRef} end. -%% @private Start delay send timer. +%% Start delay send timer. start_delay_send_timer(Timeout) -> MsgRef = make_ref(), TRef = erlang:send_after(Timeout, self(), {delayed_send, MsgRef}), {TRef, MsgRef}. -%% @private Ensure delay send timer is canceled. +%% Ensure delay send timer is canceled. %% But not flushing the possibly already sent (stale) message %% Stale message should be discarded in handle_info -%% @end cancel_delay_send_timer(?undef) -> ok; cancel_delay_send_timer({Tref, _Msg}) -> _ = erlang:cancel_timer(Tref). @@ -396,9 +379,9 @@ stop_fake_kafka(FakeKafka) when is_pid(FakeKafka) -> fake_kafka_loop() -> receive - {produce, FromPid, CorrId, KvList} -> + {produce, FromPid, Ref, KvList} -> ok = fake_kafka_process_msgs(KvList), - FromPid ! {ack_from_kafka, CorrId}, + FromPid ! {ack_from_kafka, Ref}, fake_kafka_loop(); stop -> exit(normal); diff --git a/test/brod_producer_stub_SUITE.erl b/test/brod_producer_stub_SUITE.erl index 068a17aa..74fc3414 100644 --- a/test/brod_producer_stub_SUITE.erl +++ b/test/brod_producer_stub_SUITE.erl @@ -1,8 +1,18 @@ -%%%============================================================================= -%%% @doc -%%% @copyright 2017 Klarna AB -%%% @end -%%% ============================================================================ +%%% +%%% Copyright (c) 2017-2018 Klarna Bank AB (publ) +%%% +%%% Licensed under the Apache License, Version 2.0 (the "License"); +%%% you may not use this file except in compliance with the License. +%%% You may obtain a copy of the License at +%%% +%%% http://www.apache.org/licenses/LICENSE-2.0 +%%% +%%% Unless required by applicable law or agreed to in writing, software +%%% distributed under the License is distributed on an "AS IS" BASIS, +%%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +%%% See the License for the specific language governing permissions and +%%% limitations under the License. +%%% %% @private -module(brod_producer_stub_SUITE). @@ -19,8 +29,8 @@ %% Test cases -export([ t_normal_flow/1 , t_no_required_acks/1 - , t_retry_on_same_socket/1 - , t_sock_down_retry/1 + , t_retry_on_same_connection/1 + , t_connection_down_retry/1 , t_leader_migration/1 ]). @@ -58,7 +68,12 @@ end_per_suite(_Config) -> ok. init_per_testcase(Case, Config) -> meck_module(brod_client), - meck_module(brod_sock), + meck_module(kpro), + meck_module(brod_kafka_apis), + meck:expect(brod_kafka_apis, pick_version, + fun(_, API) -> + brod_kafka_apis:default_version(API) + end), try ?MODULE:Case({'init', Config}) catch @@ -68,8 +83,8 @@ init_per_testcase(Case, Config) -> end_per_testcase(Case, Config) -> meck:unload(brod_client), - meck:unload(brod_sock), - _ = erase(corr_id), + meck:unload(kpro), + meck:unload(brod_kafka_apis), try ?MODULE:Case({'end', Config}) catch @@ -88,17 +103,16 @@ all() -> [F || {F, _A} <- module_info(exports), t_normal_flow(Config) when is_list(Config) -> Tester = self(), meck:expect(brod_client, get_leader_connection, - fun(client, <<"topic">>, 0) -> {ok, Tester} end), - meck:expect(brod_sock, request_async, - fun(Pid, KafkaReq) -> - CorrId = corr_id(), - Pid ! {request_async, CorrId, KafkaReq}, - {ok, CorrId} + fun(_, <<"topic">>, 0) -> {ok, Tester} end), + meck:expect(kpro, request_async, + fun(Connection, KafkaReq) -> + Connection ! {request_async, KafkaReq}, + ok end), {ok, Producer} = brod_producer:start_link(client, <<"topic">>, 0, []), {ok, CallRef} = brod_producer:produce(Producer, <<"key">>, <<"val">>), - ?WAIT({request_async, CorrId, _KafkaReq}, - Producer ! {msg, Tester, fake_rsp(CorrId, <<"topic">>, 0)}, 2000), + ?WAIT({request_async, #kpro_req{ref = Ref}}, + Producer ! {msg, Tester, fake_rsp(Ref, <<"topic">>, 0)}, 2000), ?WAIT(#brod_produce_reply{call_ref = CallRef, result = brod_produce_req_acked}, ok, 2000), @@ -108,18 +122,18 @@ t_normal_flow(Config) when is_list(Config) -> %% no produce response will be received from kafka %% but still caller should receive 'acked' reply t_no_required_acks({init, Config}) -> - meck_module(kpro), + meck_module(brod_kafka_request), Config; t_no_required_acks({'end', Config}) -> - meck:unload(kpro), + meck:unload(brod_kafka_request), Config; t_no_required_acks(Config) when is_list(Config) -> MaxLingerCount = 3, Tester = self(), meck:expect(brod_client, get_leader_connection, - fun(client, <<"topic">>, 0) -> {ok, Tester} end), - meck:expect(kpro, produce_request, - fun(_Vsn = 0, <<"topic">>, 0, KvList, 0, _, no_compression) -> + fun(_, <<"topic">>, 0) -> {ok, Tester} end), + meck:expect(brod_kafka_request, produce, + fun(_, <<"topic">>, 0, KvList, 0, _, no_compression) -> case length(KvList) =:= MaxLingerCount of true -> ?assertEqual([{<<"1">>, <<"1">>}, @@ -129,14 +143,9 @@ t_no_required_acks(Config) when is_list(Config) -> false -> ?assertEqual([{<<"4">>, <<"4">>}], KvList) end, - <<"fake-produce-request">> - end), - meck:expect(brod_sock, request_async, - fun(_Pid, <<"fake-produce-request">>) -> - %% mocking brod_sock's behaviour when - %% required-acks is 0 - ok + #kpro_req{api = produce, no_ack = true} end), + meck:expect(kpro, request_async, fun(_Conn, _Req) -> ok end), ProducerConfig = [{required_acks, 0}, {max_linger_ms, 1000}, {max_linger_count, MaxLingerCount}], @@ -158,40 +167,38 @@ t_no_required_acks(Config) when is_list(Config) -> result = brod_produce_req_acked}, ok, 2000), ok = brod_producer:stop(Producer). -t_retry_on_same_socket(Config) when is_list(Config) -> +t_retry_on_same_connection(Config) when is_list(Config) -> Tester = self(), - %% A mocked socket process which expects 2 requests to be sent + %% A mocked connection process which expects 2 requests to be sent %% reply a retriable error code for the first one %% and succeed the second one - SockLoop = + ConnLoop = fun Loop(N) -> receive - {produce, CorrId, ProducerPid, <<"the req">>} -> + {produce, ProducerPid, #kpro_req{ref = Ref}} -> Rsp = case N of - 1 -> fake_rsp(CorrId, <<"topic">>, 0, - ?EC_REQUEST_TIMED_OUT); - 2 -> fake_rsp(CorrId, <<"topic">>, 0) + 1 -> fake_rsp(Ref, <<"topic">>, 0, ?request_timed_out); + 2 -> fake_rsp(Ref, <<"topic">>, 0) end, Tester ! {'try', N}, ProducerPid ! {msg, self(), Rsp}, Loop(N + 1) end end, - SockPid = erlang:spawn_link(fun() -> SockLoop(1) end), + ConnPid = erlang:spawn_link(fun() -> ConnLoop(1) end), meck:expect(brod_client, get_leader_connection, fun(client, <<"topic">>, 0) -> - {ok, SockPid} + {ok, ConnPid} end), ProducerConfig = [{required_acks, 1}, {max_linger_ms, 0}, {retry_backoff_ms, 100}], {ok, Producer} = brod_producer:start_link(client, <<"topic">>, 0, ProducerConfig), - meck:expect(brod_sock, request_async, - fun(SockPid_, _KafkaReq) -> - CorrId = corr_id(), - SockPid_ ! {produce, CorrId, Producer, <<"the req">>}, - {ok, CorrId} + meck:expect(kpro, request_async, + fun(Connection, Req) -> + Connection ! {produce, Producer, Req}, + ok end), {ok, CallRef} = brod_producer:produce(Producer, <<"k">>, <<"v">>), ?WAIT({'try', 1}, ok, 1000), @@ -201,52 +208,48 @@ t_retry_on_same_socket(Config) when is_list(Config) -> ok, 2000), ok = brod_producer:stop(Producer). -%% This is a tipical socket restart scenario: -%% 0. Allow two requests on wire +%% This is a typical connection restart scenario: +%% 0. Start producer allowing two requests on wire %% 1. Send first request on wire, but no ack yet -%% 2. Reply {error, {sock_down, reason}} for the second produce call -%% 3. Kill the mocked socket pid +%% 2. Reply {error, {connection_down, Reason}} for the second produce call +%% 3. Kill the mocked connection pid %% Expect brod_producer to retry. %% The retried produce request should have both messages in one message set -t_sock_down_retry({init, Config}) -> - meck_module(kpro), - meck:expect(kpro, produce_request, - fun(_Vsn = 0, <<"topic">>, 0, KvList, _RequiredAcks, +t_connection_down_retry({init, Config}) -> + meck_module(brod_kafka_request), + meck:expect(brod_kafka_request, produce, + fun(_Vsn, <<"topic">>, 0, KvList, _RequiredAcks, _AckTimeout, no_compression) -> - {<<"fake-req">>, KvList} + #kpro_req{msg = KvList} end), Config; -t_sock_down_retry({'end', Config}) -> - meck:unload(kpro), +t_connection_down_retry({'end', Config}) -> + meck:unload(brod_kafka_request), Config; -t_sock_down_retry(Config) when is_list(Config) -> +t_connection_down_retry(Config) when is_list(Config) -> Tester = self(), - %% A mocked socket process which expects 2 requests to be sent + %% A mocked connection process which expects 2 requests to be sent %% black-hole the first request and succeed the second one - SockFun = + ConnFun = fun L() -> receive - {produce, CorrId, ProducerPid, {<<"fake-req">>, KvList}} -> - Tester ! {sent, CorrId, KvList}, - case CorrId of - 0 -> + {produce, ProducerPid, #kpro_req{ref = Ref, msg = KvList}} -> + Tester ! {sent, KvList}, + case length(KvList) of + 1 -> ok; - 2 -> - Rsp = fake_rsp(CorrId, <<"topic">>, 0), - ProducerPid ! {msg, self(), Rsp}; - X -> - %% this should pollute Tester's message queue - %% and cause test to fail - Tester ! {<<"unexpected corr_id">>, X} + _ -> + Rsp = fake_rsp(Ref, <<"topic">>, 0), + ProducerPid ! {msg, self(), Rsp} end, L() end end, meck:expect(brod_client, get_leader_connection, fun(client, <<"topic">>, 0) -> - SockPid = erlang:spawn(SockFun), - Tester ! {connected, SockPid}, - {ok, SockPid} + ConnPid = erlang:spawn(ConnFun), + Tester ! {connected, ConnPid}, + {ok, ConnPid} end), ProducerConfig = [{required_acks, 1}, {max_linger_ms, 0}, @@ -254,81 +257,85 @@ t_sock_down_retry(Config) when is_list(Config) -> {retry_backoff_ms, 1000}], {ok, Producer} = brod_producer:start_link(client, <<"topic">>, 0, ProducerConfig), - meck:expect(brod_sock, request_async, - fun(SockPid_, KafkaReq) -> - case corr_id() of + WhichCall = fun() -> + CallCount = case get(call_count) of + undefined -> 0; + N -> N + 1 + end, + put(call_count, CallCount), + CallCount + end, + meck:expect(kpro, request_async, + fun(Connection, KafkaReq) -> + case WhichCall() of 0 -> - SockPid_ ! {produce, 0, Producer, KafkaReq}, - {ok, 0}; + Connection ! {produce, Producer, KafkaReq}, + ok; 1 -> Tester ! {called, 1}, - {error, {sock_down, test}}; + {error, {connection_down, test}}; 2 -> - SockPid_ ! {produce, 2, Producer, KafkaReq}, - {ok, 2} + Connection ! {produce, Producer, KafkaReq}, + ok end end), {ok, CallRef1} = brod_producer:produce(Producer, <<"k1">>, <<"v1">>), - SockPid1 = ?WAIT({connected, Pid}, Pid, 1000), - ?WAIT({sent, 0, [{<<"k1">>, <<"v1">>}]}, ok, 1000), + ConnPid1 = ?WAIT({connected, Pid}, Pid, 1000), + ?WAIT({sent, [{<<"k1">>, <<"v1">>}]}, ok, 1000), {ok, CallRef2} = brod_producer:produce(Producer, <<"k2">>, <<"v2">>), ?WAIT({called, 1}, ok, 1000), - erlang:exit(SockPid1, kill), + erlang:exit(ConnPid1, kill), ?WAIT({connected, _}, ok, 3000), - ?WAIT({sent, 2, [{<<"k1">>, _}, {<<"k2">>, _}]}, ok, 1000), + ?WAIT({sent, [{<<"k1">>, _}, {<<"k2">>, _}]}, ok, 1000), ?WAIT(#brod_produce_reply{call_ref = CallRef1, result = brod_produce_req_acked}, ok, 1000), ?WAIT(#brod_produce_reply{call_ref = CallRef2, result = brod_produce_req_acked}, ok, 1000), ok = brod_producer:stop(Producer). -%% leader migration for brod_producer means new socket pid -%% expect brod_producer to retry on the new socket +%% leader migration for brod_producer means new connection pid +%% expect brod_producer to retry on the new connection t_leader_migration({init, Config}) -> - meck_module(kpro), - meck:expect(kpro, produce_request, + meck_module(brod_kafka_request), + meck:expect(brod_kafka_request, produce, fun(_Vsn = 0, <<"topic">>, 0, KvList, _RequiredAcks, _AckTimeout, no_compression) -> - {<<"fake-req">>, KvList} + #kpro_req{msg = KvList} end), Config; t_leader_migration({'end', Config}) -> - meck:unload(kpro), + meck:unload(brod_kafka_request), Config; t_leader_migration(Config) when is_list(Config) -> Tester = self(), - SockFunFun = - fun(ExpectedCorrId) -> - fun L() -> - receive - {produce, CorrId, ProducerPid, {<<"fake-req">>, KvList}} -> - ?assertEqual(ExpectedCorrId, CorrId), - Tester ! {sent, CorrId, KvList}, - Rsp = case CorrId of - 0 -> fake_rsp(CorrId, <<"topic">>, 0, - ?EC_NOT_LEADER_FOR_PARTITION); - 1 -> fake_rsp(CorrId, <<"topic">>, 0) - end, - ProducerPid ! {msg, self(), Rsp}, - %% keep looping. - %% i.e. do not exit as if the old leader is still alive - L() - end + ConnFun = + fun L() -> + receive + {produce, ProducerPid, #kpro_req{ref = Ref, msg = KvList}} -> + Tester ! {sent, KvList}, + Rsp = + case length(KvList) of + 1 -> fake_rsp(Ref, <<"topic">>, 0, ?not_leader_for_partition); + _ -> fake_rsp(Ref, <<"topic">>, 0) + end, + ProducerPid ! {msg, self(), Rsp}, + %% keep looping. i.e. do not exit as if the old leader is still alive + L() end end, meck:expect(brod_client, get_leader_connection, fun(client, <<"topic">>, 0) -> - SockPid = + ConnPid = case get(<<"which-leader">>) of undefined -> - Pid = erlang:spawn_link(SockFunFun(0)), + Pid = erlang:spawn_link(ConnFun), put(<<"which-leader">>, Pid), Pid; _Pid -> - erlang:spawn_link(SockFunFun(1)) + erlang:spawn_link(ConnFun) end, - Tester ! {connected, SockPid}, - {ok, SockPid} + Tester ! {connected, ConnPid}, + {ok, ConnPid} end), ProducerConfig = [{required_acks, 1}, {max_linger_ms, 0}, @@ -336,20 +343,19 @@ t_leader_migration(Config) when is_list(Config) -> {retry_backoff_ms, 1000}], {ok, Producer} = brod_producer:start_link(client, <<"topic">>, 0, ProducerConfig), - meck:expect(brod_sock, request_async, - fun(SockPid_, KafkaReq) -> - CorrId = corr_id(), - SockPid_ ! {produce, CorrId, Producer, KafkaReq}, - {ok, CorrId} + meck:expect(kpro, request_async, + fun(Connection, KafkaReq) -> + Connection ! {produce, Producer, KafkaReq}, + ok end), {ok, CallRef1} = brod_producer:produce(Producer, <<"k1">>, <<"v1">>), {ok, CallRef2} = brod_producer:produce(Producer, <<"k2">>, <<"v2">>), {ok, CallRef3} = brod_producer:produce(Producer, <<"k3">>, <<"v3">>), - SockPid1 = ?WAIT({connected, P}, P, 1000), - ?WAIT({sent, 0, [{<<"k1">>, _}]}, ok, 1000), - SockPid2 = ?WAIT({connected, P}, P, 2000), - ?assert(SockPid1 =/= SockPid2), - ?WAIT({sent, 1, [{<<"k1">>, _}, {<<"k2">>, _}, {<<"k3">>, _}]}, ok, 1000), + ConnPid1 = ?WAIT({connected, P}, P, 1000), + ?WAIT({sent, [{<<"k1">>, _}]}, ok, 1000), + ConnPid2 = ?WAIT({connected, P}, P, 2000), + ?assert(ConnPid1 =/= ConnPid2), + ?WAIT({sent, [{<<"k1">>, _}, {<<"k2">>, _}, {<<"k3">>, _}]}, ok, 1000), ?WAIT(#brod_produce_reply{call_ref = CallRef1, result = brod_produce_req_acked}, ok, 1000), ?WAIT(#brod_produce_reply{call_ref = CallRef2, @@ -363,21 +369,13 @@ t_leader_migration(Config) when is_list(Config) -> meck_module(Module) -> meck:new(Module, [passthrough, no_passthrough_cover, no_history]). -corr_id() -> - CorrId = case get(corr_id) of - undefined -> 0; - N -> N + 1 - end, - put(corr_id, CorrId), - CorrId. - -fake_rsp(CorrId, Topic, Partition) -> - fake_rsp(CorrId, Topic, Partition, ?EC_NONE). +fake_rsp(Ref, Topic, Partition) -> + fake_rsp(Ref, Topic, Partition, ?no_error). -fake_rsp(CorrId, Topic, Partition, ErrorCode) -> - #kpro_rsp{ tag = produce_response +fake_rsp(Ref, Topic, Partition, ErrorCode) -> + #kpro_rsp{ api = produce , vsn = 0 - , corr_id = CorrId + , ref = Ref , msg = [{responses, [[{topic, Topic} ,{partition_responses, diff --git a/test/brod_sock_SUITE.erl b/test/brod_sock_SUITE.erl deleted file mode 100644 index a8efeb71..00000000 --- a/test/brod_sock_SUITE.erl +++ /dev/null @@ -1,109 +0,0 @@ -%%% -%%% Copyright (c) 2015-2017, Klarna AB -%%% -%%% Licensed under the Apache License, Version 2.0 (the "License"); -%%% you may not use this file except in compliance with the License. -%%% You may obtain a copy of the License at -%%% -%%% http://www.apache.org/licenses/LICENSE-2.0 -%%% -%%% Unless required by applicable law or agreed to in writing, software -%%% distributed under the License is distributed on an "AS IS" BASIS, -%%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -%%% See the License for the specific language governing permissions and -%%% limitations under the License. -%%% - -%% @private --module(brod_sock_SUITE). - -%% Test framework --export([ init_per_suite/1 - , end_per_suite/1 - , init_per_testcase/2 - , end_per_testcase/2 - , all/0 - , suite/0 - ]). - -%% Test cases --export([ t_request_timeout/1 - ]). - - --include_lib("common_test/include/ct.hrl"). --include_lib("eunit/include/eunit.hrl"). --include("brod_int.hrl"). - -%%%_* ct callbacks ============================================================= - -suite() -> [{timetrap, {seconds, 30}}]. - -init_per_suite(Config) -> - Config. - -end_per_suite(_Config) -> - ok. - -init_per_testcase(_Case, Config) -> - Config. - -end_per_testcase(_Case, Config) when is_list(Config) -> - ok. - -all() -> [F || {F, _A} <- module_info(exports), - case atom_to_list(F) of - "t_" ++ _ -> true; - _ -> false - end]. - -t_request_timeout({init, Config}) -> - meck:new(ssl, [passthrough, no_passthrough_cover, no_history]), - Config; -t_request_timeout({'end', Config}) -> - meck:validate(ssl), - meck:unload(ssl), - Config; -t_request_timeout(Config) when is_list(Config) -> - TesterPid = self(), - meck:expect(ssl, connect, fun(_, _, _) -> {ok, TesterPid} end), - meck:expect(ssl, setopts, fun(_, _) -> ok end), - meck:expect(ssl, send, fun(Pid, _Bin) -> - ?assertEqual(Pid, TesterPid), - ok - end), - ClientConfig = - [{query_api_versions, false}, - {ssl, true}, %% so we can meck ssl module because gen_tcp module is sticky - {request_timeout, 1000}], - %% spawn an isolated middleman - %% so we dont get killed when brod_sock exits - {Pid, Ref} = - spawn_monitor( - fun() -> - {ok, SockPid} = brod_sock:start_link(self(), "localhost", 9092, - client_id, ClientConfig), - TesterPid ! {sock, SockPid}, - receive Msg -> exit({<<"unexpected message">>, Msg}) - after 10000 -> exit(<<"test timeout">>) - end - end), - Sock = receive {sock, P} -> P - after 5000 -> erlang:exit(timeout) - end, - ProduceRequest = - brod_kafka_request:produce_request(0, <<"t">>, 0, [{<<"K">>, <<"V">>}], - 1, 1000, no_compression), - _ = brod_sock:request_async(Sock, ProduceRequest), - receive - {_DOWN, Ref, process, Pid, Reason} -> - ?assertEqual(Reason, request_timeout); - Msg -> - erlang:exit({<<"unexpected">>, Msg}) - end. - -%%%_* Emacs ==================================================================== -%%% Local Variables: -%%% allout-layout: t -%%% erlang-indent-level: 2 -%%% End: diff --git a/test/brod_topic_subscriber_SUITE.erl b/test/brod_topic_subscriber_SUITE.erl index 5cddf7ce..583063c5 100644 --- a/test/brod_topic_subscriber_SUITE.erl +++ b/test/brod_topic_subscriber_SUITE.erl @@ -1,5 +1,5 @@ %%% -%%% Copyright (c) 2015-2017, Klarna AB +%%% Copyright (c) 2015-2018, Klarna Bank AB (publ) %%% %%% Licensed under the Apache License, Version 2.0 (the "License"); %%% you may not use this file except in compliance with the License. @@ -14,12 +14,6 @@ %%% limitations under the License. %%% -%%%============================================================================= -%%% @doc -%%% @copyright 20150-2016 Klarna AB -%%% @end -%%% ============================================================================ - %% @private -module(brod_topic_subscriber_SUITE).