├── .gitignore ├── LICENSE ├── README.md ├── config └── config.exs ├── docker ├── docker-compose.yml.example └── kafka │ ├── Dockerfile │ ├── README.md │ ├── broker-list.sh │ ├── docker-compose-single-broker.yml │ ├── docker-compose.yml │ ├── start-kafka-shell.sh │ └── start-kafka.sh ├── integration_test ├── lock │ ├── consul_test.exs │ └── zk_test.exs ├── support │ └── zk_helper.exs ├── test_helper.exs ├── topic │ └── server_test.exs └── zk │ ├── leader_test.exs │ ├── lock_test.exs │ └── util_test.exs ├── lib ├── cafex.ex ├── cafex │ ├── application.ex │ ├── connection.ex │ ├── consumer.ex │ ├── consumer │ │ ├── group_manager │ │ │ ├── kafka.ex │ │ │ └── zk.ex │ │ ├── load_balancer.ex │ │ ├── log_consumer.ex │ │ ├── manager.ex │ │ ├── offset_manager.ex │ │ ├── supervisor.ex │ │ ├── util.ex │ │ ├── worker.ex │ │ └── worker_partition.ex │ ├── kafka │ │ ├── group_coordinator.ex │ │ ├── heartbeat.ex │ │ ├── metadata.ex │ │ └── metadata_fsm.ex │ ├── lock.ex │ ├── lock │ │ ├── consul.ex │ │ ├── consul │ │ │ ├── heartbeat.ex │ │ │ ├── session.ex │ │ │ └── watch.ex │ │ ├── zk.ex │ │ └── zk │ │ │ └── server.ex │ ├── partitioner.ex │ ├── partitioner │ │ ├── hashed.ex │ │ └── random.ex │ ├── producer.ex │ ├── producer │ │ ├── supervisor.ex │ │ └── worker.ex │ ├── protocol.ex │ ├── protocol │ │ ├── codec.ex │ │ ├── compression.ex │ │ ├── decoder.ex │ │ ├── describe_groups.ex │ │ ├── errors.ex │ │ ├── fetch.ex │ │ ├── group_coordinator.ex │ │ ├── heartbeat.ex │ │ ├── join_group.ex │ │ ├── leave_group.ex │ │ ├── list_groups.ex │ │ ├── message.ex │ │ ├── metadata.ex │ │ ├── offset.ex │ │ ├── offset_commit.ex │ │ ├── offset_fetch.ex │ │ ├── produce.ex │ │ ├── request.ex │ │ └── sync_group.ex │ ├── supervisor.ex │ ├── topic │ │ ├── server.ex │ │ └── supervisor.ex │ ├── util.ex │ └── zk │ │ ├── leader.ex │ │ ├── lock.ex │ │ └── util.ex └── mix │ ├── cafex.ex │ └── tasks │ ├── cafex.groups.ex │ ├── cafex.offset_commit.ex │ ├── cafex.offset_fetch.ex │ ├── cafex.offset_reset.ex │ └── cafex.topic.ex ├── mix.exs ├── mix.lock └── test ├── cafex_test.exs ├── connection_test.exs ├── consumer ├── load_balancer_test.exs ├── offset_manager_test.exs ├── util_test.exs ├── worker_partition_test.exs └── worker_test.exs ├── protocol ├── codec_test.exs ├── describe_groups_test.exs ├── fetch_test.exs ├── group_coordinator_test.exs ├── heartbeat_test.exs ├── join_group_test.exs ├── leave_group_test.exs ├── list_groups_test.exs ├── metadata_test.exs ├── offset_commit_test.exs ├── offset_fetch_test.exs ├── offset_test.exs ├── produce_test.exs └── sync_group_test.exs ├── protocol_test.exs └── test_helper.exs /.gitignore: -------------------------------------------------------------------------------- 1 | /doc 2 | /docker/docker-compose.yml 3 | /config/dev.exs 4 | /config/test.exs 5 | /config/integration.exs 6 | 7 | # Created by https://www.gitignore.io/api/elixir 8 | 9 | ### Elixir ### 10 | /_build 11 | /cover 12 | /deps 13 | erl_crash.dump 14 | *.ez 15 | *.tar 16 | 17 | # Created by https://www.gitignore.io/api/tags 18 | 19 | ### Tags ### 20 | # Ignore tags created by etags, ctags, gtags (GNU global) and cscope 21 | TAGS 22 | .TAGS 23 | !TAGS/ 24 | tags 25 | .tags 26 | !tags/ 27 | gtags.files 28 | GTAGS 29 | GRTAGS 30 | GPATH 31 | cscope.files 32 | cscope.out 33 | cscope.in.out 34 | cscope.po.out 35 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Copyright (c) 2016 UPYUN. 2 | 3 | Licensed under the Apache License, Version 2.0 (the "License"); 4 | you may not use this file except in compliance with the License. 5 | You may obtain a copy of the License at 6 | 7 | http://www.apache.org/licenses/LICENSE-2.0 8 | 9 | Unless required by applicable law or agreed to in writing, software 10 | distributed under the License is distributed on an "AS IS" BASIS, 11 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | See the License for the specific language governing permissions and 13 | limitations under the License. 14 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | Cafex 2 | ===== 3 | 4 | Cafex is a pure Elixir implementation of [Kafka][kafka] client with [ZooKeeper][zookeeper] and [Consul][consul.io] intergration. 5 | 6 | Cafex support Kafka 0.8 and 0.9 group membership APIs. 7 | 8 | Cafex provides all kafka APIs encapsulation, producer implementation and high-level consumer implementation. 9 | 10 | ## Producer 11 | 12 | ### Example 13 | 14 | ```elixir 15 | iex> Application.start :cafex 16 | iex> topic_name = "test_topic" 17 | iex> brokers = [{"127.0.0.1", 9092}] 18 | iex> {:ok, producer} = Cafex.start_producer topic_name, client_id: "myproducer", 19 | brokers: brokers, 20 | partitioner: MyPartitioner, 21 | acks: 1, 22 | batch_num: 100, 23 | linger_ms: 10 24 | iex> Cafex.produce producer, "message", key: "key" 25 | iex> Cafex.async_produce producer, "message", key: "key" 26 | ``` 27 | 28 | ### Producer options 29 | 30 | #### `partitioner` 31 | 32 | The partitioner for partitioning messages amongst sub-topics. 33 | The default partitioner is `Cafex.Partitioner.Random`. 34 | 35 | #### `client_id` 36 | 37 | The client id is a user-specified string sent in each request to help trace 38 | calls. It should logically identify the application making the request. 39 | 40 | Default `cafex_producer`. 41 | 42 | #### `acks` 43 | 44 | The number of acknowledgments the producer requires the leader to have received 45 | before considering a request complete. This controls the durability of records 46 | that are sent. 47 | 48 | Default value is `1`. 49 | 50 | #### `batch_num` 51 | 52 | The number of messages to send in one batch when `linger_ms` is not zero. 53 | The producer will wait until either this number of messages are ready to send. 54 | 55 | #### `linger_ms` 56 | This setting is the same as `linger.ms` config in the new official producer configs. 57 | This setting defaults to 0 (i.e. no delay). 58 | 59 | > NOTE: If `linger_ms` is set to `0`, the `batch_num` will not take effect. 60 | 61 | ## Consumer 62 | 63 | ### Example 64 | 65 | ```elixir 66 | defmodule MyConsumer do 67 | use Cafex.Consumer 68 | 69 | def consume(msg, state) do 70 | # handle the msg 71 | {:ok, state} 72 | end 73 | end 74 | 75 | iex> Application.start :cafex 76 | iex> topic_name = "test_topic" 77 | iex> brokers = [{"127.0.0.1", 9092}] 78 | iex> options = [client_id: "myconsumer", 79 | topic: topic_name, 80 | brokers: brokers, 81 | offset_storage: :kafka, 82 | group_manager: :kafka, 83 | lock: :consul, 84 | group_session_timeout: 7000, # ms 85 | auto_commit: true, 86 | auto_commit_interval: 500, # ms 87 | auto_commit_max_buffers: 50, 88 | fetch_wait_time: 100, # ms 89 | fetch_min_bytes: 32 * 1024, 90 | fetch_max_bytes: 64 * 1024, 91 | handler: {MyConsumer, []}] 92 | iex> {:ok, consumer} = Cafex.start_consumer :myconsumer, options 93 | ``` 94 | 95 | The `options` argument of the function `start_consumer` can be put in the 96 | `config/config.exs`: 97 | 98 | ```elixir 99 | config :cafex, :myconsumer, 100 | client_id: "cafex", 101 | topic: "test_topic", 102 | brokers: [ 103 | {"192.168.99.100", 9092}, 104 | {"192.168.99.101", 9092} 105 | ], 106 | offset_storage: :kafka, 107 | group_manager: :kafka, 108 | lock: :consul, 109 | group_session_timeout: 7000, # ms 110 | auto_commit: true, 111 | auto_commit_interval: 500, # ms 112 | auto_commit_max_buffers: 50, 113 | fetch_wait_time: 100, # ms 114 | fetch_min_bytes: 32 * 1024, 115 | fetch_max_bytes: 64 * 1024, 116 | handler: {MyConsumer, []} 117 | ``` 118 | 119 | By default, cafex will use `:kafka` as the offset storage, use the new kafka 120 | group membership API, which was added in the 0.9.x, as the group manager, 121 | and use the `:consul` as the worker lock. Make suer your Kafka server is 0.9.x 122 | or above. 123 | 124 | But `:zookeeper` is another option for these. If you use zookeeper, the starting 125 | options of `:erlzk` must be specified under the `:zookeeper` key: 126 | 127 | ```elixir 128 | config :cafex, :myconsumer, 129 | client_id: "cafex", 130 | topic: "test_topic", 131 | brokers: [...], 132 | offset_storage: :zookeeper, 133 | group_manager: :zookeeper, 134 | lock: :zookeeper, 135 | zookeeper: [ 136 | timeout: 5000, 137 | servers: [{"192.168.99.100", 2181}], 138 | chroot: "/cafex" 139 | ], 140 | ... 141 | ``` 142 | 143 | ## TODO 144 | 145 | * Support kafka 0.10.x.x 146 | * Add tests 147 | 148 | [kafka]: http://kafka.apache.org 149 | [zookeeper]: http://zookeeper.apache.org 150 | [consul.io]: https://consul.io 151 | -------------------------------------------------------------------------------- /config/config.exs: -------------------------------------------------------------------------------- 1 | # This file is responsible for configuring your application 2 | # and its dependencies with the aid of the Mix.Config module. 3 | use Mix.Config 4 | 5 | # This configuration is loaded before any dependency and is restricted 6 | # to this project. If another project depends on this project, this 7 | # file won't be loaded nor affect the parent project. For this reason, 8 | # if you want to provide default values for your application for third- 9 | # party users, it should be done in your mix.exs file. 10 | 11 | # Sample configuration of consumers: 12 | # 13 | # config :cafex, :myconsumer, 14 | # client_id: "cafex", 15 | # topic: "a_topic", 16 | # brokers: [ 17 | # {"192.168.99.100", 9902}, 18 | # {"192.168.99.101", 9902}, 19 | # {"192.168.99.102", 9902} 20 | # ], 21 | # offset_storage: :kafka, # :kafka or :zookeeper 22 | # group_manager: :kafka, # :kafka or :zookeeper 23 | # lock: :consul, # :consul or :zookeeper 24 | # group_session_timeout: 7000, # ms 25 | # auto_commit: true, 26 | # auto_commit_interval: 500, # ms 27 | # auto_commit_max_buffers: 50, 28 | # auto_offset_reset: :latest, 29 | # fetch_wait_time: 100, # ms 30 | # fetch_min_bytes: 32 * 1024, 31 | # fetch_max_bytes: 1024 * 1024, 32 | # pre_fetch_size: 50, 33 | # handler: {Cafex.Consumer.LogConsumer, [level: :debug]} 34 | # 35 | # config :cafex, :myconsumer2, 36 | # client_id: "cafex", 37 | # group_manager: :zookeeper, 38 | # locke: :zookeeper, 39 | # zookeeper: [ 40 | # timeout: 5000, # ms 41 | # servers: [{"192.168.99.100", 2181}], 42 | # path: "/elena/cafex" 43 | # ], 44 | # handler: {Cafex.Consumer.LogConsumer, [level: :debug]} 45 | 46 | config :consul, 47 | host: "localhost", 48 | port: 8500 49 | 50 | config :logger, 51 | level: :debug 52 | 53 | config :logger, :console, 54 | format: "$date $time $metadata[$level] $levelpad$message\n" 55 | 56 | if File.exists? Path.join([__DIR__, "#{Mix.env}.exs"]) do 57 | import_config "#{Mix.env}.exs" 58 | end 59 | -------------------------------------------------------------------------------- /docker/docker-compose.yml.example: -------------------------------------------------------------------------------- 1 | kafka: 2 | build: ./kafka 3 | ports: 4 | - "9092" 5 | expose: 6 | - "9092" 7 | environment: 8 | KAFKA_ADVERTISED_HOST_NAME: 192.168.99.100 9 | KAFKA_ZOOKEEPER_CONNECT: 192.168.99.1:2181,192.168.99.1:2182,192.168.99.1:2183/kafka 10 | KAFKA_DELETE_TOPIC_EANBLE: true 11 | volumes: 12 | - /var/run/docker.sock:/var/run/docker.sock 13 | -------------------------------------------------------------------------------- /docker/kafka/Dockerfile: -------------------------------------------------------------------------------- 1 | FROM frolvlad/alpine-oraclejdk8:cleaned 2 | 3 | ENV KAFKA_VERSION="0.9.0.0" SCALA_VERSION="2.11" 4 | 5 | RUN apk add --update wget docker bash jq && \ 6 | rm -rf /var/cache/apk/* && \ 7 | mkdir /opt && \ 8 | wget -q -O - http://mirror.bit.edu.cn/apache/kafka/${KAFKA_VERSION}/kafka_${SCALA_VERSION}-${KAFKA_VERSION}.tgz | tar xzf - -C /opt && \ 9 | apk del wget 10 | 11 | VOLUME ["/kafka"] 12 | 13 | ENV KAFKA_HOME /opt/kafka_${SCALA_VERSION}-${KAFKA_VERSION} 14 | ADD start-kafka.sh /usr/bin/start-kafka.sh 15 | ADD broker-list.sh /usr/bin/broker-list.sh 16 | CMD start-kafka.sh 17 | -------------------------------------------------------------------------------- /docker/kafka/README.md: -------------------------------------------------------------------------------- 1 | kafka-docker 2 | ============ 3 | 4 | Dockerfile for [Apache Kafka](http://kafka.apache.org/) 5 | 6 | The image is available directly from https://registry.hub.docker.com/ 7 | 8 | ##Pre-Requisites 9 | 10 | - install docker-compose [https://docs.docker.com/compose/install/](https://docs.docker.com/compose/install/) 11 | - modify the ```KAFKA_ADVERTISED_HOST_NAME``` in ```docker-compose.yml``` to match your docker host IP (Note: Do not use localhost or 127.0.0.1 as the host ip if you want to run multiple brokers.) 12 | - if you want to customise any Kafka parameters, simply add them as environment variables in ```docker-compose.yml```, e.g. in order to increase the ```message.max.bytes``` parameter set the environment to ```KAFKA_MESSAGE_MAX_BYTES: 2000000```. To turn off automatic topic creation set ```KAFKA_AUTO_CREATE_TOPICS_ENABLE: 'false'``` 13 | 14 | ##Usage 15 | 16 | Start a cluster: 17 | 18 | - ```docker-compose up -d ``` 19 | 20 | Add more brokers: 21 | 22 | - ```docker-compose scale kafka=3``` 23 | 24 | Destroy a cluster: 25 | 26 | - ```docker-compose stop``` 27 | 28 | ##Note 29 | 30 | The default ```docker-compose.yml``` should be seen as a starting point. By default each broker will get a new port number and broker id on restart. Depending on your use case this might not be desirable. If you need to use specific ports and broker ids, modify the docker-compose configuration accordingly, e.g. [docker-compose-single-broker.yml](https://github.com/wurstmeister/kafka-docker/blob/master/docker-compose-single-broker.yml): 31 | 32 | - ```docker-compose -f docker-compose-single-broker.yml up``` 33 | 34 | ##Broker IDs 35 | 36 | If you don't specify a broker id in your docker-compose file, it will automatically be generated based on the name that docker-compose gives the container. This allows scaling up and down. In this case it is recommended to use the ```--no-recreate``` option of docker-compose to ensure that containers are not re-created and thus keep their names and ids. 37 | 38 | 39 | ##Automatically create topics 40 | 41 | If you want to have kafka-docker automatically create topics in Kafka during 42 | creation, a ```KAFKA_CREATE_TOPICS``` environment variable can be 43 | added in ```docker-compose.yml```. 44 | 45 | Here is an example snippet from ```docker-compose.yml```: 46 | 47 | environment: 48 | KAFKA_CREATE_TOPICS: "Topic1:1:3,Topic2:1:1" 49 | 50 | ```Topic 1``` will have 1 partition and 3 replicas, ```Topic 2``` will have 1 partition and 1 replica. 51 | 52 | ##Tutorial 53 | 54 | [http://wurstmeister.github.io/kafka-docker/](http://wurstmeister.github.io/kafka-docker/) 55 | 56 | 57 | 58 | -------------------------------------------------------------------------------- /docker/kafka/broker-list.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | CONTAINERS=$(docker ps | grep 9092 | awk '{print $1}') 4 | BROKERS=$(for CONTAINER in $CONTAINERS; do docker port $CONTAINER 9092 | sed -e "s/0.0.0.0:/$HOST_IP:/g"; done) 5 | echo $BROKERS | sed -e 's/ /,/g' 6 | -------------------------------------------------------------------------------- /docker/kafka/docker-compose-single-broker.yml: -------------------------------------------------------------------------------- 1 | zookeeper: 2 | image: wurstmeister/zookeeper 3 | ports: 4 | - "2181:2181" 5 | kafka: 6 | image: wurstmeister/kafka:0.9.0.0 7 | ports: 8 | - "9092:9092" 9 | links: 10 | - zookeeper:zk 11 | environment: 12 | KAFKA_ADVERTISED_HOST_NAME: 192.168.59.103 13 | volumes: 14 | - /var/run/docker.sock:/var/run/docker.sock 15 | -------------------------------------------------------------------------------- /docker/kafka/docker-compose.yml: -------------------------------------------------------------------------------- 1 | zookeeper: 2 | image: wurstmeister/zookeeper 3 | ports: 4 | - "2181" 5 | kafka: 6 | build: . 7 | ports: 8 | - "9092" 9 | links: 10 | - zookeeper:zk 11 | environment: 12 | KAFKA_ADVERTISED_HOST_NAME: 192.168.99.100 13 | volumes: 14 | - /var/run/docker.sock:/var/run/docker.sock 15 | -------------------------------------------------------------------------------- /docker/kafka/start-kafka-shell.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | docker run --rm -v /var/run/docker.sock:/var/run/docker.sock -e HOST_IP=$1 -e ZK=$2 -i -t docker_kafka /bin/bash 3 | -------------------------------------------------------------------------------- /docker/kafka/start-kafka.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | if [[ -z "$KAFKA_ADVERTISED_PORT" ]]; then 4 | export KAFKA_ADVERTISED_PORT=$(docker port `hostname` 9092 | sed -r "s/.*:(.*)/\1/g") 5 | fi 6 | if [[ -z "$KAFKA_BROKER_ID" ]]; then 7 | export KAFKA_BROKER_ID=$(docker inspect `hostname` | jq --raw-output '.[0] | .Name' | awk -F_ '{print $3}') 8 | fi 9 | if [[ -z "$KAFKA_LOG_DIRS" ]]; then 10 | export KAFKA_LOG_DIRS="/kafka/kafka-logs-$KAFKA_BROKER_ID" 11 | fi 12 | if [[ -z "$KAFKA_ZOOKEEPER_CONNECT" ]]; then 13 | export KAFKA_ZOOKEEPER_CONNECT=$(env | grep ZK.*PORT_2181_TCP= | sed -e 's|.*tcp://||' | join ,) 14 | fi 15 | 16 | if [[ -n "$KAFKA_HEAP_OPTS" ]]; then 17 | sed -r -i "s/(export KAFKA_HEAP_OPTS)=\"(.*)\"/\1=\"$KAFKA_HEAP_OPTS\"/g" $KAFKA_HOME/bin/kafka-server-start.sh 18 | unset KAFKA_HEAP_OPTS 19 | fi 20 | 21 | for VAR in `env` 22 | do 23 | if [[ $VAR =~ ^KAFKA_ && ! $VAR =~ ^KAFKA_HOME ]]; then 24 | kafka_name=`echo "$VAR" | sed -r "s/KAFKA_(.*)=.*/\1/g" | tr '[:upper:]' '[:lower:]' | tr _ .` 25 | env_var=`echo "$VAR" | sed -r "s/(.*)=.*/\1/g"` 26 | if egrep -q "(^|^#)$kafka_name=" $KAFKA_HOME/config/server.properties; then 27 | sed -r -i "s@(^|^#)($kafka_name)=(.*)@\2=${!env_var}@g" $KAFKA_HOME/config/server.properties #note that no config values may contain an '@' char 28 | else 29 | echo "$kafka_name=${!env_var}" >> $KAFKA_HOME/config/server.properties 30 | fi 31 | fi 32 | done 33 | 34 | 35 | $KAFKA_HOME/bin/kafka-server-start.sh $KAFKA_HOME/config/server.properties & 36 | KAFKA_SERVER_PID=$! 37 | 38 | while netstat -lnt | awk '$4 ~ /:9092$/ {exit 1}'; do sleep 1; done 39 | 40 | if [[ -n $KAFKA_CREATE_TOPICS ]]; then 41 | IFS=','; for topicToCreate in $KAFKA_CREATE_TOPICS; do 42 | IFS=':' read -a topicConfig <<< "$topicToCreate" 43 | $KAFKA_HOME/bin/kafka-topics.sh --create --zookeeper $KAFKA_ZOOKEEPER_CONNECT --replication-factor ${topicConfig[2]} --partition ${topicConfig[1]} --topic "${topicConfig[0]}" 44 | done 45 | fi 46 | 47 | wait $KAFKA_SERVER_PID 48 | -------------------------------------------------------------------------------- /integration_test/lock/consul_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Integration.Lock.ConsulTest do 2 | use ExUnit.Case, async: true 3 | 4 | @path "cafex_integration_test" 5 | 6 | alias Cafex.Lock.Consul, as: Lock 7 | 8 | test "consul lock" do 9 | {:ok, lock1} = Lock.acquire(@path) 10 | {:wait, lock2} = Lock.acquire(@path) 11 | assert :ok = Lock.release(lock2) 12 | 13 | {:wait, lock2} = Lock.acquire(@path) 14 | assert :ok = Lock.release(lock1) 15 | assert_receive {:lock, :ok, lock2} 16 | 17 | assert :ok = Lock.release(lock2) 18 | end 19 | end 20 | -------------------------------------------------------------------------------- /integration_test/lock/zk_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Integration.Lock.ZkTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Lock.ZK, as: Lock 5 | 6 | @path "/lock" 7 | @cfg Application.get_env(:cafex, :zookeeper) 8 | 9 | setup do 10 | servers = @cfg[:servers] |> Enum.map(fn {h, p} -> 11 | {:erlang.bitstring_to_list(h), p} 12 | end) 13 | opts = [servers: servers, 14 | timeout: @cfg[:timeout], 15 | chroot: @cfg[:chroot]] 16 | {:ok, opts: opts} 17 | end 18 | test "zk lock", context do 19 | opts = context[:opts] 20 | {:ok, lock1} = Lock.acquire(@path, opts) 21 | {:wait, lock2} = Lock.acquire(@path, opts) 22 | 23 | assert :ok = Lock.release(lock2) 24 | 25 | {:wait, lock2} = Lock.acquire(@path, opts) 26 | assert :ok = Lock.release(lock1) 27 | assert_receive {:lock, :ok, lock2} 28 | 29 | assert :ok = Lock.release(lock2) 30 | end 31 | end 32 | -------------------------------------------------------------------------------- /integration_test/support/zk_helper.exs: -------------------------------------------------------------------------------- 1 | defmodule ZKHelper do 2 | def connect(zk_cfg) do 3 | zk_servers = Keyword.get(zk_cfg, :servers) 4 | |> Enum.map(fn {h, p} -> {:erlang.bitstring_to_list(h), p} end) 5 | zk_timeout = Keyword.get(zk_cfg, :timeout) 6 | :erlzk.connect(zk_servers, zk_timeout) 7 | end 8 | 9 | def close(pid) do 10 | :erlzk.close(pid) 11 | end 12 | 13 | def rmr(pid, path) do 14 | case :erlzk.delete(pid, path) do 15 | :ok -> :ok 16 | {:error, :no_node} -> :ok 17 | {:error, :not_empty} -> 18 | case :erlzk.get_children(pid, path) do 19 | {:ok, children} -> 20 | Enum.each(children, fn child_path -> 21 | rmr(pid, Path.join(path, child_path)) 22 | end) 23 | rmr(pid, path) 24 | {:error, :no_node} -> 25 | :ok 26 | {:error, _} = error -> 27 | error 28 | end 29 | error -> error 30 | end 31 | end 32 | end 33 | -------------------------------------------------------------------------------- /integration_test/test_helper.exs: -------------------------------------------------------------------------------- 1 | Application.start :erlzk 2 | 3 | Code.require_file("support/zk_helper.exs", __DIR__) 4 | Logger.remove_backend(:console) 5 | 6 | zk_cfg = Application.get_env(:cafex, :zookeeper) 7 | zk_prefix = Keyword.get(zk_cfg, :chroot) 8 | {:ok, pid} = ZKHelper.connect(zk_cfg) 9 | :ok = ZKHelper.rmr(pid, zk_prefix) 10 | # ZKHelper.close(pid) 11 | 12 | ExUnit.start() 13 | -------------------------------------------------------------------------------- /integration_test/topic/server_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Integration.Topic.ServerTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Topic.Server 5 | alias Cafex.Protocol.Offset 6 | alias Cafex.Protocol.Fetch 7 | 8 | @default_topic "cafex_integration_test_topic" 9 | 10 | setup do 11 | topic = Application.get_env(:cafex, :topic, @default_topic) 12 | brokers = case Application.get_env(:cafex, :brokers) do 13 | nil -> Process.exit(self, "must set brokers for Integration test") 14 | brokers -> brokers 15 | end 16 | 17 | {:ok, pid} = Cafex.start_topic topic, brokers 18 | {:ok, topic_pid: pid, topic_name: topic, brokers: brokers} 19 | end 20 | 21 | test "Topic server", context do 22 | pid = context[:topic_pid] 23 | brokers = context[:brokers] 24 | topic_name = context[:topic_name] 25 | 26 | metadata = Server.metadata(pid) 27 | assert is_map(metadata) 28 | assert Map.has_key?(metadata, :name) 29 | assert Map.has_key?(metadata, :brokers) 30 | assert Map.has_key?(metadata, :leaders) 31 | assert Map.has_key?(metadata, :partitions) 32 | assert topic_name, metadata.name 33 | assert Enum.sort(brokers) == metadata.brokers |> HashDict.to_list |> Keyword.values |> Enum.sort 34 | 35 | Enum.map(0..metadata.partitions - 1, fn partition -> 36 | assert {:ok, %Offset.Response{offsets: [{^topic_name, [%{error: :no_error, offsets: [offset], partition: ^partition}]}]}} = Server.offset(pid, partition, :earliest, 1) 37 | assert {:ok, %Fetch.Response{topics: [{^topic_name, [%{error: :no_error, hwm_offset: _, messages: _messages, partition: ^partition}]}]}} = Server.fetch(pid, partition, offset) 38 | end) 39 | end 40 | end 41 | -------------------------------------------------------------------------------- /integration_test/zk/leader_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Integration.ZK.LeaderTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.ZK.Leader 5 | 6 | setup do 7 | zk_cfg = Application.get_env(:cafex, :zookeeper) 8 | zk_servers = Keyword.get(zk_cfg, :servers) 9 | |> Enum.map(fn {h, p} -> {:erlang.bitstring_to_list(h), p} end) 10 | zk_timeout = Keyword.get(zk_cfg, :timeout) 11 | chroot = Keyword.get(zk_cfg, :chroot) 12 | zk_prefix = "/leader_test" 13 | {:ok, pid1} = :erlzk.connect(zk_servers, zk_timeout, chroot: chroot) 14 | {:ok, pid2} = :erlzk.connect(zk_servers, zk_timeout, chroot: chroot) 15 | {:ok, pid3} = :erlzk.connect(zk_servers, zk_timeout, chroot: chroot) 16 | 17 | on_exit fn -> 18 | ZKHelper.rmr(pid1, zk_prefix) 19 | :erlzk.close pid1 20 | :erlzk.close pid2 21 | :erlzk.close pid3 22 | end 23 | 24 | {:ok, zk_prefix: zk_prefix, pid1: pid1, pid2: pid2, pid3: pid3} 25 | end 26 | 27 | test "leader election", context do 28 | pid1 = context[:pid1] 29 | pid2 = context[:pid2] 30 | pid3 = context[:pid3] 31 | prefix = context[:zk_prefix] 32 | 33 | path = Path.join(prefix, "leader_election_test") 34 | assert {:true, seq1} = Leader.election(pid1, path) 35 | assert {:false, seq2} = Leader.election(pid2, path) 36 | assert {:false, seq3} = Leader.election(pid3, path) 37 | 38 | assert :ok == :erlzk.delete(pid1, seq1) 39 | assert_receive {:leader_election, ^seq2} 40 | assert {:false, seq1} = Leader.election(pid1, path) 41 | 42 | assert :ok == :erlzk.delete(pid2, seq2) 43 | assert_receive {:leader_election, ^seq3} 44 | assert {:false, seq2} = Leader.election(pid2, path) 45 | 46 | assert :ok == :erlzk.delete(pid3, seq3) 47 | assert_receive {:leader_election, ^seq1} 48 | assert {:false, seq3} = Leader.election(pid3, path) 49 | 50 | [{pid1, seq1}, {pid2, seq2}, {pid3, seq3}] 51 | |> Enum.each fn {pid, seq} -> 52 | assert :ok == :erlzk.delete(pid, seq), "seq #{inspect seq} should be deleted" 53 | end 54 | 55 | elected = [pid1, pid2, pid3] 56 | |> Enum.shuffle 57 | |> Enum.map(fn pid -> 58 | Leader.election(pid, path) 59 | end) 60 | |> Enum.group_by(fn {leader, _seq} -> 61 | leader 62 | end) 63 | 64 | assert 1 == Map.get(elected, true) |> length 65 | assert 2 == Map.get(elected, false) |> length 66 | end 67 | end 68 | -------------------------------------------------------------------------------- /integration_test/zk/lock_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Integration.ZK.LockTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.ZK.Lock 5 | 6 | setup do 7 | zk_cfg = Application.get_env(:cafex, :zookeeper) 8 | zk_servers = Keyword.get(zk_cfg, :servers) 9 | |> Enum.map(fn {h, p} -> {:erlang.bitstring_to_list(h), p} end) 10 | zk_timeout = Keyword.get(zk_cfg, :timeout) 11 | chroot = Keyword.get(zk_cfg, :chroot) 12 | zk_prefix = "/lock_test" 13 | {:ok, pid} = :erlzk.connect(zk_servers, zk_timeout, chroot: chroot) 14 | 15 | on_exit fn -> 16 | ZKHelper.rmr(pid, zk_prefix) 17 | end 18 | 19 | {:ok, zk_pid: pid, zk_prefix: zk_prefix} 20 | end 21 | 22 | test "zk lock", context do 23 | pid = context[:zk_pid] 24 | prefix = context[:zk_prefix] 25 | 26 | lock_path = Path.join(prefix, "lock") 27 | assert {:ok, seq1} = Lock.acquire(pid, lock_path) 28 | 29 | assert {:error, :locked} == Lock.acquire(pid, lock_path) 30 | assert {:error, :timeout} == Lock.acquire(pid, lock_path, 10) 31 | assert {:wait, seq2} = Lock.acquire(pid, lock_path, :infinity) 32 | 33 | assert :ok == Lock.release(pid, seq1) 34 | assert_receive {:lock_again, seq2} 35 | assert {:ok, seq2} == Lock.reacquire(pid, lock_path, seq2) 36 | end 37 | 38 | end 39 | -------------------------------------------------------------------------------- /integration_test/zk/util_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Integration.ZK.UtilTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.ZK.Util 5 | 6 | setup_all do 7 | zk_cfg = Application.get_env(:cafex, :zookeeper) 8 | zk_servers = Keyword.get(zk_cfg, :servers) 9 | |> Enum.map(fn {h, p} -> {:erlang.bitstring_to_list(h), p} end) 10 | zk_timeout = Keyword.get(zk_cfg, :timeout) 11 | chroot = Keyword.get(zk_cfg, :chroot) 12 | zk_prefix = "/util_test" 13 | {:ok, pid} = :erlzk.connect(zk_servers, zk_timeout, chroot: chroot) 14 | 15 | on_exit fn -> 16 | ZKHelper.rmr(pid, zk_prefix) 17 | end 18 | 19 | {:ok, zk_pid: pid, zk_prefix: zk_prefix} 20 | end 21 | 22 | test "create node", context do 23 | pid = context[:zk_pid] 24 | prefix = context[:zk_prefix] 25 | 26 | path = Path.join [prefix, "/a", "/b"] 27 | assert {:error, :no_node} == :erlzk.exists(pid, path) 28 | assert :ok == Util.create_node(pid, path) 29 | assert {:ok, _} = :erlzk.exists(pid, path) 30 | assert :ok == Util.create_node(pid, path), "Should return :ok if node already exists" 31 | end 32 | 33 | test "create nodes", context do 34 | pid = context[:zk_pid] 35 | prefix = context[:zk_prefix] 36 | 37 | paths = [ 38 | Path.join([prefix, "/x", "/x1"]), 39 | Path.join([prefix, "/y", "/y1"]), 40 | Path.join([prefix, "/z", "/z1"]), 41 | ] 42 | 43 | Enum.each(paths, fn path -> 44 | assert {:error, :no_node} == :erlzk.exists(pid, path) 45 | end) 46 | 47 | assert :ok == Util.create_nodes(pid, paths) 48 | Enum.each(paths, fn path -> 49 | assert {:ok, _} = :erlzk.exists(pid, path) 50 | end) 51 | end 52 | 53 | test "get children", context do 54 | pid = context[:zk_pid] 55 | prefix = context[:zk_prefix] 56 | 57 | parent = Path.join prefix, "/parent_get_children_test" 58 | paths = [ 59 | Path.join([parent, "/c1"]), 60 | Path.join([parent, "/c2"]), 61 | Path.join([parent, "/c3"]), 62 | ] 63 | 64 | assert {:error, :no_node} == Util.get_children(pid, parent) 65 | assert :ok == Util.create_node(pid, parent) 66 | assert {:ok, []} == Util.get_children(pid, parent) 67 | assert {:ok, HashDict.new} == Util.get_children_with_data(pid, parent) 68 | 69 | assert :ok == Util.create_nodes(pid, paths) 70 | 71 | assert {:ok, children} = Util.get_children(pid, parent) 72 | assert ["c1", "c2", "c3"] == Enum.sort(children) 73 | 74 | assert {:ok, Enum.into([{"c1", ""}, {"c2", ""}, {"c3", ""}], HashDict.new)} == Util.get_children_with_data(pid, parent) 75 | 76 | Enum.each paths, fn path -> 77 | assert {:ok, _} = :erlzk.set_data(pid, path, path) 78 | end 79 | assert {:ok, Enum.map(paths, fn path -> 80 | {Path.basename(path), path} 81 | end)|> Enum.into(HashDict.new)} == Util.get_children_with_data(pid, parent) 82 | 83 | refute_received {_, _}, "Should not received any messages from erlzk" 84 | end 85 | 86 | test "get children with watcher", context do 87 | pid = context[:zk_pid] 88 | prefix = context[:zk_prefix] 89 | 90 | parent = Path.join(prefix, "/parent_get_children_with_watcher_test") 91 | assert {:error, :no_node} == Util.get_children(pid, parent, self) 92 | assert :ok == Util.create_node(pid, parent) 93 | refute_receive {:node_created, ^parent} 94 | 95 | child = Path.join(parent, "/ccc") 96 | assert {:ok, []} == Util.get_children(pid, parent, self) 97 | assert :ok == Util.create_node(pid, child) 98 | assert_receive {:node_children_changed, ^parent} 99 | 100 | assert {:ok, ["ccc"]} == Util.get_children(pid, parent, self) 101 | assert :ok == :erlzk.delete(pid, child) 102 | assert_receive {:node_children_changed, ^parent} 103 | end 104 | 105 | test "get data", context do 106 | pid = context[:zk_pid] 107 | prefix = context[:zk_prefix] 108 | 109 | parent = Path.join([prefix, "/parent_get_data_test"]) 110 | child = Path.join([parent, "/ccc1"]) 111 | assert {:error, :no_node} == Util.get_data(pid, child) 112 | assert :ok == Util.create_nodes(pid, [ parent, child ]) 113 | refute_receive {_, ^child} 114 | 115 | assert {:ok, {"", _stat}} = Util.get_data(pid, child) 116 | 117 | assert {:ok, _} = :erlzk.set_data(pid, child, "hello") 118 | refute_receive {:node_data_changed, ^child} 119 | 120 | assert {:ok, {"hello", _stat}} = Util.get_data(pid, child) 121 | 122 | refute_received {_, _}, "Should not received any messages from erlzk" 123 | end 124 | 125 | test "get data with watcher", context do 126 | pid = context[:zk_pid] 127 | prefix = context[:zk_prefix] 128 | 129 | parent = Path.join([prefix, "/parent_get_data_with_watcher_test"]) 130 | child = Path.join([parent, "/ccc1"]) 131 | 132 | assert {:error, :no_node} == Util.get_data(pid, child, self) 133 | assert :ok == Util.create_nodes(pid, [ parent, child ]) 134 | assert_receive {:node_created, ^child} 135 | 136 | assert {:ok, {"", _stat}} = Util.get_data(pid, child, self) 137 | 138 | assert {:ok, _} = :erlzk.set_data(pid, child, "hello") 139 | assert_receive {:node_data_changed, ^child} 140 | 141 | assert {:ok, {"hello", _stat}} = Util.get_data(pid, child, self) 142 | 143 | assert :ok == :erlzk.delete(pid, child) 144 | assert_receive {:node_deleted, ^child} 145 | end 146 | end 147 | -------------------------------------------------------------------------------- /lib/cafex.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex do 2 | 3 | @type server :: {host :: String.t, port :: 0..65535} 4 | @type broker :: server 5 | @type client_id :: String.t 6 | 7 | def start_topic(name, brokers, opts \\ []) do 8 | Cafex.Supervisor.start_topic(name, brokers, opts) 9 | end 10 | 11 | @doc """ 12 | Start a producer. 13 | 14 | Read `Cafex.Producer` for more details. 15 | """ 16 | @spec start_producer(topic_name :: String.t, opts :: Cafex.Producer.options) :: Supervisor.on_start_child 17 | def start_producer(topic_name, opts \\ []) do 18 | Cafex.Supervisor.start_producer(topic_name, opts) 19 | end 20 | defdelegate stop_producer(sup), to: Cafex.Supervisor 21 | 22 | @doc """ 23 | Produce message to kafka server in the synchronous way. 24 | 25 | See `Cafex.Producer.produce/3` 26 | """ 27 | def produce(producer, value, opts \\ []) do 28 | Cafex.Producer.produce(producer, value, opts) 29 | end 30 | 31 | @doc """ 32 | Produce message to kafka server in the asynchronous way. 33 | 34 | See `Cafex.Producer.produce/3` 35 | """ 36 | def async_produce(producer, value, opts \\ []) do 37 | Cafex.Producer.async_produce(producer, value, opts) 38 | end 39 | 40 | def fetch(topic_pid, partition, offset) when is_integer(partition) 41 | and is_integer(offset) do 42 | Cafex.Topic.Server.fetch topic_pid, partition, offset 43 | end 44 | 45 | @doc """ 46 | Start a consumer. 47 | 48 | Read `Cafex.Consumer.Manager` for more details. 49 | """ 50 | @spec start_consumer(name :: atom, Cafex.Consumer.Manager.options) :: Supervisor.on_start_child 51 | def start_consumer(name, opts \\ []) do 52 | Cafex.Supervisor.start_consumer(name, opts) 53 | end 54 | defdelegate stop_consumer(name), to: Cafex.Supervisor 55 | end 56 | -------------------------------------------------------------------------------- /lib/cafex/application.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Application do 2 | @moduledoc false 3 | 4 | use Application 5 | 6 | def start(_type, _args) do 7 | Cafex.Supervisor.start_link 8 | end 9 | 10 | def stop(_state) do 11 | end 12 | end 13 | -------------------------------------------------------------------------------- /lib/cafex/connection.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Connection do 2 | use GenServer 3 | 4 | require Logger 5 | 6 | alias Cafex.Protocol.Decoder 7 | 8 | defmodule State do 9 | @moduledoc false 10 | defstruct client_id: nil, 11 | correlation_id: 0, 12 | host: nil, 13 | port: nil, 14 | send_buffer: nil, 15 | timeout: nil, 16 | socket: nil 17 | end 18 | 19 | @default_client_id "cafex" 20 | @default_send_buffer 10_000_000 21 | @default_timeout 5000 22 | 23 | @type conn :: pid 24 | @type request :: Cafex.Protocol.Request.t 25 | @type receiver :: {:fsm, pid} | {:server, pid} | pid 26 | @type response :: Decoder.response 27 | 28 | # =================================================================== 29 | # API 30 | # =================================================================== 31 | 32 | def start_link(host, port, opts \\ []) do 33 | GenServer.start_link __MODULE__, [host, port, opts] 34 | end 35 | 36 | def start(host, port, opts \\ []) do 37 | GenServer.start __MODULE__, [host, port, opts] 38 | end 39 | 40 | @spec request(conn, request) :: :ok | {:ok, response} | {:error, term} 41 | def request(pid, request) do 42 | GenServer.call pid, {:request, request} 43 | end 44 | 45 | @spec async_request(conn, request, receiver) :: :ok 46 | def async_request(pid, request, receiver) do 47 | GenServer.cast pid, {:async_request, request, receiver} 48 | end 49 | 50 | def close(pid) do 51 | GenServer.call pid, :close 52 | end 53 | 54 | # =================================================================== 55 | # GenServer callbacks 56 | # =================================================================== 57 | 58 | def init([host, port, opts]) do 59 | client_id = Keyword.get(opts, :client_id, @default_client_id) 60 | buffer = Keyword.get(opts, :send_buffer, @default_send_buffer) 61 | timeout = Keyword.get(opts, :timeout, @default_timeout) 62 | 63 | state = %State{ client_id: client_id, 64 | host: host, 65 | port: port, 66 | send_buffer: buffer, 67 | timeout: timeout } |> maybe_open_socket 68 | {:ok, state} 69 | end 70 | 71 | def handle_call({:request, request}, _from, state) do 72 | case do_request(request, state) do 73 | {:ok, state} -> 74 | {:reply, :ok, state} 75 | {{:ok, reply}, state} -> 76 | {:reply, {:ok, reply}, state} 77 | {{:error, reason}, state} -> 78 | {:stop, reason, state} 79 | end 80 | end 81 | 82 | def handle_call(:close, _from, state) do 83 | {:stop, :normal, :ok, state} 84 | end 85 | 86 | def handle_cast({:async_request, request, receiver}, state) do 87 | case do_request(request, state) do 88 | {:ok, state} -> 89 | {:noreply, state} 90 | {{:ok, reply}, state} -> 91 | send_reply(receiver, {:ok, reply}) 92 | {:noreply, state} 93 | {{:error, reason}, state} -> 94 | {:stop, reason, state} 95 | end 96 | end 97 | 98 | def handle_info({:tcp_closed, socket}, %{socket: socket} = state) do 99 | Logger.info fn -> "Connection closed by peer" end 100 | {:noreply, %{state | socket: nil}} 101 | end 102 | 103 | def terminate(reason, %{host: host, port: port, socket: socket}) do 104 | Logger.debug "Connection closed: #{host}:#{port}, reason: #{inspect reason}" 105 | if socket, do: :gen_tcp.close(socket) 106 | :ok 107 | end 108 | 109 | # =================================================================== 110 | # Internal functions 111 | # =================================================================== 112 | 113 | defp maybe_open_socket(%{socket: nil, host: host, port: port, send_buffer: buffer} = state) do 114 | case :gen_tcp.connect(:erlang.bitstring_to_list(host), port, 115 | [:binary, {:packet, 4}, {:sndbuf, buffer}]) do 116 | {:ok, socket} -> 117 | %{state | socket: socket} 118 | {:error, reason} -> 119 | throw reason 120 | end 121 | end 122 | defp maybe_open_socket(state), do: state 123 | 124 | defp send_sync_request(socket, data, timeout, has_response) do 125 | case :gen_tcp.send(socket, data) do 126 | :ok -> 127 | case has_response do 128 | true -> recv_response(socket, timeout) 129 | false -> :ok 130 | end 131 | {:error, _reason} = error -> 132 | error 133 | end 134 | end 135 | 136 | defp recv_response(socket, timeout) do 137 | receive do 138 | {:tcp, ^socket, data} -> 139 | {:ok, data} 140 | {:tcp_closed, ^socket} -> 141 | {:error, :closed} 142 | after 143 | timeout -> 144 | {:error, :timeout} 145 | end 146 | end 147 | 148 | defp do_request(request, %{client_id: client_id, 149 | correlation_id: correlation_id, 150 | timeout: timeout} = state) do 151 | data = Cafex.Protocol.encode_request(client_id, correlation_id, request) 152 | has_response = Cafex.Protocol.has_response?(request) 153 | decoder = Cafex.Protocol.Request.decoder(request) 154 | 155 | state = %{state | correlation_id: correlation_id + 1} |> maybe_open_socket 156 | 157 | case send_sync_request(state.socket, data, timeout, has_response) do 158 | :ok -> 159 | {:ok, state} 160 | {:ok, data} -> 161 | {_, reply} = Cafex.Protocol.Codec.decode_response(decoder, data) 162 | {{:ok, reply}, state} 163 | {:error, reason} -> 164 | Logger.error "Error sending request to broker: #{state.host}:#{state.port}: #{inspect reason}" 165 | {{:error, reason}, state} 166 | end 167 | end 168 | 169 | defp send_reply({:fsm, pid}, reply) when is_pid(pid) do 170 | cast_send pid, {:"$gen_event", {:kafka_response, reply}} 171 | end 172 | defp send_reply({:server, pid}, reply) when is_pid(pid) do 173 | cast_send pid, {:"$gen_cast", {:kafka_response, reply}} 174 | end 175 | defp send_reply(pid, reply) when is_pid(pid) do 176 | cast_send pid, {:kafka_response, reply} 177 | end 178 | 179 | defp cast_send(dest, msg) do 180 | try do 181 | :erlang.send dest, msg, [:noconnect, :nosuspend] 182 | catch 183 | _, reason -> {:error, reason} 184 | end 185 | end 186 | end 187 | -------------------------------------------------------------------------------- /lib/cafex/consumer.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Consumer do 2 | @moduledoc """ 3 | Consumer worker implementation specification. 4 | 5 | ## Callbacks 6 | 7 | * `init(args)` 8 | 9 | * `consume(message, state)` 10 | 11 | * `terminate(state)` 12 | """ 13 | 14 | @type state :: term 15 | @type done :: :ok | :nocommit 16 | 17 | @callback init(args :: term) :: {:ok, state} | {:error, reason :: term} 18 | 19 | @callback consume(message :: Cafex.Protocol.Message.t, state) :: {done, state} | {:pause, timeout} 20 | 21 | @callback terminate(state) :: :ok 22 | 23 | @doc false 24 | defmacro __using__(_) do 25 | quote do 26 | @behaviour unquote(__MODULE__) 27 | 28 | def init(args), do: {:ok, args} 29 | def consume(_msg, state), do: {:ok, state} 30 | def terminate(_state), do: :ok 31 | 32 | defoverridable [init: 1, consume: 2, terminate: 1] 33 | end 34 | end 35 | end 36 | -------------------------------------------------------------------------------- /lib/cafex/consumer/load_balancer.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Consumer.LoadBalancer do 2 | @moduledoc """ 3 | Balance partition assignment between Cafex consumers 4 | """ 5 | 6 | @type layout :: [{node, [partition]}] 7 | @type partition :: non_neg_integer 8 | 9 | @doc """ 10 | Balance partition assignment between Cafex consumers 11 | 12 | ## Examples 13 | 14 | iex> rebalance [], 5 15 | [] 16 | 17 | iex> rebalance [{:a, [0, 1, 2, 3, 4]}], 5 18 | [{:a, [0, 1, 2, 3, 4]}] 19 | 20 | iex> rebalance [{:a, [0, 1, 2, 3, 4]}, {:b, []}], 5 21 | [{:a, [0, 1, 2]}, {:b, [3, 4]}] 22 | 23 | iex> rebalance [{:a, [0, 1, 2, 3, 4]}, {:b, []}, {:c, []}], 5 24 | [{:a, [0, 1]}, {:b, [2, 3]}, {:c, [4]}] 25 | 26 | iex> rebalance [{:a, [0, 1, 2]}, {:b, [3, 4]}, {:c, []}], 5 27 | [{:a, [0, 1]}, {:b, [3, 4]}, {:c, [2]}] 28 | 29 | iex> rebalance [{:a, [0, 1]}, {:c, [2]}], 5 30 | [{:a, [0, 1, 3]}, {:c, [2, 4]}] 31 | 32 | iex> rebalance [{:a, []}, {:b, [0, 1, 2, 3, 4]}], 5 33 | [{:a, [3, 4]}, {:b, [0, 1, 2]}] 34 | 35 | More details see the source of this module or test. 36 | """ 37 | @spec rebalance(layout, partitions :: non_neg_integer) :: layout 38 | def rebalance([], _partitions), do: [] 39 | def rebalance(layout, partitions) do 40 | consumers = Keyword.keys(layout) 41 | count = Float.floor(partitions / length(consumers)) |> trunc 42 | remainder = rem(partitions, length(consumers)) 43 | all = Enum.into(0..(partitions - 1), HashSet.new) 44 | 45 | assigned = layout |> Keyword.values 46 | |> List.flatten 47 | |> Enum.into(HashSet.new) 48 | not_assigned = all |> HashSet.difference(assigned) 49 | |> Enum.uniq 50 | |> Enum.sort 51 | 52 | {new_layout, [], 0} = 53 | layout |> Enum.sort(fn {_c1, p1}, {_c2, p2} -> 54 | length(p1) >= length(p2) 55 | end) 56 | |> Enum.reduce({[], not_assigned, remainder}, fn 57 | {consumer, partitions}, {layout, not_assigned, remainder} when remainder > 0 -> 58 | {keep, rest} = assign(partitions, count + 1, not_assigned) 59 | {[{consumer, keep}|layout], rest, remainder - 1} 60 | 61 | {consumer, partitions}, {layout, not_assigned, remainder} when remainder == 0 -> 62 | {keep, rest} = assign(partitions, count, not_assigned) 63 | {[{consumer, keep}|layout], rest, remainder} 64 | 65 | end) 66 | 67 | Enum.sort(new_layout) 68 | end 69 | 70 | defp assign(current, count, not_assigned) when length(current) > count do 71 | {partitions, rest} = Enum.split(current, count) 72 | {partitions, Enum.sort(rest ++ not_assigned)} 73 | end 74 | defp assign(current, count, not_assigned) when length(current) < count do 75 | {partitions, rest} = Enum.split(not_assigned, count - length(current)) 76 | {Enum.sort(current ++ partitions), rest} 77 | end 78 | defp assign(current, count, not_assigned) when length(current) == count do 79 | {current, not_assigned} 80 | end 81 | end 82 | -------------------------------------------------------------------------------- /lib/cafex/consumer/log_consumer.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Consumer.LogConsumer do 2 | @moduledoc """ 3 | A simple `Cafex.Consumer` implementation. 4 | 5 | It just print the message into logger output. 6 | """ 7 | use Cafex.Consumer 8 | 9 | @type options :: [level: Logger.level] 10 | @type state :: options 11 | 12 | require Logger 13 | 14 | @spec init(opts :: options) :: {:ok, state} 15 | def init(options) do 16 | {:ok, options} 17 | end 18 | 19 | @spec consume(Cafex.Protocol.Message.t, state) :: {:ok, state} 20 | def consume(message, state) 21 | def consume(%{offset: offset, key: key, value: value}, opts) do 22 | level = Keyword.get(opts, :level, :debug) 23 | Logger.log level, fn -> "{offset: #{offset}, key: #{key}, value: #{inspect value}}" end 24 | {:ok, opts} 25 | end 26 | end 27 | -------------------------------------------------------------------------------- /lib/cafex/consumer/supervisor.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Consumer.Supervisor do 2 | @moduledoc """ 3 | Manage consumers under the supervisor tree. 4 | """ 5 | 6 | use Supervisor 7 | 8 | @doc false 9 | def start_link do 10 | Supervisor.start_link __MODULE__, [], name: __MODULE__ 11 | end 12 | 13 | @doc """ 14 | Start a consumer manager. 15 | 16 | Read `Cafex.Consumer.Manager` for more details. 17 | """ 18 | @spec start_consumer(name :: atom, Cafex.Consumer.Manager.options) :: Supervisor.on_start_child 19 | def start_consumer(name, opts) do 20 | Supervisor.start_child __MODULE__, [name, opts] 21 | end 22 | 23 | defdelegate stop_consumer(name_or_pid), to: Cafex.Consumer.Manager, as: :stop 24 | 25 | @doc false 26 | def init([]) do 27 | children = [ 28 | worker(Cafex.Consumer.Manager, [], restart: :transient, 29 | shutdown: 2000) 30 | ] 31 | supervise children, strategy: :simple_one_for_one, 32 | max_restarts: 10, 33 | max_seconds: 60 34 | end 35 | end 36 | -------------------------------------------------------------------------------- /lib/cafex/consumer/util.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Consumer.Util do 2 | 3 | @doc """ 4 | Decode partitioins string stored outside. 5 | 6 | ## Example 7 | 8 | iex> decode_partitions(nil) 9 | [] 10 | iex> decode_partitions("") 11 | [] 12 | iex> decode_partitions("[1,2,3,4,5]") 13 | [1,2,3,4,5] 14 | iex> decode_partitions("[1,2,3,]") 15 | [1,2,3] 16 | iex> decode_partitions("[1,2,3") 17 | [1,2,3] 18 | iex> decode_partitions("1,2,3") 19 | [1,2,3] 20 | """ 21 | def decode_partitions(nil), do: [] 22 | def decode_partitions(""), do: [] 23 | def decode_partitions(value) do 24 | value |> String.lstrip(?[) 25 | |> String.rstrip(?]) 26 | |> String.split(",") 27 | |> Enum.filter(&(String.length(&1) > 0)) 28 | |> Enum.map(&String.to_integer/1) 29 | end 30 | 31 | @doc """ 32 | Encode partitions list to string. 33 | 34 | ## Example 35 | 36 | iex> encode_partitions(nil) 37 | "" 38 | iex> encode_partitions([]) 39 | "[]" 40 | iex> encode_partitions([1,2,3,4]) 41 | "[1,2,3,4]" 42 | """ 43 | def encode_partitions(nil), do: "" 44 | def encode_partitions(partitions) do 45 | "[#{ partitions |> Enum.map(&Integer.to_string/1) |> Enum.join(",") }]" 46 | end 47 | end 48 | -------------------------------------------------------------------------------- /lib/cafex/consumer/worker_partition.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Consumer.WorkerPartition do 2 | @moduledoc """ 3 | The partitions and their corresponding worker pids 4 | 5 | ``` 6 | { 7 | #Map{worker_pid => partition}, 8 | #Map{partition => worker_pid} 9 | } 10 | ``` 11 | 12 | """ 13 | 14 | def new do 15 | {%{}, %{}} 16 | end 17 | 18 | def partitions({_w2p, p2w}) do 19 | Map.keys(p2w) 20 | end 21 | 22 | def workers({w2p, _p2w}) do 23 | Map.keys(w2p) 24 | end 25 | 26 | def partition({w2p, _p2w}, worker) do 27 | Map.get(w2p, worker) 28 | end 29 | 30 | def worker({_w2p, p2w}, partition) do 31 | Map.get(p2w, partition) 32 | end 33 | 34 | def update({w2p, p2w} = index, partition, worker) do 35 | old_worker = worker(index, partition) 36 | old_partition = partition(index, worker) 37 | 38 | w2p = w2p 39 | |> Map.delete(old_worker) 40 | |> Map.put(worker, partition) 41 | 42 | p2w = p2w 43 | |> Map.delete(old_partition) 44 | |> Map.put(partition, worker) 45 | 46 | {w2p, p2w} 47 | end 48 | 49 | def delete({w2p, p2w}, partition, worker) do 50 | {Map.delete(w2p, worker), Map.delete(p2w, partition)} 51 | end 52 | 53 | def delete_by_worker(index, worker) do 54 | partition = partition(index, worker) 55 | delete(index, partition, worker) 56 | end 57 | 58 | def delete_by_partition(index, partition) do 59 | worker = worker(index, partition) 60 | delete(index, partition, worker) 61 | end 62 | end 63 | -------------------------------------------------------------------------------- /lib/cafex/kafka/group_coordinator.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Kafka.GroupCoordinator do 2 | use Cafex.Kafka.MetadataFsm 3 | 4 | require Logger 5 | alias Cafex.Protocol.GroupCoordinator.Request 6 | 7 | def make_request(group) do 8 | %Request{group_id: group} 9 | end 10 | 11 | def do_request(state) do 12 | case try_fetch_group_coordinator(state) do 13 | {:ok, result} -> %{state | result: result} 14 | {:error, error} -> %{state | error: error} 15 | end 16 | end 17 | 18 | defp try_fetch_group_coordinator(state) do 19 | try_fetch_group_coordinator(state, 10) 20 | end 21 | 22 | defp try_fetch_group_coordinator(_state, 0) do 23 | {:error, :group_coordinator_not_available} 24 | end 25 | defp try_fetch_group_coordinator(state, retries) when is_integer(retries) and retries > 0 do 26 | case send_request(state) do 27 | {:ok, %{error: :no_error, coordinator_host: host, coordinator_port: port}} -> 28 | {:ok, {host, port}} 29 | {:ok, %{error: :group_coordinator_not_available}} -> 30 | # We have to send a GroupCoordinatorRequest and retry with back off if 31 | # you receive a ConsumerCoordinatorNotAvailableCode returned as an 32 | # error. 33 | # See 34 | # [Kafka Error Codes](https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-ErrorCodes) 35 | # [issue on StackOverflow](http://stackoverflow.com/questions/28513744/kafka-0-8-2-consumermetadatarequest-always-return-consumercoordinatornotavailabl) 36 | Logger.warn "Try fetch group coordinator error: :group_coordinator_not_available, retry" 37 | :timer.sleep(2000) 38 | try_fetch_group_coordinator(state, retries - 1) 39 | {:ok, %{error: code}} -> 40 | Logger.error "Try fetch group coordinator error: #{inspect code}" 41 | {:error, code} 42 | {:error, reason} -> 43 | Logger.error "Try fetch group coordinator error: #{inspect reason}" 44 | {:error, reason} 45 | end 46 | end 47 | end 48 | -------------------------------------------------------------------------------- /lib/cafex/kafka/heartbeat.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Kafka.Heartbeat do 2 | use GenServer 3 | 4 | require Logger 5 | 6 | defmodule State do 7 | @moduledoc false 8 | defstruct pid: nil, 9 | conn: nil, 10 | request: nil, 11 | interval: nil 12 | end 13 | 14 | def start_link(pid, conn, request, interval) do 15 | GenServer.start_link __MODULE__, [pid, conn, request, interval] 16 | end 17 | 18 | def stop(pid), do: GenServer.call pid, :stop 19 | 20 | def init([pid, conn, request, interval]) do 21 | {:ok, %State{pid: pid, 22 | conn: conn, 23 | request: request, 24 | interval: interval}, 0} 25 | end 26 | 27 | def handle_call(:stop, _from, state) do 28 | {:stop, :normal, :ok, state} 29 | end 30 | 31 | def handle_info(:timeout, %{pid: pid, 32 | conn: conn, 33 | request: request, 34 | interval: interval} = state) do 35 | case beat(conn, request) do 36 | :ok -> 37 | {:noreply, state, interval} 38 | {:error, reason} -> 39 | Logger.warn "heartbeat error #{inspect reason}" 40 | send pid, {:no_heartbeat, reason} 41 | {:stop, :normal, state} 42 | error -> 43 | {:stop, error, state} 44 | end 45 | end 46 | 47 | defp beat(conn, request) do 48 | case Cafex.Connection.request(conn, request) do 49 | {:ok, %{error: :no_error}} -> 50 | :ok 51 | {:ok, %{error: error}} -> 52 | {:error, error} 53 | error -> 54 | error 55 | end 56 | end 57 | end 58 | -------------------------------------------------------------------------------- /lib/cafex/kafka/metadata.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Kafka.Metadata do 2 | use Cafex.Kafka.MetadataFsm 3 | 4 | require Logger 5 | 6 | alias Cafex.Protocol.Metadata.Request 7 | 8 | def make_request() do 9 | %Request{topics: []} 10 | end 11 | 12 | def make_request(nil), do: make_request 13 | def make_request(topic) do 14 | %Request{topics: [topic]} 15 | end 16 | 17 | def extract_metadata(metadata) do 18 | brokers = metadata.brokers |> Enum.map(fn b -> {b.node_id, {b.host, b.port}} end) 19 | |> Enum.into(%{}) 20 | 21 | [%{name: name, partitions: partitions}] = metadata.topics 22 | 23 | leaders = partitions |> Enum.map(fn p -> {p.partition_id, p.leader} end) 24 | |> Enum.into(%{}) 25 | 26 | %{name: name, brokers: brokers, leaders: leaders, partitions: length(partitions)} 27 | end 28 | 29 | def do_request(%{conn: conn, 30 | feed_brokers: [broker|rest], 31 | dead_brokers: deads} = state) do 32 | case send_request(state) do 33 | {:ok, %{topics: [%{error: :unknown_topic_or_partition}]}} -> 34 | %{state | error: :unknown_topic_or_partition} 35 | {:ok, %{topics: [%{error: :leader_not_available}]}} -> 36 | :timer.sleep(100) 37 | do_request(state) 38 | {:ok, metadata} -> 39 | %{state | result: metadata} 40 | {:error, reason} -> 41 | Logger.error "Failed to get metadata from: #{inspect conn}, reason: #{inspect reason}" 42 | %{state | feed_brokers: rest, dead_brokers: [broker|deads]} 43 | |> close_conn 44 | |> do_request 45 | end 46 | end 47 | end 48 | -------------------------------------------------------------------------------- /lib/cafex/kafka/metadata_fsm.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Kafka.MetadataFsm do 2 | @moduledoc false 3 | 4 | alias Cafex.Connection 5 | 6 | require Logger 7 | 8 | defmodule State do 9 | @moduledoc false 10 | @type t :: %__MODULE__{} 11 | defstruct from: nil, 12 | request: nil, 13 | conn: nil, 14 | error: nil, 15 | result: nil, 16 | feed_brokers: nil, 17 | dead_brokers: nil 18 | end 19 | 20 | @callback do_request(State.t) :: State.t 21 | @callback make_request(request :: term) :: request :: term 22 | 23 | defmacro __using__(_opts) do 24 | quote location: :keep do 25 | @behaviour :gen_fsm 26 | @behaviour unquote(__MODULE__) 27 | alias Cafex.Connection 28 | import unquote(__MODULE__) 29 | 30 | def start_link(brokers, request) do 31 | :gen_fsm.start_link __MODULE__, [brokers, request], [] 32 | end 33 | 34 | def request(brokers, request) do 35 | request = make_request(request) 36 | {:ok, pid} = __MODULE__.start_link(brokers, request) 37 | :gen_fsm.sync_send_event(pid, :request) 38 | end 39 | 40 | def make_request(request), do: request 41 | 42 | @doc false 43 | def init([brokers, request]) do 44 | state = %State{request: request, 45 | feed_brokers: Enum.shuffle(brokers), 46 | dead_brokers: []} 47 | {:ok, :prepared, state} 48 | end 49 | 50 | @doc false 51 | def prepared(:request, from, state) do 52 | %{state | from: from} 53 | |> open_conn 54 | |> do_request 55 | |> finalize 56 | end 57 | 58 | @doc false 59 | def handle_event(event, state_name, state) do 60 | {:stop, {:bad_event, state_name, event}, state} 61 | end 62 | 63 | @doc false 64 | def handle_sync_event(event, _from, state_name, state) do 65 | {:stop, {:bad_sync_event, state_name, event}, state} 66 | end 67 | 68 | @doc false 69 | def handle_info({_port, :closed}, :stopped, state) do 70 | {:stop, :shutdown, state} 71 | end 72 | 73 | @doc false 74 | def code_change(_old, state_name, state_data, _extra) do 75 | {:ok, state_name, state_data} 76 | end 77 | 78 | @doc false 79 | def terminate(_reason, _state_name, state_data) do 80 | close_conn(state_data) 81 | :ok 82 | end 83 | 84 | def do_request(%{conn: conn, request: request} = state) do 85 | case send_request(state) do 86 | {:ok, result} -> %{state | result: result} 87 | {:error, error} -> %{state | error: error} 88 | end 89 | end 90 | 91 | defoverridable [do_request: 1, make_request: 1] 92 | end 93 | end 94 | 95 | def finalize(%State{from: from, error: nil, result: result} = state) do 96 | :gen_fsm.reply from, {:ok, result} 97 | {:stop, :normal, state} 98 | end 99 | def finalize(%State{from: from, error: error} = state) do 100 | :gen_fsm.reply from, {:error, error} 101 | {:stop, :normal, state} 102 | end 103 | 104 | def open_conn(%State{conn: nil, 105 | feed_brokers: [], 106 | dead_brokers: deads} = state) do 107 | :timer.sleep(1000) 108 | open_conn(%{state | feed_brokers: Enum.shuffle(deads), 109 | dead_brokers: []}) 110 | end 111 | def open_conn(%State{conn: nil, 112 | feed_brokers: [{host, port}|rest], 113 | dead_brokers: deads} = state) do 114 | case Connection.start(host, port) do 115 | {:ok, pid} -> 116 | %{state|conn: pid} 117 | {:error, reason} -> 118 | Logger.warn "Open connection to #{host}:#{port} error: #{inspect reason}, try others" 119 | open_conn(%{state | conn: nil, feed_brokers: rest, 120 | dead_brokers: [{host, port}|deads]}) 121 | end 122 | end 123 | def open_conn(%State{conn: conn} = state) when is_pid(conn), do: state 124 | 125 | def close_conn(%State{conn: nil} = state), do: state 126 | def close_conn(%State{conn: conn} = state) do 127 | Connection.close(conn) 128 | %{state|conn: nil} 129 | end 130 | 131 | def send_request(%State{conn: conn, request: request}) do 132 | Connection.request(conn, request) 133 | end 134 | 135 | end 136 | -------------------------------------------------------------------------------- /lib/cafex/lock.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Lock do 2 | @moduledoc """ 3 | A general, backend-pluggable lock implementation. 4 | 5 | It's an FSM. 6 | """ 7 | 8 | @behaviour :gen_fsm 9 | 10 | require Logger 11 | 12 | @typedoc "`Cafex.Lock` module" 13 | @type locker :: atom 14 | 15 | @typedoc "Arguments for `locker`" 16 | @type args :: [term] 17 | 18 | @type state :: term 19 | 20 | @callback init(term) :: {:ok, state} | {:error, term} 21 | 22 | @doc """ 23 | Handle acquire callback 24 | 25 | Non-blocking function, return `{:ok, state}` if acquired the lock. 26 | Or `{:wait, state}` if waiting the lock, continue waiting in a asynchronized way(i.e. a process), 27 | then if lock changed, a `:lock_changed` message will send to this process. 28 | """ 29 | @callback handle_acquire(state) :: {:ok | :wait, state} | {:error, term} 30 | @callback handle_release(state) :: {:ok, state} | {:error, term} 31 | @callback terminate(state) :: :ok 32 | 33 | defmodule State do 34 | @moduledoc false 35 | defstruct [:locker, :timeout, :from, :data, :timeout_start] 36 | end 37 | 38 | # =================================================================== 39 | # Macros 40 | # =================================================================== 41 | 42 | defmacro __using__(_opts) do 43 | quote do 44 | @behaviour unquote(__MODULE__) 45 | def init(_args), do: {:ok, []} 46 | # def handle_acquire(state), do: {:ok, state} 47 | def handle_release(state), do: {:ok, state} 48 | def terminate(_state), do: :ok 49 | 50 | defoverridable [init: 1, handle_release: 1, terminate: 1] 51 | end 52 | end 53 | 54 | # =================================================================== 55 | # API 56 | # =================================================================== 57 | 58 | @doc """ 59 | Non-blocking function to acquire a lock 60 | 61 | The `args` is pass to `locker.init(args)`. 62 | 63 | If at first it hold the lock, this function will return `{:ok, pid}`. 64 | 65 | Or else, it will return `{:error, :locked}` and stop the FSM if `timeout` is `0`. 66 | Or it will return `{:wait, pid}` if `timeout` is other then `0`, and the FSM will 67 | wait for the lock until `timeout`. 68 | 69 | The caller will receive a message `{:lock, :ok}` if it get the lock before timeout, 70 | or `{:lock, :timeout}` and then stop. 71 | 72 | __NOTE__: The caller process is linked to the FSM process. 73 | """ 74 | @spec acquire(locker, args, timeout) :: {:ok, pid} | {:error, term} 75 | def acquire(locker, args, timeout \\ :infinity) do 76 | {:ok, pid} = :gen_fsm.start_link __MODULE__, [locker, args, timeout], [] 77 | case :gen_fsm.sync_send_event pid, :acquire, :infinity do 78 | :ok -> {:ok, pid} 79 | :wait -> {:wait, pid} 80 | error -> {:error, error} 81 | end 82 | end 83 | 84 | def release(pid) do 85 | :gen_fsm.sync_send_all_state_event pid, :release, :infinity 86 | end 87 | 88 | # =================================================================== 89 | # :gen_fsm callbacks 90 | # =================================================================== 91 | 92 | @doc false 93 | def init([locker, args, timeout]) do 94 | state = %State{timeout: timeout, locker: locker} 95 | {:ok, data} = locker.init(args) 96 | {:ok, :prepared, %{state | data: data}} 97 | end 98 | 99 | @doc false 100 | def prepared(:acquire, from, %{timeout: timeout} = state) do 101 | case do_acquire(state) do 102 | {:ok, data} -> 103 | {:reply, :ok, :locked, %{state | data: data}} 104 | {:wait, data} -> 105 | state = %{state | from: from, data: data, timeout_start: :os.timestamp} 106 | case timeout do 107 | 0 -> {:stop, :normal, {:error, :locked}, state} 108 | t -> {:reply, :wait, :waiting, state, t} 109 | end 110 | {:error, _reason} = error -> 111 | {:stop, :normal, error, state} 112 | end 113 | end 114 | 115 | @doc false 116 | def prepared(:timeout, %{from: {pid, _}, timeout_start: start, timeout: timeout} = state) do 117 | case do_acquire(state) do 118 | {:ok, data} -> 119 | send pid, {:lock, :ok, self} 120 | {:next_state, :locked, %{state | data: data}} 121 | {:wait, data} -> 122 | {:next_state, :waiting, %{state | data: data}, time_left(timeout, start)} 123 | {:error, _reason} = error -> 124 | {:stop, error, state} 125 | end 126 | end 127 | 128 | @doc false 129 | def waiting(:timeout, %{from: from} = state) do 130 | send from, {:lock, :timeout, self} 131 | {:stop, :normal, state} 132 | end 133 | 134 | @doc false 135 | def handle_event(event, state_name, state_data) do 136 | {:stop, {:bad_event, state_name, event}, state_data} 137 | end 138 | 139 | @doc false 140 | def handle_sync_event(:release, _from, _state_name, %{locker: locker, data: data} = state_data) do 141 | {:ok, data} = locker.handle_release(data) 142 | {:stop, :normal, :ok, %{state_data | data: data}} 143 | end 144 | def handle_sync_event(event, _from, state_name, state_data) do 145 | {:stop, {:bad_sync_event, state_name, event}, state_data} 146 | end 147 | 148 | @doc false 149 | def handle_info(:lock_changed, :waiting, state_data) do 150 | Logger.debug "lock_changed" 151 | {:next_state, :prepared, state_data, 0} 152 | end 153 | def handle_info(:lock_changed, :locked, state_data) do 154 | Logger.warn "The lock holder lose the lock" 155 | {:stop, :lose_lock, state_data} 156 | end 157 | def handle_info(info, state_name, state_data) do 158 | {:stop, {:bad_info, state_name, info}, state_data} 159 | end 160 | 161 | @doc false 162 | def terminate(_reason, _state_name, %{locker: locker, data: data}) do 163 | locker.terminate(data) 164 | :ok 165 | end 166 | 167 | @doc false 168 | def code_change(_old_vsn, state_name, state_data, _extra) do 169 | {:ok, state_name, state_data} 170 | end 171 | 172 | # =================================================================== 173 | # Internal functions 174 | # =================================================================== 175 | 176 | defp do_acquire(%{locker: locker, data: data}) do 177 | locker.handle_acquire(data) 178 | end 179 | 180 | defp time_left(:infinity, _start), do: :infinity 181 | defp time_left(timeout, start) when is_integer(timeout) do 182 | case timeout - div(:timer.now_diff(:os.timestamp, start), 1000) do 183 | t when t <= 0 -> 0 184 | t -> t 185 | end 186 | end 187 | end 188 | -------------------------------------------------------------------------------- /lib/cafex/lock/consul.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Lock.Consul do 2 | @moduledoc """ 3 | Distributed Lock implementation with Consul 4 | """ 5 | 6 | use Cafex.Lock 7 | 8 | require Logger 9 | 10 | alias Cafex.Lock.Consul.Watch 11 | alias Cafex.Lock.Consul.Session 12 | 13 | defmodule State do 14 | @moduledoc false 15 | defstruct [:session, :path, :lock, :watcher] 16 | end 17 | 18 | # =================================================================== 19 | # API 20 | # =================================================================== 21 | 22 | @doc """ 23 | ## Options 24 | 25 | All this options are defined as the Session's request body, 26 | see more [Session](https://www.consul.io/docs/agent/http/session.html) 27 | 28 | * `:ttl` Optional, based on millisecond, default is 10*1000 29 | * `:lock_delay` Optional, based on millisecond, default is 0, no delay 30 | * `:behavior` Optional, default is `:release` 31 | """ 32 | @spec acquire(String.t, Keyword.t) :: {:ok, pid} | {:error, term} 33 | def acquire(path, opts \\ []) do 34 | Cafex.Lock.acquire __MODULE__, [path, opts], :infinity 35 | end 36 | 37 | def release(pid) do 38 | Cafex.Lock.release pid 39 | end 40 | 41 | # =================================================================== 42 | # Cafex.Lock callbacks 43 | # =================================================================== 44 | 45 | def init([path, opts]) do 46 | {:ok, pid} = Session.start_link(opts) 47 | path = Path.join ["service", "cafex", path, "lock"] 48 | {:ok, %State{session: pid, path: path}} 49 | end 50 | 51 | def handle_acquire(%{path: path, session: pid} = state) do 52 | case Consul.Kv.put(path, "", acquire: Session.get(pid)) do 53 | true -> 54 | Logger.debug "Held the lock '#{path}'" 55 | {:ok, wait_change(%{state | lock: true})} 56 | false -> 57 | {:wait, wait_change(state)} 58 | end 59 | end 60 | 61 | def handle_release(%{lock: nil} = state), do: {:ok, state} 62 | def handle_release(%{path: path, session: pid} = state) do 63 | Logger.debug "Release the lock '#{path}'" 64 | case Consul.Kv.put(path, "", release: Session.get(pid)) do 65 | true -> 66 | {:ok, %{state | lock: nil}} 67 | error -> 68 | Logger.error("Consul error on putting release session request: #{inspect error}") 69 | {:error, :consul_error} 70 | end 71 | end 72 | 73 | def terminate(%{session: pid} = state) do 74 | handle_release state 75 | Session.stop pid 76 | :ok 77 | end 78 | 79 | # =================================================================== 80 | # Internal functions 81 | # =================================================================== 82 | 83 | defp wait_change(%{path: path} = state) do 84 | {:ok, %{body: [body]} = response} = Consul.Kv.fetch(path) 85 | if is_nil(body["Session"]) do 86 | send self, :lock_changed 87 | state 88 | else 89 | index = Consul.Response.consul_index response 90 | start_watcher(index, state) 91 | end 92 | end 93 | 94 | defp start_watcher(index, %{watcher: pid, path: path} = state) do 95 | if is_nil(pid) or not Process.alive?(pid) do 96 | {:ok, pid} = Watch.start_link(path, index, self) 97 | %{state | watcher: pid} 98 | else 99 | state 100 | end 101 | end 102 | end 103 | -------------------------------------------------------------------------------- /lib/cafex/lock/consul/heartbeat.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Lock.Consul.Heartbeat do 2 | @moduledoc false 3 | use GenServer 4 | 5 | require Logger 6 | 7 | defmodule State do 8 | @moduledoc false 9 | defstruct [:timer, :session, :interval] 10 | end 11 | 12 | def start_link(session, interval) do 13 | GenServer.start_link __MODULE__, [session, interval] 14 | end 15 | 16 | def stop(pid) do 17 | GenServer.call pid, :stop 18 | end 19 | 20 | def init([session, interval]) do 21 | {:ok, %State{session: session, interval: interval}, 0} 22 | end 23 | 24 | def handle_call(:stop, _from, state) do 25 | {:stop, :normal, :ok, state} 26 | end 27 | 28 | def handle_info(:timeout, state) do 29 | handle_info(:beat, state) 30 | end 31 | 32 | def handle_info(:beat, %{timer: timer, session: session, interval: interval} = state) do 33 | unless is_nil(timer) do 34 | :erlang.cancel_timer(timer) 35 | end 36 | 37 | case Consul.Session.renew session do 38 | {:ok, _ret} -> 39 | new_timer = :erlang.send_after(interval, self, :beat) 40 | {:noreply, %{state | timer: new_timer}} 41 | error -> 42 | Logger.warn "Consul session [#{inspect session}] renew error: #{inspect error}" 43 | {:stop, :consul_error, state} 44 | end 45 | end 46 | end 47 | -------------------------------------------------------------------------------- /lib/cafex/lock/consul/session.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Lock.Consul.Session do 2 | use GenServer 3 | 4 | alias Consul.Session 5 | alias Cafex.Lock.Consul.Heartbeat 6 | 7 | @default_behavior :release 8 | @default_ttl 10 * 1000 9 | 10 | defmodule State do 11 | @moduledoc false 12 | defstruct [:id, :heartbeat] 13 | end 14 | 15 | def start_link(opts \\ []) do 16 | GenServer.start_link __MODULE__, [opts] 17 | end 18 | 19 | def get(pid) do 20 | GenServer.call pid, :get 21 | end 22 | 23 | def stop(pid) do 24 | GenServer.call pid, :stop 25 | end 26 | 27 | def init([opts]) do 28 | behavior = Keyword.get(opts, :behavior, @default_behavior) |> to_string 29 | lock_delay_ms = Keyword.get opts, :lock_delay, 0 30 | ttl_ms = Keyword.get opts, :ttl, @default_ttl 31 | lock_delay = (lock_delay_ms |> div(1000) |> Integer.to_string) <> "s" 32 | ttl = (ttl_ms |> div(1000) |> Integer.to_string) <> "s" 33 | 34 | id = 35 | %{:Name => Keyword.get(opts, :name), 36 | :Node => Keyword.get(opts, :node_name), 37 | :LockDelay => lock_delay, 38 | :TTL => ttl, 39 | :Behavior => behavior} 40 | |> Enum.filter(fn {_k, v} -> v end) 41 | |> Enum.into(%{}) 42 | |> Session.create! 43 | 44 | {:ok, pid} = Heartbeat.start_link(id, div(ttl_ms, 3) * 2) 45 | state = %State{id: id, heartbeat: pid} 46 | {:ok, state} 47 | end 48 | 49 | def handle_call(:get, _from, %{id: id} = state) do 50 | {:reply, id, state} 51 | end 52 | 53 | def handle_call(:stop, _from, state) do 54 | {:stop, :normal, :ok, state} 55 | end 56 | 57 | def terminate(_reason, %{id: id, heartbeat: pid} = _state) do 58 | Heartbeat.stop pid 59 | do_destroy(id) 60 | :ok 61 | end 62 | 63 | defp do_destroy(id) do 64 | Session.destroy id 65 | end 66 | end 67 | -------------------------------------------------------------------------------- /lib/cafex/lock/consul/watch.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Lock.Consul.Watch do 2 | @moduledoc false 3 | use GenServer 4 | 5 | @wait "5m" 6 | @retry_ms 30 * 1000 7 | 8 | defmodule State do 9 | @moduledoc false 10 | defstruct [:path, :index, :from] 11 | end 12 | 13 | def start_link(path, index, from) do 14 | GenServer.start_link __MODULE__, [path, index, from] 15 | end 16 | 17 | def init([path, index, from]) do 18 | {:ok, %State{path: path, index: index, from: from}, 0} 19 | end 20 | 21 | def handle_info(:timeout, state) do 22 | do_wait(state) 23 | end 24 | 25 | defp do_wait(%{path: path, index: index, from: from} = state) do 26 | # blocking query 27 | case Consul.Kv.fetch(path, wait: @wait, index: index) do 28 | {:ok, %{body: _body} = response} -> 29 | case Consul.Response.consul_index(response) do 30 | ^index -> 31 | {:noreply, state, 0} 32 | _ -> 33 | send from, :lock_changed 34 | {:stop, :normal, state} 35 | end 36 | {:error, %{reason: :timeout}} -> 37 | {:noreply, state, 0} 38 | {:error, _} -> 39 | {:noreply, state, @retry_ms} 40 | end 41 | end 42 | end 43 | -------------------------------------------------------------------------------- /lib/cafex/lock/zk.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Lock.ZK do 2 | use Cafex.Lock 3 | 4 | require Logger 5 | 6 | @chroot '/cafex' 7 | @timeout 5000 8 | 9 | defmodule State do 10 | @moduledoc false 11 | defstruct zk: nil, 12 | zk_server: nil, 13 | lock: {false, nil}, 14 | path: nil, 15 | timeout: nil 16 | end 17 | 18 | # =================================================================== 19 | # API 20 | # =================================================================== 21 | 22 | def acquire(path, opts \\ []) do 23 | Cafex.Lock.acquire __MODULE__, [path, opts], :infinity 24 | end 25 | 26 | def release(pid) do 27 | Cafex.Lock.release pid 28 | end 29 | 30 | # =================================================================== 31 | # Cafex.Lock callbacks 32 | # =================================================================== 33 | 34 | def init([path, opts]) do 35 | timeout = Keyword.get(opts, :timeout, @timeout) 36 | servers = Keyword.get(opts, :servers) 37 | chroot = Keyword.get(opts, :chroot, @chroot) |> :erlang.binary_to_list 38 | path = Path.join("/", path) 39 | 40 | {:ok, zk_pid} = :erlzk.connect(servers, timeout, [chroot: chroot]) 41 | {:ok, zk_server} = Cafex.Lock.ZK.Server.start_link self 42 | state = %State{zk: zk_pid, zk_server: zk_server, path: path, timeout: timeout} 43 | {:ok, state} 44 | end 45 | 46 | def handle_acquire(%{lock: {false, seq}, path: path, zk: zk_pid, zk_server: zk_server} = state) do 47 | case Cafex.Lock.ZK.Server.acquire(zk_server, zk_pid, path, seq) do 48 | {:ok, seq} -> 49 | Logger.debug "Held the lock #{inspect self}" 50 | {:ok, %{state | lock: {true, seq}}} 51 | {:wait, seq} -> 52 | state = %{state | lock: {false, seq}} 53 | {:wait, state} 54 | end 55 | end 56 | 57 | def handle_release(%{lock: {_, nil}} = state), do: {:ok, state} 58 | def handle_release(%{lock: {_, seq}, zk_server: zk_server, zk: zk_pid} = state) do 59 | Cafex.Lock.ZK.Server.release(zk_server, zk_pid, seq) 60 | {:ok, %{state | lock: {false, nil}}} 61 | end 62 | 63 | def terminate(%{zk_server: zk_server, zk: zk_pid} = state) do 64 | handle_release(state) 65 | Cafex.Lock.ZK.Server.stop(zk_server) 66 | :erlzk.close(zk_pid) 67 | end 68 | 69 | end 70 | -------------------------------------------------------------------------------- /lib/cafex/lock/zk/server.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Lock.ZK.Server do 2 | use GenServer 3 | 4 | def start_link(parent) do 5 | GenServer.start_link __MODULE__, [parent] 6 | end 7 | 8 | def acquire(pid, zk_pid, path, seq) do 9 | GenServer.call pid, {:acquire, zk_pid, path, seq} 10 | end 11 | 12 | def release(pid, zk_pid, seq) do 13 | GenServer.call pid, {:release, zk_pid, seq} 14 | end 15 | 16 | def stop(pid) do 17 | GenServer.call pid, :stop 18 | end 19 | 20 | def init([parent]) do 21 | {:ok, parent} 22 | end 23 | 24 | def handle_call({:acquire, zk_pid, path, nil}, _from, state) do 25 | reply = Cafex.ZK.Lock.acquire(zk_pid, path, :infinity) 26 | {:reply, reply, state} 27 | end 28 | def handle_call({:acquire, zk_pid, path, seq}, _from, state) do 29 | reply = Cafex.ZK.Lock.reacquire(zk_pid, path, seq, :infinity) 30 | {:reply, reply, state} 31 | end 32 | 33 | def handle_call({:release, zk_pid, seq}, _from, state) do 34 | Cafex.ZK.Lock.release(zk_pid, seq) 35 | {:reply, :ok, state} 36 | end 37 | 38 | def handle_call(:stop, _from, state) do 39 | {:stop, :normal, :ok, state} 40 | end 41 | 42 | def handle_info({:lock_again, _seq}, parent) do 43 | send parent, :lock_changed 44 | {:noreply, parent} 45 | end 46 | 47 | def terminate(_reason, _state) do 48 | :ok 49 | end 50 | 51 | end 52 | -------------------------------------------------------------------------------- /lib/cafex/partitioner.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Partitioner do 2 | @moduledoc """ 3 | Partitioner implementation specification for Kafka produce request API. 4 | 5 | ## Callbacks 6 | 7 | * `init(partitions)` 8 | 9 | * `partition(message, state)` 10 | """ 11 | 12 | @type state :: term 13 | 14 | @callback init(partitions :: integer) :: {:ok, state} | {:error, term} 15 | 16 | @callback partition(message :: Cafex.Protocol.Message.t, state) :: {integer, state} 17 | end 18 | -------------------------------------------------------------------------------- /lib/cafex/partitioner/hashed.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Partitioner.Hashed do 2 | @moduledoc """ 3 | Hashed partitioner implementation. 4 | 5 | Read `Cafex.Partitioner` behaviour. 6 | """ 7 | 8 | @behaviour Cafex.Partitioner 9 | 10 | def init(partitions) do 11 | {:ok, partitions} 12 | end 13 | 14 | def partition(message, partitions) do 15 | hash = :erlang.phash2(message.key, partitions) 16 | {hash, partitions} 17 | end 18 | end 19 | -------------------------------------------------------------------------------- /lib/cafex/partitioner/random.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Partitioner.Random do 2 | @moduledoc """ 3 | Random partitioner implementation. 4 | 5 | Read `Cafex.Partitioner` behaviour. 6 | """ 7 | 8 | @behaviour Cafex.Partitioner 9 | 10 | def init(partitions) do 11 | :rand.seed(:exs1024) 12 | {:ok, partitions} 13 | end 14 | 15 | def partition(_message, partitions) do 16 | {:rand.uniform(partitions) - 1, partitions} 17 | end 18 | end 19 | -------------------------------------------------------------------------------- /lib/cafex/producer.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Producer do 2 | @moduledoc """ 3 | Kafka producer 4 | """ 5 | 6 | use GenServer 7 | 8 | @default_client_id "cafex_producer" 9 | @default_acks 1 10 | @default_batch_num 200 11 | # @default_max_request_size 1024 * 1024 12 | @default_linger_ms 0 13 | @default_timeout 60000 14 | 15 | @typedoc "Options used by the `start_link/2` functions" 16 | @type options :: [option] 17 | @type option :: {:client_id, Cafex.client_id} | 18 | {:brokers, [Cafex.broker]} | 19 | {:acks, -1..32767} | 20 | {:batch_num, pos_integer} | 21 | {:linger_ms, non_neg_integer} 22 | 23 | require Logger 24 | 25 | alias Cafex.Protocol.Message 26 | 27 | defmodule State do 28 | @moduledoc false 29 | defstruct topic: nil, 30 | topic_name: nil, 31 | feed_brokers: [], 32 | partitioner: nil, 33 | partitioner_state: nil, 34 | brokers: nil, 35 | leaders: nil, 36 | partitions: 0, 37 | workers: %{}, 38 | client_id: nil, 39 | worker_opts: nil 40 | end 41 | 42 | # =================================================================== 43 | # API 44 | # =================================================================== 45 | 46 | @spec start_link(topic_name :: String.t, opts :: options) :: GenServer.on_start 47 | def start_link(topic_name, opts) do 48 | GenServer.start_link __MODULE__, [topic_name, opts] 49 | end 50 | 51 | @doc """ 52 | Produce message to kafka server in the synchronous way. 53 | 54 | ## Options 55 | 56 | * `:key` The key is an optional message key that was used for partition assignment. The key can be `nil`. 57 | * `:partition` The partition that data is being published to. 58 | * `:metadata` The metadata is used for partition in case of you wan't to use key to do that. 59 | """ 60 | @spec produce(pid :: pid, value :: binary, opts :: [Keyword.t]) :: :ok | {:error, term} 61 | def produce(pid, value, opts \\ []) do 62 | key = Keyword.get(opts, :key) 63 | partition = Keyword.get(opts, :partition) 64 | metadata = Keyword.get(opts, :metadata) 65 | message = %Message{key: key, value: value, partition: partition, metadata: metadata} 66 | worker_pid = GenServer.call pid, {:get_worker, message} 67 | Cafex.Producer.Worker.produce(worker_pid, message) 68 | end 69 | 70 | @doc """ 71 | Produce message to kafka server in the asynchronous way. 72 | 73 | ## Options 74 | 75 | See `produce/3` 76 | """ 77 | @spec async_produce(pid :: pid, value :: binary, opts :: [Keyword.t]) :: :ok 78 | def async_produce(pid, value, opts \\ []) do 79 | key = Keyword.get(opts, :key) 80 | partition = Keyword.get(opts, :partition) 81 | 82 | message = %Message{key: key, value: value, partition: partition} 83 | worker_pid = GenServer.call pid, {:get_worker, message} 84 | Cafex.Producer.Worker.async_produce(worker_pid, message) 85 | end 86 | 87 | @spec stop(pid :: pid) :: :ok 88 | def stop(pid) do 89 | GenServer.call pid, :stop 90 | end 91 | 92 | # =================================================================== 93 | # GenServer callbacks 94 | # =================================================================== 95 | 96 | def init([topic_name, opts]) do 97 | Process.flag(:trap_exit, true) 98 | 99 | client_id = Keyword.get(opts, :client_id, @default_client_id) 100 | brokers = Keyword.get(opts, :brokers) 101 | acks = Keyword.get(opts, :acks, @default_acks) 102 | batch_num = Keyword.get(opts, :batch_num, @default_batch_num) 103 | # max_request_size = Keyword.get(opts, :max_request_size, @default_max_request_size) 104 | linger_ms = Keyword.get(opts, :linger_ms, @default_linger_ms) 105 | compression = Keyword.get(opts, :compression) 106 | 107 | state = %State{topic_name: topic_name, 108 | feed_brokers: brokers, 109 | client_id: client_id, 110 | worker_opts: [ 111 | client_id: client_id, 112 | acks: acks, 113 | batch_num: batch_num, 114 | # max_request_size: max_request_size, 115 | linger_ms: linger_ms, 116 | timeout: @default_timeout, 117 | compression: compression 118 | ]} |> load_metadata 119 | |> start_workers 120 | 121 | partitioner = Keyword.get(opts, :partitioner, Cafex.Partitioner.Random) 122 | {:ok, partitioner_state} = partitioner.init(state.partitions) 123 | 124 | {:ok, %{state | partitioner: partitioner, 125 | partitioner_state: partitioner_state}} 126 | end 127 | 128 | def handle_call({:get_worker, message}, _from, state) do 129 | {worker, state} = dispatch(message, state) 130 | {:reply, worker, state} 131 | end 132 | 133 | def handle_call(:stop, _from, state) do 134 | {:stop, :normal, :ok, state} 135 | end 136 | 137 | def handle_info({:EXIT, pid, reason}, %{workers: workers} = state) do 138 | state = 139 | case Enum.find(workers, fn {_k, v} -> v == pid end) do 140 | nil -> 141 | state 142 | {k, _} -> 143 | Logger.error "Producer worker down: #{inspect reason}, restarting" 144 | start_worker(k, %{state | workers: Map.delete(workers, k)}) 145 | end 146 | {:noreply, state} 147 | end 148 | 149 | def terminate(_reason, %{workers: workers}=_state) do 150 | for {_, pid} <- workers do 151 | Cafex.Producer.Worker.stop pid 152 | end 153 | :ok 154 | end 155 | 156 | # =================================================================== 157 | # Internal functions 158 | # =================================================================== 159 | 160 | defp dispatch(%{partition: nil} = message, %{partitioner: partitioner, 161 | partitioner_state: partitioner_state, 162 | workers: workers} = state) do 163 | {partition, new_state} = partitioner.partition(message, partitioner_state) 164 | # TODO: check partition availability 165 | worker_pid = Map.get(workers, partition) 166 | {worker_pid, %{state | partitioner_state: new_state}} 167 | end 168 | defp dispatch(%{partition: partition}, %{workers: workers} = state) do 169 | # TODO: check partition availability 170 | worker_pid = Map.get(workers, partition) 171 | {worker_pid, state} 172 | end 173 | 174 | defp load_metadata(%{feed_brokers: brokers, topic_name: topic} = state) do 175 | {:ok, metadata} = Cafex.Kafka.Metadata.request(brokers, topic) 176 | metadata = Cafex.Kafka.Metadata.extract_metadata(metadata) 177 | 178 | %{state | topic: metadata.name, 179 | brokers: metadata.brokers, 180 | leaders: metadata.leaders, 181 | partitions: metadata.partitions} 182 | end 183 | 184 | defp start_workers(%{partitions: partitions} = state) do 185 | Enum.reduce 0..(partitions - 1), state, fn partition, acc -> 186 | start_worker(partition, acc) 187 | end 188 | end 189 | 190 | defp start_worker(partition, %{topic: topic, brokers: brokers, 191 | leaders: leaders, workers: workers, 192 | worker_opts: worker_opts} = state) do 193 | leader = Map.get(leaders, partition) 194 | broker = Map.get(brokers, leader) 195 | Logger.debug fn -> "Starting producer worker { topic: #{topic}, partition: #{partition}, broker: #{inspect broker} } ..." end 196 | {:ok, pid} = Cafex.Producer.Worker.start_link(broker, topic, partition, worker_opts) 197 | %{state | workers: Map.put(workers, partition, pid)} 198 | end 199 | 200 | end 201 | -------------------------------------------------------------------------------- /lib/cafex/producer/supervisor.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Producer.Supervisor do 2 | @moduledoc """ 3 | Manage producers under the supervisor tree. 4 | """ 5 | 6 | use Supervisor 7 | 8 | @doc false 9 | def start_link do 10 | Supervisor.start_link __MODULE__, [], name: __MODULE__ 11 | end 12 | 13 | @spec start_producer(topic :: String.t, Cafex.Producer.options) :: Supervisor.on_start_child 14 | def start_producer(topic, opts) do 15 | Supervisor.start_child __MODULE__, [topic, opts] 16 | end 17 | 18 | defdelegate stop_producer(pid), to: Cafex.Producer, as: :stop 19 | 20 | @doc false 21 | def init([]) do 22 | children = [ 23 | worker(Cafex.Producer, [], restart: :temporary, 24 | shutdown: 2000) 25 | ] 26 | supervise children, strategy: :simple_one_for_one, 27 | max_restarts: 10, 28 | max_seconds: 60 29 | end 30 | end 31 | -------------------------------------------------------------------------------- /lib/cafex/producer/worker.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Producer.Worker do 2 | use GenServer 3 | 4 | defmodule State do 5 | @moduledoc false 6 | defstruct broker: nil, 7 | topic: nil, 8 | partition: nil, 9 | client_id: nil, 10 | conn: nil, 11 | acks: 1, 12 | batch_num: nil, 13 | batches: [], 14 | # max_request_size: nil, 15 | linger_ms: 0, 16 | timer: nil, 17 | timeout: 60000, 18 | compression: nil 19 | end 20 | 21 | alias Cafex.Connection 22 | alias Cafex.Protocol.Produce.Request 23 | alias Cafex.Protocol.Produce.Response 24 | 25 | # =================================================================== 26 | # API 27 | # =================================================================== 28 | 29 | def start_link(broker, topic, partition, opts \\ []) do 30 | GenServer.start_link __MODULE__, [broker, topic, partition, opts] 31 | end 32 | 33 | def produce(pid, message) do 34 | GenServer.call pid, {:produce, message} 35 | end 36 | 37 | def async_produce(pid, message) do 38 | GenServer.cast pid, {:produce, message} 39 | end 40 | 41 | def stop(pid) do 42 | GenServer.call pid, :stop 43 | end 44 | 45 | # =================================================================== 46 | # GenServer callbacks 47 | # =================================================================== 48 | 49 | def init([{host, port} = broker, topic, partition, opts]) do 50 | acks = Keyword.get(opts, :acks, 1) 51 | timeout = Keyword.get(opts, :timeout, 60000) 52 | client_id = Keyword.get(opts, :client_id, "cafex") 53 | batch_num = Keyword.get(opts, :batch_num) 54 | # max_request_size = Keyword.get(opts, :max_request_size) 55 | linger_ms = Keyword.get(opts, :linger_ms) 56 | compression = Keyword.get(opts, :compression) 57 | 58 | state = %State{ broker: broker, 59 | topic: topic, 60 | partition: partition, 61 | client_id: client_id, 62 | acks: acks, 63 | batch_num: batch_num, 64 | # max_request_size: max_request_size, 65 | linger_ms: linger_ms, 66 | timeout: timeout, 67 | compression: compression} 68 | 69 | case Connection.start_link(host, port, client_id: client_id) do 70 | {:ok, pid} -> 71 | {:ok, %{state | conn: pid}} 72 | {:error, reason} -> 73 | {:stop, reason} 74 | end 75 | end 76 | 77 | def handle_call({:produce, message}, from, state) do 78 | maybe_produce(message, from, state) 79 | end 80 | def handle_call(:stop, _from, state) do 81 | {:stop, :normal, :ok, state} 82 | end 83 | 84 | def handle_cast({:produce, message}, state) do 85 | maybe_produce(message, nil, state) 86 | end 87 | 88 | def handle_info({:timeout, timer, :linger_timeout}, %{timer: timer, batches: batches} = state) do 89 | result = batches |> Enum.reverse |> do_produce(state) 90 | state = %{state|timer: nil, batches: []} 91 | case result do 92 | :ok -> 93 | {:noreply, state} 94 | {:error, reason} -> 95 | {:stop, reason, state} 96 | end 97 | end 98 | 99 | def terminate(reason, %{conn: conn, batches: batches}) do 100 | case batches do 101 | nil -> :ok 102 | [] -> :ok 103 | batches -> 104 | Enum.each(batches, fn {from, _} -> 105 | do_reply({from, {:error, reason}}) 106 | end) 107 | end 108 | 109 | if conn, do: Connection.close(conn) 110 | :ok 111 | end 112 | 113 | # =================================================================== 114 | # Internal functions 115 | # =================================================================== 116 | 117 | defp maybe_produce(message, from, %{linger_ms: linger_ms} = state) when is_integer(linger_ms) and linger_ms <= 0 do 118 | case do_produce([{from, message}], state) do 119 | :ok -> {:noreply, state} 120 | {:error, reason} -> {:stop, reason, state} 121 | end 122 | end 123 | defp maybe_produce(message, from, %{batches: batches, batch_num: batch_num} = state) when length(batches) + 1 >= batch_num do 124 | result = [{from, message}|batches] |> Enum.reverse |> do_produce(state) 125 | state = %{state|batches: []} 126 | case result do 127 | :ok -> {:noreply, state} 128 | {:error, reason} -> {:stop, reason, state} 129 | end 130 | end 131 | defp maybe_produce(message, from, %{linger_ms: linger_ms, batches: batches, timer: timer} = state) do 132 | timer = case timer do 133 | nil -> 134 | :erlang.start_timer(linger_ms, self, :linger_timeout) 135 | timer -> 136 | timer 137 | end 138 | {:noreply, %{state|batches: [{from, message}|batches], timer: timer}} 139 | end 140 | 141 | defp do_produce([], _state), do: :ok 142 | defp do_produce(message_pairs, state) do 143 | case do_request(message_pairs, state) do 144 | {:ok, replies} -> 145 | Enum.each(replies, &do_reply/1) 146 | :ok 147 | {:error, reason} -> 148 | # Enum.each(message_pairs, &(do_reply({elem(&1, 0), reason}))) 149 | Enum.each(message_pairs, fn {from, _message} -> 150 | do_reply({from, reason}) 151 | end) 152 | {:error, reason} 153 | end 154 | end 155 | 156 | defp do_request(message_pairs, %{topic: topic, 157 | partition: partition, 158 | acks: acks, 159 | timeout: timeout, 160 | compression: compression, 161 | conn: conn}) do 162 | 163 | messages = Enum.map(message_pairs, fn {_from, message} -> 164 | %{message | topic: topic, partition: partition} 165 | end) 166 | 167 | request = %Request{ required_acks: acks, 168 | timeout: timeout, 169 | compression: compression, 170 | messages: messages } 171 | 172 | case Connection.request(conn, request) do 173 | {:ok, %Response{topics: [{^topic, [%{error: :no_error, partition: ^partition}]}]}} -> 174 | replies = Enum.map(message_pairs, fn {from, _} -> 175 | {from, :ok} 176 | end) 177 | {:ok, replies} 178 | {:ok, %Response{topics: [{^topic, [%{error: reason}]}]}} -> 179 | replies = Enum.map(message_pairs, fn {from, _} -> 180 | {from, {:error, reason}} 181 | end) 182 | {:ok, replies} 183 | {:error, reason} -> 184 | {:error, reason} 185 | end 186 | end 187 | 188 | defp do_reply({nil, _reply}), do: :ok 189 | defp do_reply({from, reply}), do: GenServer.reply(from, reply) 190 | end 191 | -------------------------------------------------------------------------------- /lib/cafex/protocol.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol do 2 | @moduledoc """ 3 | This module provide encode/decode functions for common structures in Kafka protocol. 4 | 5 | And also provide conveniences for implementing API request and the `Cafex.Protocol.Request`. 6 | 7 | ## APIs 8 | 9 | * `Cafex.Protocol.Metadata` 10 | - `Cafex.Protocol.Metadata.Request` 11 | - `Cafex.Protocol.Metadata.Response` 12 | * `Cafex.Protocol.Produce` 13 | - `Cafex.Protocol.Produce.Request` 14 | - `Cafex.Protocol.Produce.Response` 15 | * `Cafex.Protocol.Fetch` 16 | - `Cafex.Protocol.Fetch.Request` 17 | - `Cafex.Protocol.Fetch.Response` 18 | * `Cafex.Protocol.Offset` 19 | - `Cafex.Protocol.Offset.Request` 20 | - `Cafex.Protocol.Offset.Response` 21 | * `Cafex.Protocol.ConsumerMetadata` 22 | - `Cafex.Protocol.ConsumerMetadata.Request` 23 | - `Cafex.Protocol.ConsumerMetadata.Response` 24 | * `Cafex.Protocol.OffsetCommit` 25 | - `Cafex.Protocol.OffsetCommit.Request` 26 | - `Cafex.Protocol.OffsetCommit.Response` 27 | * `Cafex.Protocol.OffsetFetch` 28 | - `Cafex.Protocol.OffsetFetch.Request` 29 | - `Cafex.Protocol.OffsetFetch.Response` 30 | """ 31 | 32 | @type api_version :: 0 | 1 | 2 33 | @type api_key :: 0..16 34 | @type error :: Cafex.Protocol.Errors.t 35 | 36 | @apis %{ 37 | :produce => 0, 38 | :fetch => 1, 39 | :offset => 2, 40 | :metadata => 3, 41 | :offset_commit => 8, 42 | :offset_fetch => 9, 43 | :group_coordinator => 10, 44 | :join_group => 11, 45 | :heartbeat => 12, 46 | :leave_group => 13, 47 | :sync_group => 14, 48 | :describe_groups => 15, 49 | :list_groups => 16, 50 | } 51 | 52 | alias Cafex.Protocol.Request 53 | 54 | for {key, value} <- @apis do 55 | def api_key(unquote(key)), do: unquote(value) 56 | end 57 | 58 | defmacro __using__(opts) do 59 | {opts, []} = Code.eval_quoted(opts, [], __CALLER__) 60 | api = Keyword.get opts, :api 61 | 62 | api_version = Keyword.get opts, :api_version, 0 63 | mod = __CALLER__.module 64 | 65 | if api == nil do 66 | raise CompileError, file: __CALLER__.file, line: __CALLER__.line, description: "To use #{inspect __MODULE__}, `api` must be set" 67 | end 68 | 69 | if ! Map.has_key?(@apis, api) do 70 | raise CompileError, file: __CALLER__.file, line: __CALLER__.line, description: "Unsupported api: #{api}" 71 | end 72 | 73 | Module.put_attribute mod, :api, api 74 | Module.put_attribute mod, :api_version, api_version 75 | 76 | quote do 77 | import unquote(__MODULE__), only: [defrequest: 0, defrequest: 1, defresponse: 1] 78 | import Cafex.Protocol.Codec 79 | @behaviour Cafex.Protocol.Codec 80 | @before_compile unquote(__MODULE__) 81 | 82 | end 83 | end 84 | 85 | defmacro __before_compile__(env) do 86 | # TODO check defrequest and defresponse 87 | mod = env.module 88 | request = Module.get_attribute mod, :request 89 | response = Module.get_attribute mod, :response 90 | 91 | quoted = [] 92 | 93 | # ListGroups Request is empty, generate an empty request 94 | quoted = quoted ++ if request != true do 95 | [quote do 96 | unquote(__MODULE__).defrequest 97 | def encode(_request), do: <<>> 98 | end] 99 | else 100 | [] 101 | end 102 | 103 | if response == nil do 104 | raise CompileError, file: __CALLER__.file, line: __CALLER__.line, description: "Use #{inspect __MODULE__} must call `defresponse`" 105 | end 106 | 107 | api = Module.get_attribute mod, :api 108 | api_version = Module.get_attribute mod, :api_version 109 | 110 | quoted ++ [quote do 111 | def has_response?(%__MODULE__.Request{}), do: true 112 | def decoder(%__MODULE__.Request{}), do: __MODULE__ 113 | def api_key(%__MODULE__.Request{}), do: unquote(__MODULE__).api_key(unquote(api)) 114 | def api_version(%__MODULE__.Request{}), do: unquote(api_version) 115 | 116 | defoverridable [has_response?: 1, api_version: 1, decoder: 1] 117 | end] 118 | end 119 | 120 | defmacro defrequest(opts \\ []) do 121 | block = Keyword.get(opts, :do) 122 | mod = __CALLER__.module 123 | 124 | impl_protocol = impl_request_protocol(mod) 125 | 126 | quote do 127 | defmodule Request do 128 | import unquote(__MODULE__), only: [field: 3, field: 2] 129 | 130 | Module.register_attribute(__MODULE__, :fields, accumulate: true) 131 | Module.register_attribute(__MODULE__, :struct_fields, accumulate: true) 132 | 133 | unquote(block) 134 | unquote(impl_protocol) 135 | 136 | Module.eval_quoted __ENV__, [ 137 | Cafex.Protocol.__struct__(@struct_fields), 138 | Cafex.Protocol.__typespec__(__MODULE__) 139 | ] 140 | end 141 | Module.put_attribute __MODULE__, :request, true 142 | end 143 | end 144 | 145 | defmacro defresponse(do: block) do 146 | quote do 147 | defmodule Response do 148 | Module.register_attribute(__MODULE__, :fields, accumulate: true) 149 | Module.register_attribute(__MODULE__, :struct_fields, accumulate: true) 150 | import unquote(__MODULE__), only: [field: 3, field: 2] 151 | 152 | unquote(block) 153 | 154 | Module.eval_quoted __ENV__, [ 155 | Cafex.Protocol.__struct__(@struct_fields), 156 | Cafex.Protocol.__typespec__(__MODULE__) 157 | ] 158 | end 159 | Module.put_attribute __MODULE__, :response, true 160 | end 161 | end 162 | 163 | defmacro field(name, opts \\ [], type) do 164 | type = Macro.escape(type) 165 | quote do 166 | Cafex.Protocol.__field__(__MODULE__, unquote(name), unquote(type), unquote(opts)) 167 | end 168 | end 169 | 170 | defdelegate encode_request(client_id, correlation_id, request), to: Cafex.Protocol.Codec 171 | defdelegate encode_string(data), to: Cafex.Protocol.Codec 172 | defdelegate has_response?(request), to: Cafex.Protocol.Request 173 | 174 | @doc false 175 | def __typespec__(mod) do 176 | types = Module.get_attribute(mod, :fields) 177 | 178 | {:%, [], [name, {:%{}, [], _}]} = quote do 179 | %unquote(mod){} 180 | end 181 | 182 | type_specs = {:%, [], [name, {:%{}, [], types}]} 183 | 184 | quote do 185 | @type t :: unquote(type_specs) 186 | end 187 | end 188 | 189 | @doc false 190 | def __field__(mod, name, type, opts) do 191 | default = Keyword.get(opts, :default) 192 | Module.put_attribute(mod, :fields, {name, type}) 193 | put_struct_field(mod, name, default) 194 | end 195 | 196 | @doc false 197 | def __struct__(struct_fields) do 198 | quote do 199 | defstruct unquote(Macro.escape(struct_fields)) 200 | end 201 | end 202 | 203 | defp put_struct_field(mod, name, assoc) do 204 | fields = Module.get_attribute(mod, :struct_fields) 205 | 206 | if List.keyfind(fields, name, 0) do 207 | raise ArgumentError, "field #{inspect name} is already set on #{inspect mod}" 208 | end 209 | 210 | Module.put_attribute(mod, :struct_fields, {name, assoc}) 211 | end 212 | 213 | defp impl_request_protocol(mod) do 214 | impls = [:api_key, :api_version, :has_response?, :encode, :decoder] 215 | |> Enum.map(fn func -> 216 | quote do 217 | def unquote(func)(req), do: unquote(mod).unquote(func)(req) 218 | end 219 | end) 220 | 221 | quote do 222 | defimpl Cafex.Protocol.Request do 223 | unquote(impls) 224 | end 225 | end 226 | end 227 | end 228 | -------------------------------------------------------------------------------- /lib/cafex/protocol/compression.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Compression do 2 | 3 | @spec compress(binary, :gzip | :snappy) :: binary 4 | def compress(data, :gzip) do 5 | :zlib.gzip(data) 6 | end 7 | def compress(data, :snappy) do 8 | {:ok, bin} = :snappy.compress(data) 9 | bin 10 | end 11 | 12 | @spec decompress(binary, :gzip | :snappy) :: binary 13 | def decompress(data, :gzip) do 14 | :zlib.gunzip(data) 15 | end 16 | def decompress(data, :snappy) do 17 | << _snappy_header :: 64, _snappy_version_info :: 64, rest :: binary>> = data 18 | snappy_decompress_chunk(rest, <<>>) 19 | end 20 | 21 | defp snappy_decompress_chunk(<<>>, data), do: data 22 | defp snappy_decompress_chunk(<< valsize :: 32-unsigned, 23 | value :: size(valsize)-binary, 24 | rest :: binary>>, data) do 25 | {:ok, decompressed_value} = :snappy.decompress(value) 26 | snappy_decompress_chunk(rest, data <> decompressed_value) 27 | end 28 | end 29 | -------------------------------------------------------------------------------- /lib/cafex/protocol/decoder.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Decoder do 2 | @moduledoc """ 3 | Kafka server response decoder implementation specification. 4 | """ 5 | 6 | @decoders [ Metadata, 7 | Produce, 8 | Fetch, 9 | Offset, 10 | GroupCoordinator, 11 | OffsetCommit, 12 | OffsetFetch, 13 | JoinGroup, 14 | SyncGroup, 15 | LeaveGroup, 16 | Heartbeat, 17 | ListGroups, 18 | DescribeGroups] 19 | 20 | @typedoc """ 21 | The `decode` function in each decoder will return there own response struct 22 | 23 | See `Cafex.Protocol` 24 | """ 25 | @type response :: unquote(Enum.map(@decoders, fn d -> 26 | quote do: Cafex.Protocol.unquote(d).Response.t 27 | end) |> List.foldr([], fn 28 | v, [] -> quote do: unquote(v) 29 | v, acc -> quote do: unquote(v) | unquote(acc) 30 | end)) 31 | 32 | @typedoc """ 33 | The modules which implement the `Decoder` interface 34 | 35 | See `Cafex.Protocol` 36 | """ 37 | @type decoder :: unquote(Enum.map(@decoders, fn d -> 38 | quote do: Cafex.Protocol.unquote(d) 39 | end) |> List.foldr([], fn 40 | v, [] -> quote do: unquote(v) 41 | v, acc -> quote do: unquote(v) | unquote(acc) 42 | end)) 43 | 44 | @doc """ 45 | Decode the response message in the Kafka server response 46 | """ 47 | @callback decode(binary) :: response 48 | 49 | end 50 | -------------------------------------------------------------------------------- /lib/cafex/protocol/describe_groups.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.DescribeGroups do 2 | use Cafex.Protocol, api: :describe_groups 3 | 4 | defrequest do 5 | field :groups, [group_id] 6 | @type group_id :: binary 7 | end 8 | 9 | defresponse do 10 | field :groups, [group] 11 | 12 | @type group :: %{error: Cafex.Protocol.error, 13 | group_id: binary, 14 | state: binary, 15 | protocol_type: binary, 16 | protocol: binary, 17 | members: [member]} 18 | @type member :: %{member_id: binary, 19 | client_id: binary, 20 | client_host: binary, 21 | member_metadata: Cafex.Protocol.JoinGroup.Request.protocol_metadata, 22 | member_assignment: Cafex.Protocol.SyncGroup.Request.member_assignment} 23 | end 24 | 25 | def encode(%{groups: groups}) do 26 | groups 27 | |> encode_array(&Cafex.Protocol.encode_string/1) 28 | |> IO.iodata_to_binary 29 | end 30 | 31 | def decode(data) when is_binary(data) do 32 | {groups, _} = decode_array(data, &parse_group/1) 33 | %Response{groups: groups} 34 | end 35 | 36 | defp parse_group(<< error_code :: 16-signed, 37 | group_id_len :: 16-signed, 38 | group_id :: size(group_id_len)-binary, 39 | state_len :: 16-signed, 40 | state :: size(state_len)-binary, 41 | protocol_type_len :: 16-signed, 42 | protocol_type :: size(protocol_type_len)-binary, 43 | protocol_len :: 16-signed, 44 | protocol :: size(protocol_len)-binary, 45 | rest :: binary >>) do 46 | {members, rest} = decode_array(rest, &parse_member/1) 47 | {%{error: decode_error(error_code), 48 | group_id: group_id, 49 | state: state, 50 | protocol_type: protocol_type, 51 | protocol: protocol, 52 | members: members}, rest} 53 | end 54 | 55 | defp parse_member(<< member_id_len :: 16-signed, 56 | member_id :: size(member_id_len)-binary, 57 | client_id_len :: 16-signed, 58 | client_id :: size(client_id_len)-binary, 59 | client_host_len :: 16-signed, 60 | client_host :: size(client_host_len)-binary, 61 | member_metadata_len :: 32-signed, 62 | member_metadata :: size(member_metadata_len)-binary, 63 | member_assignment_len :: 32-signed, 64 | member_assignment :: size(member_assignment_len)-binary, 65 | rest :: binary>>) do 66 | member_metadata = parse_group_protocol_metadata(member_metadata) 67 | member_assignment = parse_assignment(member_assignment) 68 | {%{member_id: member_id, 69 | client_id: client_id, 70 | client_host: client_host, 71 | member_metadata: member_metadata, 72 | member_assignment: member_assignment}, rest} 73 | end 74 | 75 | end 76 | -------------------------------------------------------------------------------- /lib/cafex/protocol/errors.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Errors do 2 | @moduledoc """ 3 | Use atom to represent the numeric error codes. 4 | 5 | For more details about errors, read [A Guide To The Kafka Protocol](https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-ErrorCodes) 6 | """ 7 | 8 | @error_map %{ 9 | 0 => :no_error, 10 | 1 => :offset_out_of_range, 11 | 2 => :invalid_message, 12 | 3 => :unknown_topic_or_partition, 13 | 4 => :invalid_message_size, 14 | 5 => :leader_not_available, 15 | 6 => :not_leader_for_partition, 16 | 7 => :request_timed_out, 17 | 8 => :broker_not_available, 18 | 9 => :replica_not_available, 19 | 10 => :message_size_too_large, 20 | 11 => :stale_controller_epoch, 21 | 12 => :offset_metadata_too_large, 22 | 14 => :group_load_in_progress, 23 | 15 => :group_coordinator_not_available, 24 | 16 => :not_coordinator_for_group, 25 | 17 => :invalid_topic, 26 | 18 => :record_list_too_large, 27 | 19 => :not_enough_replicas, 28 | 20 => :not_enough_replicas_after_append, 29 | 21 => :invalid_required_acks, 30 | 22 => :illegal_generation, 31 | 23 => :inconsistent_group_protocol, 32 | 24 => :invalid_group_id, 33 | 25 => :unknown_member_id, 34 | 26 => :invalid_session_timeout, 35 | 27 => :rebalance_in_progress, 36 | 28 => :invalid_commit_offset_size, 37 | 29 => :topic_authorization_failed, 38 | 30 => :group_authorization_failed, 39 | 31 => :cluster_authorization_failed 40 | } 41 | 42 | # Generate 43 | # @type t :: :no_error 44 | # | :offset_out_of_range 45 | # | ... 46 | 47 | @type t :: unquote(Map.values(@error_map) ++ [:unknown_error] 48 | |> List.foldr([], fn 49 | v, [] -> quote do: unquote(v) 50 | v, acc -> quote do: unquote(v) | unquote(acc) 51 | end)) 52 | 53 | @spec error(error_code :: integer) :: t 54 | def error(error_code) 55 | 56 | for {k, v} <- @error_map do 57 | def error(unquote(k)), do: unquote(v) 58 | end 59 | 60 | def error(_), do: :unknown_error 61 | end 62 | -------------------------------------------------------------------------------- /lib/cafex/protocol/fetch.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Fetch do 2 | use Cafex.Protocol, api: :fetch 3 | 4 | alias Cafex.Protocol.Message 5 | 6 | defrequest do 7 | field :replica_id, [default: -1], integer 8 | field :max_wait_time, [default: -1], integer 9 | field :min_bytes, [default: 0], integer 10 | field :topics, [default: []], [topic] 11 | 12 | @type topic :: {topic :: String.t, partitions :: [partition]} 13 | @type partition :: {partition :: integer, 14 | offset :: integer, 15 | max_bytes :: integer} 16 | end 17 | 18 | defresponse do 19 | field :topics, [topic] 20 | @type topic :: {topic :: String.t, partitions :: [partition]} 21 | @type partition :: %{partition: integer, 22 | error: Cafex.Protocol.error, 23 | hwm_offset: integer, 24 | messages: [Message.t]} 25 | end 26 | 27 | def encode(%{replica_id: replica_id, max_wait_time: max_wait_time, 28 | min_bytes: min_bytes, topics: topics}) do 29 | [<< replica_id :: 32-signed, max_wait_time :: 32-signed, min_bytes :: 32-signed >>, 30 | encode_array(topics, &encode_topic/1)] 31 | |> IO.iodata_to_binary 32 | end 33 | 34 | defp encode_topic({topic, partitions}) do 35 | [encode_string(topic), 36 | encode_array(partitions, &encode_partition/1)] 37 | end 38 | defp encode_partition({partition, offset, max_bytes}) do 39 | << partition :: 32-signed, offset :: 64-signed, max_bytes :: 32-signed >> 40 | end 41 | 42 | @spec decode(binary) :: Response.t 43 | def decode(data) when is_binary(data) do 44 | {topics, _} = decode_array(data, &decode_topic/1) 45 | %Response{topics: topics} 46 | end 47 | 48 | defp decode_topic(<< size :: 16-signed, topic :: size(size)-binary, rest :: binary >>) do 49 | {partitions, rest} = decode_array(rest, &decode_partition/1) 50 | {{topic, partitions}, rest} 51 | end 52 | 53 | defp decode_partition(<< partition :: 32-signed, 54 | error_code :: 16-signed, 55 | hwm_offset :: 64-signed, 56 | message_set_size :: 32-signed, 57 | message_set :: size(message_set_size)-binary, 58 | rest :: binary >>) do 59 | messages = decode_message_set(message_set) 60 | {%{partition: partition, 61 | error: decode_error(error_code), 62 | hwm_offset: hwm_offset, 63 | messages: messages}, rest} 64 | end 65 | end 66 | -------------------------------------------------------------------------------- /lib/cafex/protocol/group_coordinator.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.GroupCoordinator do 2 | use Cafex.Protocol, api: :group_coordinator 3 | 4 | defrequest do 5 | field :group_id, binary 6 | end 7 | 8 | defresponse do 9 | field :error, Cafex.Protocol.error 10 | field :coordinator_id, integer 11 | field :coordinator_host, binary 12 | field :coordinator_port, 0..65535 13 | end 14 | 15 | def encode(%{group_id: group_id}) do 16 | encode_string(group_id) 17 | end 18 | 19 | @spec decode(binary) :: Response.t 20 | def decode(<< error_code :: 16-signed, 21 | coordinator_id :: 32-signed, 22 | host_size :: 16-signed, 23 | coordinator_host :: size(host_size)-binary, 24 | coordinator_port :: 32-signed >>) do 25 | %Response{error: decode_error(error_code), 26 | coordinator_id: coordinator_id, 27 | coordinator_host: coordinator_host, 28 | coordinator_port: coordinator_port} 29 | end 30 | end 31 | -------------------------------------------------------------------------------- /lib/cafex/protocol/heartbeat.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Heartbeat do 2 | use Cafex.Protocol, api: :heartbeat 3 | 4 | defrequest do 5 | field :group_id, binary 6 | field :generation_id, binary 7 | field :member_id, binary 8 | end 9 | 10 | defresponse do 11 | field :error, Cafex.Protocol.error 12 | end 13 | 14 | def encode(%{group_id: group_id, 15 | generation_id: generation_id, 16 | member_id: member_id}) do 17 | [encode_string(group_id), 18 | << generation_id :: 32-signed >>, 19 | encode_string(member_id)] 20 | |> IO.iodata_to_binary 21 | end 22 | 23 | def decode(<< error_code :: 16-signed >>) do 24 | %Response{error: decode_error(error_code)} 25 | end 26 | end 27 | -------------------------------------------------------------------------------- /lib/cafex/protocol/join_group.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.JoinGroup do 2 | use Cafex.Protocol, api: :join_group 3 | 4 | defrequest do 5 | field :group_id, binary 6 | field :session_timeout, integer 7 | field :member_id, binary 8 | field :protocol_type, binary 9 | field :group_protocols, [group_protocol] 10 | 11 | @type group_protocol :: {name :: binary, protocol_metadata} 12 | @type protocol_metadata :: {version :: integer, subscription :: [topic], user_data :: binary} 13 | @type topic :: binary 14 | end 15 | 16 | defresponse do 17 | field :error, Cafex.Protocol.error 18 | field :generation_id, binary 19 | field :group_protocol, binary 20 | field :leader_id, binary 21 | field :member_id, binary 22 | field :members, [member] 23 | 24 | @type member :: {id :: binary, metadata :: Request.protocol_metadata} 25 | end 26 | 27 | def encode(%{group_id: group_id, 28 | session_timeout: timeout, 29 | member_id: member_id, 30 | protocol_type: protocol_type, 31 | group_protocols: protocols}) do 32 | [encode_string(group_id), 33 | << timeout :: 32-signed >>, 34 | encode_string(member_id), 35 | encode_string(protocol_type), 36 | encode_array(protocols, &encode_group_protocol/1)] 37 | |> IO.iodata_to_binary 38 | end 39 | 40 | def decode(<< error_code :: 16-signed, 41 | generation_id :: 32-signed, 42 | group_protocol_len :: 16-signed, 43 | group_protocol :: size(group_protocol_len)-binary, 44 | leader_id_len :: 16-signed, 45 | leader_id :: size(leader_id_len)-binary, 46 | member_id_len :: 16-signed, 47 | member_id :: size(member_id_len)-binary, 48 | rest :: binary >>) do 49 | {members, _} = decode_array(rest, &parse_member/1) 50 | %Response{error: decode_error(error_code), 51 | generation_id: generation_id, 52 | group_protocol: group_protocol, 53 | leader_id: leader_id, 54 | member_id: member_id, 55 | members: members} 56 | end 57 | 58 | defp encode_group_protocol({name, metadata}) do 59 | [encode_string(name), 60 | encode_group_protocol_metadata(metadata)] 61 | end 62 | 63 | defp parse_member(<< member_id_len :: 16-signed, 64 | member_id :: size(member_id_len)-binary, 65 | metadata_len :: 32-signed, 66 | metadata :: size(metadata_len)-binary, 67 | rest :: binary>>) do 68 | metadata = parse_group_protocol_metadata(metadata) 69 | {{member_id, metadata}, rest} 70 | end 71 | end 72 | -------------------------------------------------------------------------------- /lib/cafex/protocol/leave_group.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.LeaveGroup do 2 | use Cafex.Protocol, api: :leave_group 3 | 4 | defrequest do 5 | field :group_id, binary 6 | field :member_id, binary 7 | end 8 | 9 | defresponse do 10 | field :error, Cafex.Protocol.error 11 | end 12 | 13 | def encode(%{group_id: group_id, member_id: member_id}) do 14 | [encode_string(group_id), 15 | encode_string(member_id)] 16 | |> IO.iodata_to_binary 17 | end 18 | 19 | def decode(<< error_code :: 16-signed >>) do 20 | %Response{error: decode_error(error_code)} 21 | end 22 | end 23 | -------------------------------------------------------------------------------- /lib/cafex/protocol/list_groups.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.ListGroups do 2 | use Cafex.Protocol, api: :list_groups 3 | 4 | defresponse do 5 | field :groups, [group] 6 | field :error, binary 7 | 8 | @type group :: {group_id :: binary, 9 | protocol_type :: binary} 10 | end 11 | 12 | def decode(<< error_code :: 16-signed, rest :: binary >>) do 13 | {groups, _} = decode_array(rest, &parse_group/1) 14 | %Response{error: decode_error(error_code), groups: groups} 15 | end 16 | 17 | defp parse_group(<< group_id_len:: 16-signed, 18 | group_id :: size(group_id_len)-binary, 19 | protocol_type_len :: 16-signed, 20 | protocol_type :: size(protocol_type_len)-binary, 21 | rest :: binary>>) do 22 | {{group_id, protocol_type}, rest} 23 | end 24 | end 25 | -------------------------------------------------------------------------------- /lib/cafex/protocol/message.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Message do 2 | alias __MODULE__, as: Message 3 | defstruct topic: nil, 4 | partition: nil, 5 | value: nil, 6 | key: nil, 7 | offset: 0, 8 | magic_byte: 0, 9 | attributes: 0, 10 | timestamp_type: nil, 11 | compression: nil, 12 | metadata: nil, 13 | timestamp: 0 14 | 15 | @type t :: %Message{ topic: binary, 16 | partition: integer, 17 | value: binary, 18 | key: binary, 19 | offset: integer, 20 | magic_byte: integer, 21 | attributes: integer, 22 | compression: compression, 23 | metadata: term, 24 | timestamp: integer, 25 | timestamp_type: atom } 26 | 27 | @type compression :: nil | :gzip | :snappy 28 | 29 | @type tuple_message :: {topic :: String.t, partition :: integer, value :: binary} | 30 | {topic :: String.t, partition :: integer, value :: binary, key :: binary | nil} 31 | 32 | @spec from_tuple(tuple_message) :: Message.t 33 | def from_tuple({topic, partition, value}), do: from_tuple({topic, partition, value, nil}) 34 | def from_tuple({topic, partition, value, key}) do 35 | %Message{ topic: topic, 36 | partition: partition, 37 | value: value, 38 | key: key } 39 | end 40 | end 41 | -------------------------------------------------------------------------------- /lib/cafex/protocol/metadata.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Metadata do 2 | use Cafex.Protocol, api: :metadata 3 | 4 | defrequest do 5 | field :topics, [default: []], [binary] 6 | end 7 | 8 | defresponse do 9 | field :brokers, [broker] 10 | field :topics, [topic] 11 | 12 | @type broker :: %{node_id: integer, 13 | host: binary, 14 | port: 0..65535} 15 | @type topic :: %{error: Cafex.Protocol.error, 16 | name: binary, 17 | partitions: [partition]} 18 | @type partition :: %{error: Cafex.Protocol.error, 19 | partition_id: integer, 20 | leader: integer, 21 | replicas: [integer], 22 | isrs: [integer]} 23 | end 24 | 25 | def encode(%Request{topics: topics}) do 26 | topics 27 | |> encode_array(&Cafex.Protocol.encode_string/1) 28 | |> IO.iodata_to_binary 29 | end 30 | 31 | @spec decode(binary) :: Response.t 32 | def decode(data) when is_binary(data) do 33 | {brokers, rest} = decode_array(data, &parse_broker/1) 34 | {topics, _} = decode_array(rest, &parse_topic/1) 35 | %Response{brokers: brokers, topics: topics} 36 | end 37 | 38 | defp parse_broker(<< node_id :: 32-signed, host_len :: 16-signed, 39 | host :: size(host_len)-binary, port :: 32-signed, 40 | rest :: binary >>) do 41 | {%{node_id: node_id, host: host, port: port}, rest} 42 | end 43 | 44 | defp parse_topic(<< error_code :: 16-signed, topic_len :: 16-signed, 45 | topic :: size(topic_len)-binary, rest :: binary >>) do 46 | {partitions, rest} = decode_array(rest, &parse_partition/1) 47 | {%{error: decode_error(error_code), name: topic, partitions: partitions}, rest} 48 | end 49 | 50 | defp parse_partition(<< error_code :: 16-signed, partition_id :: 32-signed, 51 | leader :: 32-signed, rest :: binary >>) do 52 | {replicas, rest} = decode_array(rest, &parse_int32/1) 53 | {isrs, rest} = decode_array(rest, &parse_int32/1) 54 | {%{error: decode_error(error_code), 55 | partition_id: partition_id, 56 | leader: leader, 57 | replicas: replicas, 58 | isrs: isrs}, rest} 59 | end 60 | 61 | defp parse_int32(<< value :: 32-signed, rest :: binary >>), do: {value, rest} 62 | end 63 | -------------------------------------------------------------------------------- /lib/cafex/protocol/offset.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Offset do 2 | use Cafex.Protocol, api: :offset 3 | 4 | defrequest do 5 | field :replica_id, [default: -1], integer 6 | field :topics, [topic] 7 | 8 | @type topic :: {topic :: String.t, partitions :: [partition]} 9 | @type partition :: {partition :: integer, 10 | time :: integer, 11 | max_number_of_offsets :: integer} 12 | end 13 | 14 | defresponse do 15 | field :offsets, [topic] 16 | @type topic :: {topic :: String.t, partitions :: [partition]} 17 | @type partition :: {partition :: integer, 18 | error :: Cafex.Protocol.error, 19 | offsets :: [integer]} 20 | end 21 | 22 | def encode(%Request{replica_id: replica_id, topics: topics}) do 23 | [<< replica_id :: 32-signed >>, encode_array(topics, &encode_topic/1)] 24 | |> IO.iodata_to_binary 25 | end 26 | 27 | defp encode_topic({topic, partitions}) do 28 | [<< byte_size(topic) :: 16-signed, topic :: binary >>, 29 | encode_array(partitions, &encode_partition/1)] 30 | end 31 | 32 | defp encode_partition({partition, time, max_number_of_offsets}) do 33 | << partition :: 32-signed, parse_time(time) :: 64-signed, max_number_of_offsets :: 32-signed >> 34 | end 35 | 36 | @spec decode(binary) :: Response.t 37 | def decode(data) when is_binary(data) do 38 | {offsets, _rest} = decode_array(data, &parse_topic/1) 39 | %Response{offsets: offsets} 40 | end 41 | 42 | defp parse_topic(<< topic_len :: 16-signed, topic :: size(topic_len)-binary, rest :: binary >>) do 43 | {partitions, rest} = decode_array(rest, &parse_partition/1) 44 | {{topic, partitions}, rest} 45 | end 46 | 47 | defp parse_partition(<< partition :: 32-signed, error_code :: 16-signed, rest :: binary >>) do 48 | {offsets, rest} = decode_array(rest, &parse_offset/1) 49 | {%{partition: partition, error: decode_error(error_code), offsets: offsets}, rest} 50 | end 51 | 52 | defp parse_offset(<< offset :: 64-signed, rest :: binary >>), do: {offset, rest} 53 | 54 | defp parse_time(:latest), do: -1 55 | 56 | defp parse_time(:earliest), do: -2 57 | 58 | @spec parse_time(:calendar.datetime) :: integer 59 | defp parse_time(time) do 60 | current_time_in_seconds = time |> :calendar.datetime_to_gregorian_seconds 61 | unix_epoch_in_seconds = {{1970,1,1},{0,0,0}} |> :calendar.datetime_to_gregorian_seconds 62 | (current_time_in_seconds - unix_epoch_in_seconds) * 1000 63 | end 64 | end 65 | -------------------------------------------------------------------------------- /lib/cafex/protocol/offset_commit.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.OffsetCommit do 2 | @moduledoc """ 3 | This api saves out the consumer's position in the stream for one or more partitions. 4 | 5 | The offset commit request support version 0, 1 and 2. 6 | To read more details, visit the [A Guide to The Kafka Protocol](https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommitRequest). 7 | """ 8 | 9 | use Cafex.Protocol, api: :offset_commit 10 | 11 | @default_consumer_group_generation_id -1 12 | @default_timestamp -1 13 | 14 | defrequest do 15 | field :api_version, [default: 0], api_version 16 | field :consumer_group, [default: "cafex"], String.t 17 | field :consumer_group_generation_id, integer | nil 18 | field :consumer_id, String.t | nil 19 | field :retention_time, integer | nil 20 | field :topics, [topic] 21 | 22 | @type api_version :: 0 | 1 | 2 23 | @type topic :: {topic_name :: String.t, partitions :: [partition]} 24 | @type partition :: partition_v0 | partition_v1 | partition_v2 25 | @type partition_v0 :: {partition :: integer, offset :: integer, metadata:: binary} 26 | @type partition_v1 :: {partition :: integer, offset :: integer, timestamp :: integer, metadata :: binary} 27 | @type partition_v2 :: {partition :: integer, offset :: integer, metadata:: binary} 28 | end 29 | 30 | defresponse do 31 | field :topics, [topic] 32 | @type topic :: {topic_name :: String.t, partitions :: [partition]} 33 | @type partition :: {partition :: integer, error :: Cafex.Protocol.error} 34 | end 35 | 36 | def api_version(%Request{api_version: api_version}), do: api_version 37 | 38 | def encode(request) do 39 | request |> fill_default |> do_encode 40 | end 41 | 42 | defp do_encode(%{api_version: 0} = request), do: encode_0(request) 43 | defp do_encode(%{api_version: 1} = request), do: encode_1(request) 44 | defp do_encode(%{api_version: 2} = request), do: encode_2(request) 45 | 46 | defp fill_default(%{api_version: version, 47 | consumer_group_generation_id: id, 48 | consumer_id: consumer_id, 49 | retention_time: time, 50 | topics: topics} = request) do 51 | id = case id do 52 | nil -> @default_consumer_group_generation_id 53 | other -> other 54 | end 55 | 56 | time = case time do 57 | nil -> @default_timestamp 58 | other -> other 59 | end 60 | 61 | consumer_id = case consumer_id do 62 | nil -> "" 63 | other -> other 64 | end 65 | 66 | topics = case version do 67 | 1 -> 68 | Enum.map(topics, fn {topic_name, partitions} -> 69 | partitions = Enum.map(partitions, fn 70 | {p, o, m} -> {p, o, @default_timestamp, m} 71 | {_, _, _, _} = partition -> partition 72 | end) 73 | {topic_name, partitions} 74 | end) 75 | _ -> topics 76 | end 77 | %{request | consumer_group_generation_id: id, consumer_id: consumer_id, retention_time: time, topics: topics} 78 | end 79 | 80 | defp encode_0(%{consumer_group: consumer_group, topics: topics}) do 81 | [encode_string(consumer_group), 82 | encode_array(topics, &encode_topic_0/1)] 83 | |> IO.iodata_to_binary 84 | end 85 | 86 | defp encode_1(%{consumer_group: consumer_group, 87 | consumer_group_generation_id: consumer_group_generation_id, 88 | consumer_id: consumer_id, 89 | topics: topics}) do 90 | [encode_string(consumer_group), 91 | <>, 92 | encode_string(consumer_id), 93 | encode_array(topics, &encode_topic_1/1)] 94 | |> IO.iodata_to_binary 95 | end 96 | 97 | defp encode_2(%{consumer_group: consumer_group, 98 | consumer_group_generation_id: consumer_group_generation_id, 99 | consumer_id: consumer_id, 100 | retention_time: retention_time, 101 | topics: topics}) do 102 | [encode_string(consumer_group), 103 | <>, 104 | encode_string(consumer_id), 105 | <>, 106 | encode_array(topics, &encode_topic_2/1)] 107 | |> IO.iodata_to_binary 108 | end 109 | 110 | defp encode_topic_0(data), do: encode_topic(data, &encode_partition_0/1) 111 | defp encode_topic_1(data), do: encode_topic(data, &encode_partition_1/1) 112 | defp encode_topic_2(data), do: encode_topic(data, &encode_partition_2/1) 113 | 114 | defp encode_topic({topic, partitions}, func) do 115 | [encode_string(topic), 116 | encode_array(partitions, func)] 117 | end 118 | 119 | defp encode_partition_0({partition, offset, metadata}) do 120 | [<< partition :: 32-signed, offset :: 64 >>, encode_string(metadata)] 121 | end 122 | defp encode_partition_1({partition, offset, timestamp, metadata}) do 123 | [<< partition :: 32-signed, offset :: 64, timestamp :: 64 >>, encode_string(metadata)] 124 | end 125 | defp encode_partition_2(data), do: encode_partition_0(data) 126 | 127 | @spec decode(binary) :: Response.t 128 | def decode(data) when is_binary(data) do 129 | {topics, _} = decode_array(data, &decode_topic/1) 130 | %Response{topics: topics} 131 | end 132 | 133 | defp decode_topic(<< size :: 16-signed, topic :: size(size)-binary, rest :: binary >>) do 134 | {partitions, rest} = decode_array(rest, &decode_partition/1) 135 | {{topic, partitions}, rest} 136 | end 137 | 138 | defp decode_partition(<< partition :: 32-signed, error_code :: 16-signed, rest :: binary >>) do 139 | {{partition, decode_error(error_code)}, rest} 140 | end 141 | end 142 | -------------------------------------------------------------------------------- /lib/cafex/protocol/offset_fetch.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.OffsetFetch do 2 | @moduledoc """ 3 | This api reads back a consumer position previously written using the OffsetCommit api. 4 | 5 | The offset fetch request support version 0, 1 and 2. 6 | To read more details, visit the [A Guide to The Kafka Protocol](https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetFetchRequest). 7 | """ 8 | 9 | use Cafex.Protocol, api: :offset_fetch 10 | 11 | defrequest do 12 | field :api_version, [default: 0], api_version 13 | field :consumer_group, binary 14 | field :topics, [topic] 15 | @type api_version :: 0 | 1 16 | @type topic :: {topic_name :: String.t, partitions :: [partition]} 17 | @type partition :: integer 18 | end 19 | 20 | defresponse do 21 | field :topics, [topic] 22 | @type topic :: {partition :: integer, 23 | offset :: integer, 24 | metadata :: String.t, 25 | error :: Cafex.Protocol.error} 26 | end 27 | 28 | def api_version(%{api_version: api_version}), do: api_version 29 | 30 | def encode(%{consumer_group: consumer_group, topics: topics}) do 31 | [encode_string(consumer_group), 32 | encode_array(topics, &encode_topic/1)] 33 | |> IO.iodata_to_binary 34 | end 35 | 36 | defp encode_topic({topic, partitions}) do 37 | [encode_string(topic), 38 | encode_array(partitions, &encode_partition/1)] 39 | end 40 | defp encode_partition(partition), do: << partition :: 32-signed >> 41 | 42 | @spec decode(binary) :: Response.t 43 | def decode(data) when is_binary(data) do 44 | {topics, _} = decode_array(data, &decode_topic/1) 45 | %Response{topics: topics} 46 | end 47 | 48 | defp decode_topic(<< size :: 16-signed, topic :: size(size)-binary, rest :: binary >>) do 49 | {partitions, rest} = decode_array(rest, &decode_partition/1) 50 | {{topic, partitions}, rest} 51 | end 52 | 53 | defp decode_partition(<< partition :: 32-signed, offset :: 64-signed, 54 | -1 :: 16-signed, error_code :: 16-signed, rest :: binary >>) do 55 | {{partition, offset, "", decode_error(error_code)}, rest} 56 | end 57 | defp decode_partition(<< partition :: 32-signed, offset :: 64-signed, 58 | size :: 16-signed, metadata :: size(size)-binary, 59 | error_code :: 16-signed, rest :: binary >>) do 60 | {{partition, offset, metadata, decode_error(error_code)}, rest} 61 | end 62 | end 63 | -------------------------------------------------------------------------------- /lib/cafex/protocol/produce.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Produce do 2 | use Cafex.Protocol, api: :produce 3 | 4 | alias Cafex.Protocol.Message 5 | 6 | defrequest do 7 | field :required_acks, [default: 0], binary 8 | field :timeout, integer 9 | field :compression, [default: nil], Message.compression 10 | field :messages, [Message.t] 11 | end 12 | 13 | defresponse do 14 | field :topics, [topic] 15 | @type topic :: {topic :: String.t, [partition]} 16 | @type partition :: %{partition: integer, 17 | error: Cafex.Protocol.error, 18 | offset: integer} 19 | end 20 | def has_response?(%Request{required_acks: 0}), do: false 21 | def has_response?(%Request{required_acks: _}), do: true 22 | 23 | def encode(%Request{required_acks: required_acks, 24 | timeout: timeout, 25 | compression: compression_type, 26 | messages: messages}) do 27 | message_bytes = encode_messages(messages, compression_type) 28 | 29 | << required_acks :: 16-signed, timeout :: 32-signed, 30 | message_bytes :: binary >> 31 | end 32 | 33 | defp encode_messages(messages, compression_type) do 34 | messages 35 | |> group_by_topic 36 | |> encode_array(fn {topic, partitions} -> 37 | [encode_string(topic), 38 | encode_array(partitions, fn {partition, messages} -> 39 | bin = maybe_compress(messages, compression_type) |> encode_message_set 40 | << partition :: 32-signed, byte_size(bin) :: 32-signed, bin :: binary >> 41 | end)] 42 | end) 43 | |> IO.iodata_to_binary 44 | end 45 | 46 | defp maybe_compress(messages, nil), do: messages 47 | defp maybe_compress(messages, compression_type) do 48 | compressed = 49 | messages 50 | |> Enum.with_index 51 | |> Enum.map(fn {message, index} -> 52 | %Message{message | offset: index} 53 | end) 54 | |> encode_message_set 55 | |> Cafex.Protocol.Compression.compress(compression_type) 56 | 57 | [%Message{value: compressed, compression: compression_type}] 58 | end 59 | 60 | defp group_by_topic(messages) do 61 | messages |> Enum.group_by(fn %{topic: topic} -> topic end) 62 | |> Enum.map(fn {topic, msgs} -> {topic, group_by_partition(msgs)} end) 63 | end 64 | 65 | defp group_by_partition(messages) do 66 | messages |> Enum.group_by(fn %{partition: partition} -> partition end) 67 | |> Map.to_list 68 | end 69 | 70 | @spec decode(binary) :: Response.t 71 | def decode(data) when is_binary(data) do 72 | # TODO 73 | {response, _} = decode_array(data, &parse_response/1) 74 | %Response{topics: response} 75 | end 76 | 77 | defp parse_response(<< topic_size :: 16-signed, topic :: size(topic_size)-binary, rest :: binary >>) do 78 | {partitions, rest} = decode_array(rest, &parse_partition/1) 79 | {{topic, partitions}, rest} 80 | end 81 | 82 | defp parse_partition(<< partition :: 32-signed, error_code :: 16-signed, offset :: 64, rest :: binary >>) do 83 | {%{ partition: partition, 84 | error: decode_error(error_code), 85 | offset: offset}, rest} 86 | end 87 | end 88 | -------------------------------------------------------------------------------- /lib/cafex/protocol/request.ex: -------------------------------------------------------------------------------- 1 | defprotocol Cafex.Protocol.Request do 2 | @moduledoc """ 3 | The Cafex.Protocol.Request protocol used by `Cafex.Protocol` module. 4 | 5 | Read [A Guide To The Kafka Protocol](https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-Requests) for more details. 6 | """ 7 | 8 | @doc """ 9 | Return the API Key of the request type 10 | """ 11 | def api_key(req) 12 | 13 | @doc """ 14 | Return the per-api based version number of a API request 15 | """ 16 | def api_version(req) 17 | 18 | @doc """ 19 | Kafka server will reply to every request except produce request if the required_acks is 0 for now. 20 | 21 | `Cafex.Protocol` module will call this function on every request to check if server will reply or not. 22 | """ 23 | def has_response?(req) 24 | 25 | @doc """ 26 | Encode the request struct to binary 27 | """ 28 | def encode(req) 29 | 30 | @doc """ 31 | Return the decoder module of the response 32 | """ 33 | def decoder(req) 34 | end 35 | -------------------------------------------------------------------------------- /lib/cafex/protocol/sync_group.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.SyncGroup do 2 | use Cafex.Protocol, api: :sync_group 3 | 4 | defrequest do 5 | field :group_id, binary 6 | field :generation_id, integer 7 | field :member_id, binary 8 | field :group_assignment, [{member_id, member_assignment}] 9 | @type member_id :: binary 10 | @type member_assignment :: {version :: integer, [partition_assignment], user_data :: binary} 11 | @type partition_assignment :: {topic, [partition]} 12 | @type topic :: binary 13 | @type partition :: integer 14 | end 15 | 16 | defresponse do 17 | field :error, Cafex.Protocol.error 18 | field :member_assignment, Request.member_assignment 19 | end 20 | 21 | def encode(%{group_id: group_id, 22 | generation_id: generation_id, 23 | member_id: member_id, 24 | group_assignment: assignment}) do 25 | [encode_string(group_id), 26 | << generation_id :: 32-signed >>, 27 | encode_string(member_id), 28 | encode_array(assignment, &encode_member_assignment/1)] 29 | |> IO.iodata_to_binary 30 | end 31 | 32 | def decode(<< error_code :: 16-signed, 33 | assignment_size :: 32-signed, 34 | assignment :: size(assignment_size)-binary>>) do 35 | 36 | error = decode_error(error_code) 37 | assignment = case error do 38 | :no_error -> parse_assignment(assignment) 39 | _ -> nil 40 | end 41 | %Response{error: error, member_assignment: assignment} 42 | end 43 | 44 | def encode_member_assignment({member_id, assignment}) do 45 | [encode_string(member_id), encode_assignment(assignment)] 46 | end 47 | end 48 | -------------------------------------------------------------------------------- /lib/cafex/supervisor.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Supervisor do 2 | @moduledoc false 3 | 4 | use Supervisor 5 | 6 | def start_link do 7 | Supervisor.start_link __MODULE__, nil, [name: __MODULE__] 8 | end 9 | 10 | def start_producer(topic, opts \\ []) do 11 | :ok = ensure_started(Cafex.Producer.Supervisor, []) 12 | Cafex.Producer.Supervisor.start_producer(topic, opts) 13 | end 14 | 15 | defdelegate stop_producer(pid), to: Cafex.Producer.Supervisor 16 | 17 | def start_consumer(name, opts \\ []) do 18 | :ok = ensure_started(Cafex.Consumer.Supervisor, []) 19 | Cafex.Consumer.Supervisor.start_consumer(name, opts) 20 | end 21 | 22 | defdelegate stop_consumer(name), to: Cafex.Consumer.Supervisor 23 | 24 | def start_topic(name, brokers, opts \\ []) do 25 | :ok = ensure_started(Cafex.Topic.Supervisor, []) # Deprecated 26 | Cafex.Topic.Supervisor.start_topic(name, brokers, opts) 27 | end 28 | 29 | def init(_) do 30 | children = [ 31 | # supervisor(Cafex.Topic.Supervisor, []), # Deprecated 32 | # supervisor(Cafex.Producer.Supervisor, []), 33 | # supervisor(Cafex.Consumer.Supervisor, []) 34 | ] 35 | supervise children, strategy: :one_for_one, 36 | max_restarts: 10, 37 | max_seconds: 60 38 | end 39 | 40 | defp ensure_started(sup, args) do 41 | case Supervisor.start_child __MODULE__, supervisor(sup, args) do 42 | {:ok, _} -> :ok 43 | {:ok, _, _} -> :ok 44 | {:error, {:already_started, _}} -> :ok 45 | error -> error 46 | end 47 | end 48 | end 49 | -------------------------------------------------------------------------------- /lib/cafex/topic/supervisor.ex: -------------------------------------------------------------------------------- 1 | # Deprecated 2 | defmodule Cafex.Topic.Supervisor do 3 | @moduledoc false 4 | use Supervisor 5 | 6 | def start_link do 7 | Supervisor.start_link __MODULE__, [], name: __MODULE__ 8 | end 9 | 10 | def start_topic(name, brokers, opts \\ []) when is_binary(name) do 11 | Supervisor.start_child __MODULE__, [name, brokers, opts] 12 | end 13 | 14 | def init([]) do 15 | children = [ 16 | worker(Cafex.Topic.Server, [], restart: :temporary, 17 | shutdown: 2000) 18 | ] 19 | supervise children, strategy: :simple_one_for_one, 20 | max_restarts: 10, 21 | max_seconds: 60 22 | end 23 | end 24 | -------------------------------------------------------------------------------- /lib/cafex/util.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Util do 2 | 3 | def get_config(opts, fallback, key, default \\ nil) do 4 | case Keyword.get(opts, key) do 5 | nil -> Keyword.get(fallback, key, default) 6 | val -> val 7 | end 8 | end 9 | end 10 | -------------------------------------------------------------------------------- /lib/cafex/zk/leader.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.ZK.Leader do 2 | @moduledoc """ 3 | Leader Election implementation with ZooKeeper 4 | 5 | Visit [Zookeeper Recipes](http://zookeeper.apache.org/doc/trunk/recipes.html#sc_leaderElection) to read more. 6 | 7 | ## TODO 8 | 9 | Handle zookeeper event missed problem. 10 | """ 11 | 12 | @type zk :: pid 13 | @type path :: String.t 14 | @type seq :: String.t 15 | 16 | alias Cafex.ZK.Util 17 | 18 | @doc """ 19 | Leader election function. 20 | 21 | If you haven't got a sequence, call this function first, or else call 22 | `election/3` with your `seq` instead. 23 | 24 | It will return `{true, seq}` if the node is a leader. 25 | 26 | If it's not a leader, `{false, seq}` will return. And the current process 27 | will watch for leader changes. If the leader node was deleted, the process 28 | will receive a message `{:leader_election, seq}` and you must call 29 | `election/3` again to volunteer to be a leader. 30 | """ 31 | @spec election(zk, path) :: {true, seq} | {false, seq} 32 | def election(zk, path) do 33 | {:ok, seq} = create_node(zk, path) 34 | election(zk, path, seq) 35 | end 36 | 37 | @doc """ 38 | Leader election function. 39 | 40 | See `election/2`. 41 | """ 42 | @spec election(zk, path, seq) :: {true, seq} | {false, seq} 43 | def election(zk, path, seq) do 44 | case get_children(zk, path) do 45 | [^seq|_] -> 46 | {true, seq} 47 | children -> 48 | # find next lowest sequence 49 | case Enum.find_index(children, fn x -> x == seq end) do 50 | nil -> 51 | election(zk, path) # should not happen 52 | idx -> 53 | watch(zk, path, seq, Enum.at(children, idx - 1)) 54 | end 55 | end 56 | end 57 | 58 | defp create_node(zk, path) do 59 | case :erlzk.create(zk, path <> "/n_", :ephemeral_sequential) do 60 | {:ok, seq} -> 61 | {:ok, List.to_string(seq)} 62 | {:error, :no_node} -> 63 | :ok = Util.create_node(zk, path) 64 | create_node(zk, path) 65 | end 66 | end 67 | 68 | defp get_children(zk, path) do 69 | {:ok, children} = :erlzk.get_children(zk, path) 70 | children |> Enum.map(fn x -> path <> "/" <> List.to_string(x) end) 71 | |> Enum.sort 72 | end 73 | 74 | defp watch(zk, path, seq, lower) do 75 | case :erlzk.exists(zk, lower, watcher(lower, seq)) do 76 | {:ok, _} -> 77 | {false, seq} 78 | {:error, :no_node} -> 79 | election(zk, path, seq) 80 | end 81 | end 82 | 83 | defp watcher(path, seq) do 84 | parent = self 85 | spawn_link fn -> 86 | receive do 87 | {:node_deleted, ^path} -> 88 | send parent, {:leader_election, seq} 89 | end 90 | end 91 | end 92 | end 93 | -------------------------------------------------------------------------------- /lib/cafex/zk/lock.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.ZK.Lock do 2 | @moduledoc """ 3 | Distributed lock based on ZooKeeper 4 | 5 | Visit [Zookeeper Recipes](http://zookeeper.apache.org/doc/trunk/recipes.html#sc_recipes_Locks) to read more. 6 | """ 7 | 8 | @type zk :: pid 9 | @type path :: String.t 10 | @type seq :: String.t 11 | @type lock :: seq 12 | 13 | alias Cafex.ZK.Util 14 | 15 | @spec acquire(zk, path, timeout) :: {:ok, lock} | {:wait, lock} | {:error, term} 16 | def acquire(zk, path, timeout \\ 0) do 17 | {:ok, seq} = create_node(zk, path) 18 | reacquire(zk, path, seq, timeout) 19 | end 20 | 21 | @spec reacquire(zk, path, seq, timeout) :: {:ok, lock} | {:wait, lock} | {:error, term} 22 | def reacquire(zk, path, seq, timeout \\ 0) do 23 | case check_sequence(zk, path, seq, timeout) do 24 | {:ok, lock} -> 25 | {:ok, lock} 26 | {:wait, lock} -> 27 | {:wait, lock} 28 | {:error, reason} -> 29 | :erlzk.delete(zk, seq) 30 | {:error, reason} 31 | end 32 | end 33 | 34 | @spec release(zk, lock) :: :ok | {:error, term} 35 | def release(zk, lock) do 36 | :erlzk.delete(zk, lock) 37 | end 38 | 39 | defp create_node(zk, path) do 40 | case :erlzk.create(zk, path <> "/lock-", :ephemeral_sequential) do 41 | {:ok, seq} -> 42 | {:ok, List.to_string(seq)} 43 | {:error, :no_node} -> 44 | :ok = Util.create_node(zk, path) 45 | create_node(zk, path) 46 | end 47 | end 48 | 49 | defp get_children(zk, path) do 50 | {:ok, children} = :erlzk.get_children(zk, path) 51 | children |> Enum.map(fn x -> path <> "/" <> List.to_string(x) end) 52 | |> Enum.sort 53 | end 54 | 55 | defp check_sequence(zk, path, seq, 0) do 56 | case get_children(zk, path) do 57 | [^seq|_] -> {:ok, seq} 58 | _x -> {:error, :locked} 59 | end 60 | end 61 | defp check_sequence(zk, path, seq, timeout) do 62 | case get_children(zk, path) do 63 | [^seq|_] -> 64 | {:ok, seq} 65 | children -> 66 | # find next lowest sequence 67 | case Enum.find_index(children, fn x -> x == seq end) do 68 | nil -> 69 | acquire(zk, path) # should not happen 70 | idx -> 71 | check_exists(zk, path, seq, Enum.at(children, idx - 1), timeout) 72 | end 73 | end 74 | end 75 | 76 | defp check_exists(zk, path, seq, lower, :infinity) do 77 | case :erlzk.exists(zk, lower, watcher(lower, seq)) do 78 | {:ok, _} -> 79 | {:wait, seq} 80 | {:error, :no_node} -> 81 | check_sequence(zk, path, seq, :infinity) 82 | end 83 | end 84 | defp check_exists(zk, path, seq, lower, timeout) when is_integer(timeout) do 85 | case :erlzk.exists(zk, lower, watcher(lower, timeout)) do 86 | {:ok, _} -> 87 | start = :os.timestamp 88 | receive do 89 | :check_again -> 90 | timeout = div(:timer.now_diff(:os.timestamp, start), 1000) 91 | check_sequence(zk, path, seq, timeout) 92 | :timeout -> 93 | {:error, :timeout} 94 | after 95 | timeout -> 96 | {:error, :timeout} 97 | end 98 | {:error, :no_node} -> 99 | check_sequence(zk, path, seq, timeout) 100 | end 101 | end 102 | 103 | defp watcher(path, seq) when is_binary(seq) do 104 | parent = self 105 | spawn_link fn -> 106 | receive do 107 | {:node_deleted, ^path} -> 108 | send parent, {:lock_again, seq} 109 | end 110 | end 111 | end 112 | defp watcher(path, timeout) when is_integer(timeout) do 113 | parent = self 114 | spawn_link fn -> 115 | receive do 116 | {:node_deleted, ^path} -> 117 | send parent, :check_again 118 | after 119 | timeout -> 120 | send parent, :timeout 121 | end 122 | end 123 | end 124 | end 125 | -------------------------------------------------------------------------------- /lib/cafex/zk/util.ex: -------------------------------------------------------------------------------- 1 | defmodule Cafex.ZK.Util do 2 | @moduledoc """ 3 | ZooKeeper Utilities 4 | """ 5 | 6 | @type zk :: pid 7 | @type watcher :: pid 8 | @type path :: String.t 9 | 10 | @spec create_nodes(zk, [path]) :: :ok | {:error, term} 11 | def create_nodes(_zk, []), do: :ok 12 | def create_nodes( zk, [node|rest]) do 13 | case create_node(zk, node) do 14 | :ok -> create_nodes(zk, rest) 15 | error -> error 16 | end 17 | end 18 | 19 | @spec create_node(zk, path) :: :ok | {:error, term} 20 | def create_node(zk, "/") do 21 | # if use chroot, the "/" node may be not exists 22 | case :erlzk.exists(zk, "/") do 23 | {:ok, _state} -> :ok 24 | {:error, :no_node} -> 25 | case :erlzk.create(zk, "/") do 26 | {:ok, _state} -> :ok 27 | {:error, :node_exists} -> :ok 28 | error -> error 29 | end 30 | end 31 | end 32 | def create_node( zk, path) do 33 | path = String.rstrip(path, ?/) 34 | case :erlzk.exists(zk, path) do 35 | {:ok, _stat} -> 36 | :ok 37 | {:error, :no_node} -> 38 | case create_node(zk, Path.dirname(path)) do 39 | :ok -> 40 | case :erlzk.create(zk, path) do 41 | {:ok, _} -> :ok 42 | {:error, :node_exists} -> :ok 43 | error -> error 44 | end 45 | error -> 46 | error 47 | end 48 | end 49 | end 50 | 51 | @spec get_children(zk, path, watcher | nil) :: {:ok, [child_node::String.t]} | {:error, term} 52 | def get_children(zk, path, watcher \\ nil) do 53 | case do_get_children(zk, path, watcher) do 54 | {:ok, children} -> {:ok, Enum.map(children, &(List.to_string &1))} 55 | {:error, reason} -> {:error, reason} 56 | end 57 | end 58 | 59 | @spec get_children_with_data(zk, path, watcher | nil) :: {:ok, [{child_node :: String.t, data :: binary}]} | {:error, term} 60 | def get_children_with_data(zk, path, watcher \\ nil) do 61 | case do_get_children(zk, path, watcher) do 62 | {:ok, children} -> 63 | {:ok, children |> Enum.map(fn x -> 64 | x = List.to_string(x) 65 | case :erlzk.get_data(zk, Path.join(path, x)) do 66 | {:ok, {data, _stat}} -> 67 | {x, data} 68 | {:error, :no_node} -> 69 | {x, nil} 70 | end 71 | end) 72 | |> Enum.into(%{})} 73 | {:error, reason} -> 74 | {:error, reason} 75 | end 76 | end 77 | 78 | @spec get_data(zk, path, watcher | nil) :: {:ok, term} | {:error, term} 79 | def get_data(zk, path, watcher \\ nil) do 80 | case do_exists(zk, path, watcher) do 81 | {:ok, _stat} -> do_get_data(zk, path, watcher) 82 | error -> error 83 | end 84 | end 85 | 86 | defp do_get_data(zk, path, nil), do: :erlzk.get_data(zk, path) 87 | defp do_get_data(zk, path, watcher), do: :erlzk.get_data(zk, path, watcher) 88 | 89 | defp do_exists(zk, path, nil), do: :erlzk.exists(zk, path) 90 | defp do_exists(zk, path, watcher), do: :erlzk.exists(zk, path, watcher) 91 | 92 | defp do_get_children(zk, path, nil), do: :erlzk.get_children(zk, path) 93 | defp do_get_children(zk, path, watcher), do: :erlzk.get_children(zk, path, watcher) 94 | end 95 | -------------------------------------------------------------------------------- /lib/mix/cafex.ex: -------------------------------------------------------------------------------- 1 | defmodule Mix.Cafex do 2 | @moduledoc false 3 | 4 | alias Cafex.Connection 5 | alias Cafex.Protocol.GroupCoordinator 6 | 7 | def get_opts(opts, key, required \\ false, default \\ nil) do 8 | case Keyword.get(opts, key) do 9 | nil -> 10 | case required do 11 | true -> 12 | raise "Require --#{key}" 13 | false -> default 14 | end 15 | value -> value 16 | end 17 | end 18 | 19 | def info_msg(msg), do: output(:info, msg) 20 | def log_msg(msg), do: output(:info, msg) 21 | def success_msg(msg), do: output(:success, msg) 22 | def debug_msg(msg), do: output(:debug, msg) 23 | def warn_msg(msg), do: output(:warn, msg) 24 | def error_msg(msg), do: output(:error, msg) 25 | 26 | def parse_servers_url(url) do 27 | {servers, path} = case String.split(url, "/", parts: 2) do 28 | [servers] -> {servers, "/"} 29 | [servers, path] -> {servers, Enum.join(["/", path])} 30 | end 31 | servers = String.split(servers, ",") |> Enum.map(fn server -> 32 | [host, port] = String.split(server, ":", parts: 2) 33 | {host, String.to_integer(port)} 34 | end) 35 | 36 | %{servers: servers, path: path} 37 | end 38 | 39 | def ensure_started do 40 | {:ok, _} = Application.ensure_all_started(:cafex) 41 | end 42 | 43 | def get_coordinator(group, host, port) do 44 | request = %GroupCoordinator.Request{group_id: group} 45 | {:ok, conn} = Connection.start(host, port) 46 | {:ok, %{coordinator_host: host, coordinator_port: port}} = Connection.request(conn, request) 47 | :ok = Connection.close(conn) 48 | {host, port} 49 | end 50 | 51 | # =================================================================== 52 | # Internal functions 53 | # =================================================================== 54 | 55 | defp output(level, msg) when is_binary(msg) do 56 | Mix.shell.info [level_to_color(level), msg] 57 | end 58 | defp output(level, msg), do: output(level, "#{inspect msg}") 59 | 60 | defp level_to_color(:info), do: :normal 61 | defp level_to_color(:success), do: :green 62 | defp level_to_color(:debug), do: :cyan 63 | defp level_to_color(:warn), do: :yellow 64 | defp level_to_color(:error), do: :red 65 | end 66 | -------------------------------------------------------------------------------- /lib/mix/tasks/cafex.groups.ex: -------------------------------------------------------------------------------- 1 | defmodule Mix.Tasks.Cafex.Groups do 2 | use Mix.Task 3 | import Mix.Cafex 4 | import Cafex.Consumer.Util 5 | 6 | alias Cafex.Connection 7 | alias Cafex.Kafka.Metadata 8 | alias Cafex.Protocol.DescribeGroups 9 | alias Cafex.Protocol.ListGroups 10 | 11 | @shortdoc "List/Describe kafka consumer groups" 12 | @recursive true 13 | @moduledoc """ 14 | List/Describe kafka consumer groups 15 | 16 | ## Examples 17 | 18 | mix cafex.groups -b localhost:9091 19 | 20 | ## Command line options 21 | 22 | * `-b`, `--broker` - The Kafka broker in the form: `host1:port1` 23 | """ 24 | @doc false 25 | def run(args) do 26 | Mix.Task.run "compile" 27 | Logger.configure level: :warn 28 | 29 | {cli_opts, groups, _} = OptionParser.parse(args, aliases: [b: :broker]) 30 | broker = Keyword.get(cli_opts, :broker) 31 | 32 | unless broker do 33 | Mix.raise "Missed required arguments. Run `mix help #{Mix.Task.task_name(__MODULE__)}` for help" 34 | end 35 | 36 | %{servers: brokers} = parse_servers_url(broker) 37 | ensure_started 38 | 39 | {:ok, %{brokers: [%{host: host, port: port}|_] = brokers}} = Metadata.request(brokers, nil) 40 | 41 | if length(groups) > 0 do 42 | Enum.each(groups, fn group -> 43 | describe_group(group, host, port) 44 | end) 45 | else 46 | Enum.each(brokers, &list_groups/1) 47 | end 48 | 49 | :ok 50 | end 51 | 52 | defp describe_group(group, host, port) do 53 | {host, port} = get_coordinator(group, host, port) 54 | {:ok, conn} = Connection.start(host, port) 55 | request = %DescribeGroups.Request{groups: [group]} 56 | {:ok, %{groups: [%{error: error, 57 | group_id: ^group, 58 | state: state, 59 | protocol_type: protocol_type, 60 | protocol: protocol, 61 | members: members}]}} = Connection.request(conn, request) 62 | :ok = Connection.close(conn) 63 | 64 | info_msg "Group: #{group}" 65 | info_msg "Error: #{inspect error}" 66 | info_msg "State: #{state}" 67 | info_msg "Protocol Type: #{protocol_type}" 68 | info_msg "Protocol: #{protocol}" 69 | info_msg "Members:" 70 | Enum.each(members, fn %{member_id: member_id, 71 | client_id: client_id, 72 | client_host: client_host, 73 | member_metadata: {metadata_version, subscriptions, metadata_user_data}, 74 | member_assignment: {assignment_version, partitions, assignment_user_data}} -> 75 | info_msg "======================================================" 76 | info_msg " MemberId: #{member_id}" 77 | info_msg " ClientId: #{client_id}" 78 | info_msg " ClientHost: #{client_host}" 79 | info_msg " Member Metadata:" 80 | info_msg " Version: #{metadata_version}" 81 | info_msg " Subscriptions: #{inspect subscriptions}" 82 | info_msg " UserData:\n#{inspect decode_partitions(metadata_user_data), pretty: true, limit: 5000, char_lists: :as_lists}" 83 | info_msg " Member Assignment:" 84 | info_msg " Version: #{assignment_version}" 85 | info_msg " Partitions:\n#{inspect partitions, pretty: true, limit: 5000, char_lists: :as_lists}" 86 | info_msg " UserData: #{assignment_user_data}" 87 | info_msg "" 88 | end) 89 | end 90 | 91 | defp list_groups(%{node_id: id, host: host, port: port}) do 92 | {:ok, conn} = Connection.start(host, port) 93 | request = %ListGroups.Request{} 94 | {:ok, response} = Connection.request(conn, request) 95 | :ok = Connection.close(conn) 96 | 97 | info_msg "Broker: #{id} [#{host}:#{port}]" 98 | case response do 99 | %{error: :no_error, groups: groups} -> 100 | display groups 101 | %{error: error} -> 102 | error_msg "Error: #{inspect error}" 103 | end 104 | info_msg "" 105 | end 106 | 107 | defp display([]) do 108 | warn_msg "No groups in this broker" 109 | end 110 | defp display(groups) do 111 | Enum.each(groups, fn group -> 112 | info_msg group 113 | end) 114 | end 115 | end 116 | -------------------------------------------------------------------------------- /lib/mix/tasks/cafex.offset_commit.ex: -------------------------------------------------------------------------------- 1 | defmodule Mix.Tasks.Cafex.OffsetCommit do 2 | use Mix.Task 3 | import Mix.Cafex 4 | import Cafex.Consumer.Util 5 | 6 | alias Cafex.Connection 7 | alias Cafex.Protocol.OffsetCommit 8 | alias Cafex.Protocol.JoinGroup 9 | alias Cafex.Protocol.SyncGroup 10 | 11 | @shortdoc "Commit consumer group offset" 12 | @recursive true 13 | @moduledoc """ 14 | Commit consumer group offsets. 15 | You should really carful in using this task. 16 | 17 | ## Examples 18 | 19 | mix cafex.offset_commit -t topic -g group -b localhost:9092 -o offset -p partition 20 | 21 | ## Command line options 22 | 23 | * `-t`, `--topic` - Topic name 24 | * `-g`, `--group` - Consumer group name 25 | * `-b`, `--broker` - The Kafka broker in the form: `host1:port1` 26 | * `-o`, `--offset` - Offset to be committed 27 | * `-p`, `--partition` - Partition associated with the offset 28 | """ 29 | 30 | 31 | @doc false 32 | def run(args) do 33 | Mix.Task.run "compile" 34 | Logger.configure level: :warn 35 | 36 | {cli_opts, _, _} = OptionParser.parse args, 37 | switches: [partition: :integer, offset: :integer], 38 | aliases: [t: :topic, g: :group, b: :broker, o: :offset, p: :partition] 39 | topic = Keyword.get(cli_opts, :topic) 40 | group = Keyword.get(cli_opts, :group) 41 | broker = Keyword.get(cli_opts, :broker) 42 | offset = Keyword.get(cli_opts, :offset) 43 | partition = Keyword.get(cli_opts, :partition) 44 | 45 | unless topic && group && broker && offset && partition do 46 | Mix.raise "Missed required arguments. Run `mix help #{Mix.Task.task_name(__MODULE__)}` for help" 47 | end 48 | 49 | %{servers: brokers} = parse_servers_url(broker) 50 | ensure_started 51 | 52 | {:ok, %{brokers: [%{host: host, port: port}|_]}} = Cafex.Kafka.Metadata.request(brokers, topic) 53 | 54 | {host, port} = get_coordinator(group, host, port) 55 | 56 | {:ok, conn} = Connection.start(host, port) 57 | 58 | request = %JoinGroup.Request{group_id: group, 59 | member_id: "", 60 | session_timeout: 6000, 61 | protocol_type: "consumer", 62 | group_protocols: [{"cafex", {0, [topic], ""}}]} 63 | {:ok, %{error: :no_error, 64 | generation_id: generation_id, 65 | member_id: member_id, 66 | members: members}} = Connection.request(conn, request) 67 | if length(members) <= 0 do 68 | Mix.raise "Not the leader, can't commit offset!" 69 | end 70 | 71 | members = Enum.map(members, fn {member_id, {0, [^topic], user_data}} -> 72 | {member_id, decode_partitions(user_data)} 73 | end) 74 | info_msg "Assigned members are: #{inspect members}" 75 | 76 | member_assignment = {0, [{topic, [partition]}], nil} 77 | request = %SyncGroup.Request{group_id: group, 78 | member_id: member_id, 79 | generation_id: generation_id, 80 | group_assignment: [{member_id, member_assignment}]} 81 | 82 | {:ok, %{error: :no_error}} = Connection.request(conn, request) 83 | 84 | request = %OffsetCommit.Request{api_version: 1, 85 | consumer_group: group, 86 | consumer_id: member_id, 87 | consumer_group_generation_id: generation_id, 88 | topics: [{topic, [{partition, offset, ""}]}]} 89 | {:ok, %{topics: [{^topic, partition_errors}]}} = Connection.request(conn, request) 90 | case Enum.find(partition_errors, fn {_, e} -> e != :no_error end) do 91 | nil -> success_msg "OffsetCommit topic '#{topic}' partition: #{partition} with offset: #{offset} success." 92 | {p, e} -> error_msg "OffsetCommit topic '#{topic}' partition: #{p} with offset: #{offset} error: #{inspect e}" 93 | end 94 | 95 | :ok = Connection.close(conn) 96 | end 97 | end 98 | -------------------------------------------------------------------------------- /lib/mix/tasks/cafex.offset_fetch.ex: -------------------------------------------------------------------------------- 1 | defmodule Mix.Tasks.Cafex.OffsetFetch do 2 | use Mix.Task 3 | import Mix.Cafex 4 | 5 | alias Cafex.Protocol.Offset 6 | alias Cafex.Protocol.OffsetFetch 7 | 8 | @shortdoc "Fetch consumer group offsets" 9 | @recursive true 10 | @moduledoc """ 11 | Fetch consumer group offsets 12 | 13 | ## Examples 14 | 15 | mix cafex.offset_fetch -t topic -g group -b localhost:9092 16 | 17 | ## Command line options 18 | 19 | * `-t`, `--topic` - Topic name 20 | * `-g`, `--group` - Consumer group name 21 | * `-b`, `--broker` - The Kafka broker in the form: `host1:port1` 22 | """ 23 | 24 | alias Cafex.Connection 25 | 26 | @doc false 27 | def run(args) do 28 | Mix.Task.run "compile" 29 | Logger.configure level: :warn 30 | 31 | {cli_opts, _, _} = OptionParser.parse(args, aliases: [t: :topic, g: :group, b: :broker]) 32 | topic = Keyword.get(cli_opts, :topic) 33 | group = Keyword.get(cli_opts, :group) 34 | broker = Keyword.get(cli_opts, :broker) 35 | 36 | unless topic && group && broker do 37 | Mix.raise "Missed required arguments. Run `mix help #{Mix.Task.task_name(__MODULE__)}` for help" 38 | end 39 | 40 | %{servers: brokers} = parse_servers_url(broker) 41 | ensure_started 42 | 43 | {:ok, %{brokers: [%{host: host, port: port}|_]=brokers, 44 | topics: [%{partitions: partitions}]}} = Cafex.Kafka.Metadata.request(brokers, topic) 45 | partition_ids = Enum.map(partitions, fn %{partition_id: id} -> id end) 46 | brokers_map = Enum.map(brokers, fn %{node_id: node_id} = broker -> 47 | {node_id, broker} 48 | end) |> Enum.into(%{}) 49 | 50 | hwm = partitions 51 | |> Enum.group_by(fn %{leader: leader} -> leader end) 52 | |> Enum.map(fn {k, v} -> 53 | partitions = Enum.map(v, fn %{partition_id: partition} -> 54 | {partition, :latest, 1} 55 | end) 56 | request = %Offset.Request{topics: [{topic, partitions}]} 57 | %{host: host, port: port} = brokers_map[k] 58 | {:ok, conn} = Connection.start(host, port) 59 | {:ok, %{offsets: [{^topic, partitions}]}} = Connection.request(conn, request) 60 | Connection.close(conn) 61 | partitions 62 | end) 63 | |> List.flatten 64 | |> Enum.map(fn %{partition: partition, offsets: [offset]} -> 65 | {partition, offset} 66 | end) 67 | |> Enum.into(%{}) 68 | 69 | {host, port} = get_coordinator(group, host, port) 70 | {:ok, conn} = Connection.start(host, port) 71 | request = %OffsetFetch.Request{api_version: 1, consumer_group: group, topics: [{topic, partition_ids}]} 72 | {:ok, %{topics: [{^topic, partitions}]}} = Connection.request(conn, request) 73 | :ok = Connection.close(conn) 74 | 75 | success_msg "Topic: #{topic}\n" 76 | partitions 77 | |> Enum.sort 78 | |> Enum.each(fn {id, offset, meta, error} -> 79 | info_msg "partition: #{id}\t offset: #{offset}\thwmOffset: #{hwm[id]}\t " <> 80 | "lag: #{hwm[id] - offset}\t meta: #{inspect meta}\t error: #{inspect error}" 81 | end) 82 | 83 | :ok 84 | end 85 | end 86 | -------------------------------------------------------------------------------- /lib/mix/tasks/cafex.offset_reset.ex: -------------------------------------------------------------------------------- 1 | defmodule Mix.Tasks.Cafex.OffsetReset do 2 | use Mix.Task 3 | import Mix.Cafex 4 | import Cafex.Consumer.Util 5 | 6 | alias Cafex.Connection 7 | alias Cafex.Protocol.Offset 8 | alias Cafex.Protocol.OffsetCommit 9 | alias Cafex.Protocol.JoinGroup 10 | alias Cafex.Protocol.SyncGroup 11 | alias Cafex.Consumer.LoadBalancer 12 | 13 | @shortdoc "Reset consumer group's offsets to latest/earliest" 14 | @recursive true 15 | @moduledoc """ 16 | Reset consumer group's offsets. 17 | You should really carful in using this task. 18 | 19 | ## Examples 20 | 21 | mix cafex.offset_reset -t topic -g group -b localhost:9092 -s strategy 22 | 23 | ## Command line options 24 | 25 | * `-t`, `--topic` - Topic name 26 | * `-g`, `--group` - Consumer group name 27 | * `-b`, `--broker` - The Kafka broker in the form: `host1:port1` 28 | * `-s`, `--strategy` - Reset to the `latest` of `earliest` 29 | """ 30 | 31 | 32 | @doc false 33 | def run(args) do 34 | Mix.Task.run "compile" 35 | Logger.configure level: :warn 36 | 37 | {cli_opts, _, _} = OptionParser.parse args, 38 | aliases: [t: :topic, g: :group, b: :broker, s: :strategy] 39 | topic = Keyword.get(cli_opts, :topic) 40 | group = Keyword.get(cli_opts, :group) 41 | broker = Keyword.get(cli_opts, :broker) 42 | strategy = Keyword.get(cli_opts, :strategy) 43 | 44 | unless topic && group && broker && strategy do 45 | Mix.raise "Missed required arguments. Run `mix help #{Mix.Task.task_name(__MODULE__)}` for help" 46 | end 47 | 48 | strategy = strategy |> String.to_atom 49 | 50 | unless strategy in [:latest, :earliest] do 51 | Mix.raise "`-s` `--strategy` only support: [`:latest`, `:earliest`]" 52 | end 53 | 54 | %{servers: brokers} = parse_servers_url(broker) 55 | ensure_started 56 | 57 | {:ok, %{brokers: [%{host: host, port: port}|_]=brokers, 58 | topics: [%{partitions: partitions}]}} = Cafex.Kafka.Metadata.request(brokers, topic) 59 | brokers_map = Enum.map(brokers, fn %{node_id: node_id} = broker -> 60 | {node_id, broker} 61 | end) |> Enum.into(%{}) 62 | 63 | hwm = partitions 64 | |> Enum.group_by(fn %{leader: leader} -> leader end) 65 | |> Enum.map(fn {k, v} -> 66 | partitions = Enum.map(v, fn %{partition_id: partition} -> 67 | {partition, strategy, 1} 68 | end) 69 | request = %Offset.Request{topics: [{topic, partitions}]} 70 | %{host: host, port: port} = brokers_map[k] 71 | {:ok, conn} = Connection.start(host, port) 72 | {:ok, %{offsets: [{^topic, partitions}]}} = Connection.request(conn, request) 73 | Connection.close(conn) 74 | partitions 75 | end) 76 | |> List.flatten 77 | |> Enum.map(fn %{partition: partition, offsets: [offset]} -> 78 | {partition, offset, ""} 79 | end) 80 | 81 | {host, port} = get_coordinator(group, host, port) 82 | 83 | {:ok, conn} = Connection.start(host, port) 84 | 85 | request = %JoinGroup.Request{group_id: group, 86 | member_id: "", 87 | session_timeout: 6000, 88 | protocol_type: "consumer", 89 | group_protocols: [{"cafex", {0, [topic], ""}}]} 90 | {:ok, %{error: :no_error, 91 | generation_id: generation_id, 92 | member_id: member_id, 93 | members: members}} = Connection.request(conn, request) 94 | if length(members) <= 0 do 95 | Mix.raise "Not the leader, can't commit offset!" 96 | end 97 | 98 | members = Enum.map(members, fn {member_id, {0, [^topic], user_data}} -> 99 | {member_id, decode_partitions(user_data)} 100 | end) 101 | info_msg "Assigned members are: #{inspect members}" 102 | 103 | rebalanced = LoadBalancer.rebalance(members, length(partitions)) 104 | group_assignment = Enum.map(rebalanced, fn {member_id, assignment} -> 105 | member_assignment = {0, [{topic, assignment}], nil} 106 | {member_id, member_assignment} 107 | end) 108 | 109 | request = %SyncGroup.Request{group_id: group, 110 | member_id: member_id, 111 | generation_id: generation_id, 112 | group_assignment: group_assignment} 113 | 114 | {:ok, %{error: :no_error}} = Connection.request(conn, request) 115 | 116 | request = %OffsetCommit.Request{api_version: 1, 117 | consumer_group: group, 118 | consumer_id: member_id, 119 | consumer_group_generation_id: generation_id, 120 | topics: [{topic, hwm}]} 121 | {:ok, %{topics: [{^topic, partition_errors}]}} = Connection.request(conn, request) 122 | case Enum.find(partition_errors, fn {_, e} -> e != :no_error end) do 123 | nil -> success_msg "OffsetReset topic '#{topic}' with strategy: #{strategy} success." 124 | {p, e} -> error_msg "OffsetReset topic '#{topic}' partition: #{p} with #{strategy} offset error: #{inspect e}" 125 | end 126 | 127 | :ok = Connection.close(conn) 128 | end 129 | 130 | end 131 | -------------------------------------------------------------------------------- /lib/mix/tasks/cafex.topic.ex: -------------------------------------------------------------------------------- 1 | defmodule Mix.Tasks.Cafex.Topic do 2 | use Mix.Task 3 | import Mix.Cafex 4 | 5 | alias Cafex.Kafka.Metadata 6 | 7 | @shortdoc "Print topic metadata from Kafka." 8 | @recursive true 9 | @moduledoc """ 10 | This is a tool that reads topic metadata from Kafka and outputs it to standard output. 11 | 12 | ## Examples 13 | 14 | mix cafex.topic topic_name --brokers 192.168.99.100:9091 15 | 16 | ## Command line options 17 | 18 | * `-l`, `--list` - List topics 19 | * `-b`, `--broker` - The Kafka broker in the form: `host1:port1` 20 | """ 21 | 22 | @client_id "cafex_mix" 23 | 24 | @doc false 25 | def run(args) do 26 | Mix.Task.run "compile" 27 | Logger.configure level: :warn 28 | 29 | {cli_opts, args, _} = OptionParser.parse(args, aliases: [b: :broker, l: :list]) 30 | list = Keyword.get(cli_opts, :list) 31 | broker = Keyword.get(cli_opts, :broker) 32 | 33 | topic = case args do 34 | [t|_] -> t 35 | _ -> nil 36 | end 37 | 38 | args = cond do 39 | list == true -> 40 | nil 41 | topic == nil -> 42 | Mix.raise "`--topic topic` or `--list` must be given. Run `mix help #{Mix.Task.task_name __MODULE__}` for help" 43 | true -> 44 | topic 45 | end 46 | 47 | %{servers: brokers} = parse_servers_url(broker) 48 | ensure_started 49 | 50 | {:ok, metadata} = Metadata.request(brokers, args) 51 | 52 | success_msg "Brokers:" 53 | Enum.map(metadata.brokers, fn %{node_id: id, host: host, port: port} -> 54 | info_msg "Node ID: #{id}\tHost: #{host}\tPort: #{port}" 55 | end) 56 | info_msg "" 57 | 58 | cond do 59 | list == true -> 60 | list_topics(metadata.topics) 61 | true -> 62 | topic = Enum.find(metadata.topics, fn %{name: name} -> 63 | name == topic 64 | end) 65 | if topic do 66 | describe_topic(topic) 67 | else 68 | :ok 69 | end 70 | end 71 | 72 | :ok 73 | end 74 | 75 | defp list_topics(topics) do 76 | success_msg "Topics:" 77 | 78 | topics 79 | |> Enum.sort 80 | |> Enum.map(fn %{error: _error, name: name, partitions: _partitions} -> 81 | info_msg "#{name}" 82 | end) 83 | end 84 | 85 | defp describe_topic(%{error: _error, name: name, partitions: partitions}) do 86 | info_msg "Topic: #{name}" 87 | 88 | partitions 89 | |> Enum.sort(fn(%{partition_id: id1}, %{partition_id: id2}) -> 90 | id1 < id2 91 | end) 92 | |> Enum.map(fn %{error: err, partition_id: id, leader: leader, replicas: replicas, isrs: isrs} -> 93 | info_msg "partition: #{id}\tleader: #{leader}\treplicas: #{inspect replicas}\tisrs: #{inspect isrs}\terror: #{inspect err}" 94 | end) 95 | end 96 | end 97 | -------------------------------------------------------------------------------- /mix.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Mixfile do 2 | use Mix.Project 3 | 4 | def project do 5 | [app: :cafex, 6 | version: "0.0.3", 7 | elixir: "~> 1.2", 8 | description: description, 9 | build_embedded: Mix.env == :prod, 10 | start_permanent: Mix.env == :prod, 11 | consolidate_protocols: Mix.env != :test, 12 | deps: deps, 13 | test_paths: test_paths(Mix.env), 14 | 15 | aliases: ["test.all": ["test.default", "test.integration"], 16 | "test.integration": &test_integration/1, 17 | "test.default": &test_default/1], 18 | preferred_cli_env: ["test.all": :test], 19 | 20 | name: "Cafex", 21 | source_url: "https://github.com/upyun/cafex", 22 | homepage_url: "http://cafex.github.com/", 23 | docs: [extras: ["README.md"]], 24 | package: package, 25 | dialyzer: [flags: ["-Werror_handling", "-Wrace_conditions", "-Wunderspecs"]]] 26 | end 27 | 28 | # Configuration for the OTP application 29 | # 30 | # Type `mix help compile.app` for more information 31 | def application do 32 | [applications: [:logger, :erlzk, :consul], 33 | mod: {Cafex.Application, []}] 34 | end 35 | 36 | # Dependencies can be Hex packages: 37 | # 38 | # {:mydep, "~> 0.3.0"} 39 | # 40 | # Or git/path repositories: 41 | # 42 | # {:mydep, git: "https://github.com/elixir-lang/mydep.git", tag: "0.1.0"} 43 | # 44 | # Type `mix help deps` for more examples and options 45 | defp deps do 46 | [{:earmark, "~> 1.0.1", only: :dev}, 47 | {:ex_doc, "~> 0.13.0", only: :dev}, 48 | {:consul, github: "zolazhou/consul-ex"}, 49 | {:erlzk, "~> 0.6.3"}, 50 | {:snappy, "~> 1.1", only: [:dev, :test]}] 51 | end 52 | 53 | defp test_paths(:integration), do: ["integration_test"] 54 | defp test_paths(:all), do: ["test", "integration_test"] 55 | defp test_paths(_), do: ["test"] 56 | 57 | defp env_run(env, args) do 58 | args = if IO.ANSI.enabled?, do: ["--color"|args], else: ["--no-color"|args] 59 | 60 | IO.puts "==> Running tests for MIX_ENV=#{env} mix test" 61 | 62 | {_, res} = System.cmd "mix", ["test"|args], 63 | into: IO.binstream(:stdio, :line), 64 | env: [{"MIX_ENV", to_string(env)}] 65 | 66 | if res > 0 do 67 | System.at_exit(fn _ -> exit({:shutdown, 1}) end) 68 | end 69 | end 70 | 71 | defp test_integration(args), do: env_run(:integration, args) 72 | defp test_default(args), do: env_run(:test, args) 73 | 74 | defp package do 75 | [files: ~w(lib config/config.exs integration_test test LICENSE mix.exs mix.lock README.md), 76 | licenses: ["Apache 2.0"], 77 | maintainers: ["Belltoy Zhao"], 78 | links: %{"GitHub" => "https://github.com/upyun/cafex"}] 79 | end 80 | 81 | defp description do 82 | """ 83 | Cafex is a pure Elixir implementation of Kafka client. 84 | """ 85 | end 86 | end 87 | -------------------------------------------------------------------------------- /mix.lock: -------------------------------------------------------------------------------- 1 | %{"certifi": {:hex, :certifi, "0.4.0", "a7966efb868b179023618d29a407548f70c52466bf1849b9e8ebd0e34b7ea11f", [:rebar3], []}, 2 | "consul": {:git, "https://github.com/zolazhou/consul-ex.git", "66118fe59a9fd1e5ecbb860b3880777325e932b9", []}, 3 | "earmark": {:hex, :earmark, "1.0.1", "2c2cd903bfdc3de3f189bd9a8d4569a075b88a8981ded9a0d95672f6e2b63141", [:mix], []}, 4 | "erlzk": {:hex, :erlzk, "0.6.3", "54edff8f343e638da34bc176502646798da1618ddf1eb062a7fe73edc94e4e9e", [:rebar, :make], []}, 5 | "ex_doc": {:hex, :ex_doc, "0.13.0", "aa2f8fe4c6136a2f7cfc0a7e06805f82530e91df00e2bff4b4362002b43ada65", [:mix], [{:earmark, "~> 1.0", [hex: :earmark, optional: false]}]}, 6 | "hackney": {:hex, :hackney, "1.6.1", "ddd22d42db2b50e6a155439c8811b8f6df61a4395de10509714ad2751c6da817", [:rebar3], [{:certifi, "0.4.0", [hex: :certifi, optional: false]}, {:idna, "1.2.0", [hex: :idna, optional: false]}, {:metrics, "1.0.1", [hex: :metrics, optional: false]}, {:mimerl, "1.0.2", [hex: :mimerl, optional: false]}, {:ssl_verify_fun, "1.1.0", [hex: :ssl_verify_fun, optional: false]}]}, 7 | "httpoison": {:hex, :httpoison, "0.8.3", "b675a3fdc839a0b8d7a285c6b3747d6d596ae70b6ccb762233a990d7289ccae4", [:mix], [{:hackney, "~> 1.6.0", [hex: :hackney, optional: false]}]}, 8 | "idna": {:hex, :idna, "1.2.0", "ac62ee99da068f43c50dc69acf700e03a62a348360126260e87f2b54eced86b2", [:rebar3], []}, 9 | "metrics": {:hex, :metrics, "1.0.1", "25f094dea2cda98213cecc3aeff09e940299d950904393b2a29d191c346a8486", [:rebar3], []}, 10 | "mimerl": {:hex, :mimerl, "1.0.2", "993f9b0e084083405ed8252b99460c4f0563e41729ab42d9074fd5e52439be88", [:rebar3], []}, 11 | "poison": {:hex, :poison, "1.5.2", "560bdfb7449e3ddd23a096929fb9fc2122f709bcc758b2d5d5a5c7d0ea848910", [:mix], []}, 12 | "snappy": {:hex, :snappy, "1.1.1", "d63449a58f1358046b7adaec166ae8929c2b19881e76c4e13284304fe887c39a", [:rebar3], []}, 13 | "ssl_verify_fun": {:hex, :ssl_verify_fun, "1.1.0", "edee20847c42e379bf91261db474ffbe373f8acb56e9079acb6038d4e0bf414f", [:rebar, :make], []}, 14 | "ssl_verify_hostname": {:hex, :ssl_verify_hostname, "1.0.5", "2e73e068cd6393526f9fa6d399353d7c9477d6886ba005f323b592d389fb47be", [:make], []}} 15 | -------------------------------------------------------------------------------- /test/cafex_test.exs: -------------------------------------------------------------------------------- 1 | defmodule CafexTest do 2 | use ExUnit.Case 3 | 4 | test "the truth" do 5 | assert 1 + 1 == 2 6 | end 7 | end 8 | -------------------------------------------------------------------------------- /test/connection_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.ConnectionTest do 2 | use ExUnit.Case, async: true 3 | 4 | require Logger 5 | Logger.remove_backend(:console) 6 | 7 | alias Cafex.Connection 8 | 9 | defmodule Server do 10 | use GenServer 11 | 12 | def stop(pid) do 13 | GenServer.call pid, :stop 14 | end 15 | 16 | def port(pid) do 17 | GenServer.call pid, :port 18 | end 19 | 20 | def received(pid, sock) do 21 | GenServer.cast pid, {:received, sock} 22 | end 23 | 24 | def init([]) do 25 | {:ok, listen_sock} = :gen_tcp.listen(0, [:binary, {:packet, 4}, {:active, false}]) 26 | {:ok, port} = :inet.port listen_sock 27 | server_pid = self 28 | spawn fn -> 29 | accept(server_pid, listen_sock) 30 | end 31 | {:ok, {listen_sock, port}} 32 | end 33 | 34 | def handle_call(:stop, _from, state) do 35 | {:stop, :normal, :ok, state} 36 | end 37 | 38 | def handle_call(:port, _from, {_, port} = state) do 39 | {:reply, port, state} 40 | end 41 | 42 | def handle_cast({:received, sock}, state) do 43 | spawn fn -> 44 | do_receive sock 45 | end 46 | {:noreply, state} 47 | end 48 | 49 | defp accept(pid, listen_sock) do 50 | {:ok, sock} = :gen_tcp.accept(listen_sock) 51 | Server.received(pid, sock) 52 | accept(pid, listen_sock) 53 | end 54 | 55 | defp do_receive(sock) do 56 | case :gen_tcp.recv(sock, 0) do 57 | {:ok, bin} -> 58 | case bin do 59 | << 1 :: 16-signed, _rest :: binary >> -> # fetch api 60 | # Trigger server to close tcp connection 61 | :gen_tcp.close(sock) 62 | <<_api_key :: 16-signed, _api_version :: 16, correlation_id :: 32, client_len :: 16, _client_id :: size(client_len)-binary, id :: 32, msg_len :: 16, msg :: size(msg_len)-binary>> -> 63 | reply = <> 64 | :gen_tcp.send(sock, reply) 65 | do_receive(sock) 66 | _ -> 67 | :gen_tcp.close(sock) 68 | end 69 | {:error, :closed} -> 70 | :gen_tcp.close(sock) 71 | end 72 | end 73 | end 74 | 75 | defmodule TestApi do 76 | use Cafex.Protocol, api: :metadata 77 | 78 | defrequest do 79 | field :test_id, integer 80 | field :test_msg, binary 81 | end 82 | 83 | defresponse do 84 | end 85 | 86 | def decode(<>), do: {id, msg} 87 | 88 | def encode(%{test_id: id, test_msg: msg}) do 89 | <> 90 | end 91 | end 92 | 93 | defmodule BadApi do 94 | use Cafex.Protocol, api: :fetch 95 | defrequest do 96 | field :test_id, integer 97 | field :test_msg, binary 98 | end 99 | 100 | defresponse do 101 | end 102 | 103 | defdelegate encode(request), to: TestApi 104 | defdelegate decode(data), to: TestApi 105 | end 106 | 107 | setup do 108 | {:ok, pid} = GenServer.start Server, [] 109 | {:ok, pid: pid, port: Server.port(pid)} 110 | end 111 | 112 | test "connect and close", context do 113 | port = context[:port] 114 | {:ok, pid} = Connection.start "localhost", port 115 | 116 | assert Process.alive?(pid) 117 | assert :ok == Connection.close(pid) 118 | 119 | :timer.sleep(50) 120 | refute Process.alive?(pid) 121 | 122 | assert {:error, _reason} = Connection.start("unknown_host", 8080) 123 | end 124 | 125 | test "request", context do 126 | port = Server.port context[:pid] 127 | {:ok, pid} = Connection.start "localhost", port 128 | 129 | assert Process.alive?(pid) 130 | 131 | request1 = %TestApi.Request{test_id: 1, test_msg: "hello"} 132 | request2 = %TestApi.Request{test_id: 2, test_msg: "hello"} 133 | 134 | assert {:ok, {1, "hello"}} == Connection.request(pid, request1) 135 | assert {:ok, {2, "hello"}} == Connection.request(pid, request2) 136 | 137 | assert :ok == Connection.close(pid) 138 | 139 | refute Process.alive?(pid) 140 | 141 | {:ok, pid} = Connection.start "localhost", port 142 | request3 = %BadApi.Request{test_id: 3, test_msg: "hello"} 143 | assert {:closed, _} = catch_exit Connection.request(pid, request3) 144 | refute Process.alive?(pid) 145 | 146 | {:ok, pid} = Connection.start "localhost", port 147 | server_pid = context[:pid] 148 | assert Process.alive?(pid) 149 | assert :ok == Server.stop(server_pid) 150 | assert Process.alive?(pid) 151 | assert {:closed, _} = catch_exit Connection.request(pid, request1) 152 | end 153 | 154 | test "async request", context do 155 | port = Server.port context[:pid] 156 | {:ok, pid} = Connection.start "localhost", port 157 | 158 | assert Process.alive?(pid) 159 | 160 | request = %TestApi.Request{test_id: 3, test_msg: "hello"} 161 | 162 | Connection.async_request(pid, request, spawn(fn -> 163 | assert_receive {:ok, {3, "hello"}} 164 | end)) 165 | 166 | assert :ok == Connection.close(pid) 167 | 168 | :timer.sleep(50) 169 | refute Process.alive?(pid) 170 | end 171 | end 172 | -------------------------------------------------------------------------------- /test/consumer/load_balancer_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Consumer.LoadBalancer.Test do 2 | use ExUnit.Case, async: true 3 | 4 | import Cafex.Consumer.LoadBalancer 5 | doctest Cafex.Consumer.LoadBalancer 6 | 7 | test "more rebalance cases" do 8 | assert [{:a, [0, 1, 2, 3, 4]}] == rebalance([{:a, [0, 1, 2, 3, 4]}], 5) 9 | assert [{:a, [0, 1, 2, 3, 4]}] == rebalance([{:a, [0, 1, 2, 3]}], 5) 10 | assert [{:a, [0, 1, 2, 3, 4]}] == rebalance([{:a, [0, 1, 2]}], 5) 11 | assert [{:a, [0, 1, 2, 3, 4]}] == rebalance([{:a, [0, 1]}], 5) 12 | assert [{:a, [0, 1, 2, 3, 4]}] == rebalance([{:a, []}], 5) 13 | assert [{:a, [0, 1, 2, 3, 4]}] == rebalance([{:a, [0]}], 5) 14 | assert [{:a, [0, 1, 2, 3, 4]}] == rebalance([{:a, [1]}], 5) 15 | assert [{:a, [0, 1, 2, 3, 4]}] == rebalance([{:a, [1, 2, 3]}], 5) 16 | 17 | assert [{:a, [0, 1]}, {:b, [2, 3]}, {:c, [4]}] == rebalance([{:a, []}, {:b, []}, {:c, []}], 5) 18 | assert [{:a, [1, 2]}, {:b, [0, 3]}, {:c, [4]}] == rebalance([{:a, [1]}, {:b, [0]}, {:c, []}], 5) 19 | 20 | assert [{:a, [0]}, {:b, [1]}, {:c, [2]}, {:d, [3]}, {:e, []}] == rebalance([{:a, []}, {:b, []}, {:c, []}, {:d, []}, {:e, []}], 4) 21 | 22 | assert [{:a, [0, 1]}, {:b, [2]}, {:c, [3]}, {:d, [4]}] == rebalance([{:a, []}, {:b, []}, {:c, []}, {:d, []}], 5) 23 | assert [{:a, [0, 1]}, {:b, [2]}, {:c, [3]}] == rebalance([{:a, []}, {:b, []}, {:c, []}], 4) 24 | assert [{:a, [0, 1]}, {:b, [2]}, {:c, [3]}, {:d, [4]}] == rebalance([{:a, [0, 1]}, {:b, [2, 3]}, {:c, []}, {:d, []}], 5) 25 | 26 | assert [{:a, [0, 1]}, {:b, [3, 4]}, {:c, [2]}] == rebalance([{:a, [0, 1, 2]}, {:b, [3, 4]}, {:c, []}], 5) 27 | 28 | assert [{:a, [0, 1]}, {:b, [2, 3]}, {:c, [4]}, {:d, [5]}] == rebalance([{:a, []}, {:b, []}, {:c, []}, {:d, []}], 6) 29 | assert [{:a, [0, 1]}, {:b, [2, 3]}, {:c, [4, 5]}] == rebalance([{:a, []}, {:b, []}, {:c, []}], 6) 30 | assert [{:a, [0, 3]}, {:b, [4, 5]}, {:c, [1]}, {:d, [2]}] == rebalance([{:a, [0, 3]}, {:b, [4, 5]}, {:c, [1, 2]}, {:d, []}], 6) 31 | end 32 | end 33 | 34 | -------------------------------------------------------------------------------- /test/consumer/offset_manager_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Consumer.OffsetManager.Test do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Consumer.OffsetManager 5 | alias Cafex.Consumer.OffsetManager.State 6 | 7 | defmodule Conn do 8 | use GenServer 9 | 10 | def handle_call({:request, %{topics: [{topic, partitions}]}}, _from, state) do 11 | ps = Enum.map(partitions, fn({p, _o, _m}) -> 12 | {p, :no_error} 13 | end) 14 | reply = {:ok, %{topics: [{topic, ps}]}} 15 | {:reply, reply, state} 16 | end 17 | end 18 | 19 | defmodule OffsetManagerTest do 20 | use GenServer 21 | 22 | def get(pid) do 23 | GenServer.call pid, :get 24 | end 25 | 26 | def init(state) do 27 | {:ok, state} 28 | end 29 | 30 | def handle_call(:get, _from, {_, _, counts} = state) do 31 | {:reply, counts, state} 32 | end 33 | 34 | def handle_call(msg, from, {state, pid, cnt}) do 35 | case OffsetManager.handle_call(msg, from, state) do 36 | {:reply, reply, state} -> 37 | {:reply, reply, {state, pid, cnt}} 38 | {:reply, reply, state, timeout} -> 39 | {:reply, reply, {state, pid, cnt}, timeout} 40 | {:noreply, state} -> 41 | {:noreply, {state, pid, cnt}} 42 | {:noreply, state, timeout} -> 43 | {:noreply, {state, pid, cnt}, timeout} 44 | {:stop, reason, reply, state} -> 45 | {:stop, reason, reply, {state, pid, cnt}} 46 | {:stop, reason, state} -> 47 | {:stop, reason, {state, pid, cnt}} 48 | end 49 | end 50 | def handle_info({:timeout, ref, :do_commit} = msg, {state, pid, %{do_commit: c} = cnt}) do 51 | cnt = if is_reference(ref) do 52 | cnt 53 | else 54 | %{cnt| do_commit: c + 1} 55 | end 56 | do_handle_info(msg, {state, pid, cnt}) 57 | end 58 | def handle_info(msg, state) do 59 | do_handle_info(msg, state) 60 | end 61 | 62 | defp do_handle_info(msg, {state, pid, cnt}) do 63 | case OffsetManager.handle_info(msg, state) do 64 | {:noreply, state} -> 65 | {:noreply, {state, pid, cnt}} 66 | {:noreply, state, timeout} -> 67 | {:noreply, {state, pid, cnt}, timeout} 68 | {:stop, reason, state} -> 69 | {:stop, reason, {state, pid, cnt}} 70 | end 71 | end 72 | end 73 | 74 | test "Offset Manager" do 75 | partitions = 10 76 | 77 | to_be_commit = Enum.map(0..partitions, fn(p) -> 78 | {p, {0, ""}} 79 | end) |> Enum.into(%{}) 80 | 81 | {:ok, conn} = GenServer.start_link(Conn, []) 82 | 83 | state = %State{ 84 | coordinator: {"localhost", 9000}, 85 | topic: "test_topic", 86 | consumer_group: "test_group", 87 | partitions: partitions, 88 | auto_commit: true, 89 | interval: 500, 90 | max_buffers: 5, 91 | conn: conn, 92 | storage: :kafka, 93 | to_be_commit: to_be_commit 94 | } 95 | 96 | {:ok, pid} = GenServer.start_link(OffsetManagerTest, {state, self, %{do_commit: 0}}) 97 | GenServer.call pid, {:commit, 1, 1, ""} 98 | 99 | for partition <- 0..partitions do 100 | spawn(fn -> 101 | for offset <- 0..3 do 102 | GenServer.call pid, {:commit, partition, offset, ""} 103 | end 104 | end) 105 | end 106 | 107 | :timer.sleep(400) 108 | %{do_commit: commit_times} = OffsetManagerTest.get(pid) 109 | assert commit_times == 5 110 | end 111 | end 112 | -------------------------------------------------------------------------------- /test/consumer/util_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Consumer.Util.Test do 2 | use ExUnit.Case, async: true 3 | 4 | import Cafex.Consumer.Util 5 | doctest Cafex.Consumer.Util 6 | end 7 | -------------------------------------------------------------------------------- /test/consumer/worker_partition_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Consumer.WorkerPartitionTest do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Consumer.WorkerPartition 5 | 6 | test "WorkerPartition index test" do 7 | index = WorkerPartition.new 8 | 9 | assert [] == WorkerPartition.partitions(index) 10 | assert [] == WorkerPartition.workers(index) 11 | 12 | {w2p, p2w} = index = index 13 | |> WorkerPartition.update(1, 1) 14 | |> WorkerPartition.update(2, 2) 15 | |> WorkerPartition.update(3, 3) 16 | 17 | assert [1, 2, 3] = WorkerPartition.partitions(index) |> Enum.sort 18 | assert [1, 2, 3] = WorkerPartition.workers(index) |> Enum.sort 19 | 20 | assert_dict %{1 => 1, 2 => 2, 3 => 3}, w2p 21 | assert_dict %{1 => 1, 2 => 2, 3 => 3}, p2w 22 | 23 | assert 1 = WorkerPartition.worker(index, 1) 24 | assert 2 = WorkerPartition.worker(index, 2) 25 | assert 3 = WorkerPartition.worker(index, 3) 26 | 27 | assert 1 = WorkerPartition.partition(index, 1) 28 | assert 2 = WorkerPartition.partition(index, 2) 29 | assert 3 = WorkerPartition.partition(index, 3) 30 | 31 | {w2p, p2w} = index = index 32 | |> WorkerPartition.update(1, 4) 33 | |> WorkerPartition.update(2, 5) 34 | |> WorkerPartition.update(3, 6) 35 | 36 | assert [1, 2, 3] = WorkerPartition.partitions(index) |> Enum.sort 37 | assert [4, 5, 6] = WorkerPartition.workers(index) |> Enum.sort 38 | 39 | assert_dict %{4 => 1, 5 => 2, 6 => 3}, w2p 40 | assert_dict %{1 => 4, 2 => 5, 3 => 6}, p2w 41 | 42 | {w2p, p2w} = index = WorkerPartition.delete(index, 1, 4) 43 | 44 | assert [2, 3] = WorkerPartition.partitions(index) |> Enum.sort 45 | assert [5, 6] = WorkerPartition.workers(index) |> Enum.sort 46 | 47 | assert_dict %{5 => 2, 6 => 3}, w2p 48 | assert_dict %{2 => 5, 3 => 6}, p2w 49 | 50 | {w2p, p2w} = index 51 | |> WorkerPartition.update(2, 6) 52 | 53 | assert_dict %{6 => 2}, w2p 54 | assert_dict %{2 => 6}, p2w 55 | end 56 | 57 | defp assert_dict(value, dict) do 58 | assert value == Enum.into(dict, %{}) 59 | end 60 | end 61 | -------------------------------------------------------------------------------- /test/protocol/codec_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Codec.Test do 2 | use ExUnit.Case, async: true 3 | 4 | import Cafex.Protocol.Codec 5 | doctest Cafex.Protocol.Codec 6 | end 7 | -------------------------------------------------------------------------------- /test/protocol/describe_groups_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.DescribeGroups.Test do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Protocol.DescribeGroups 5 | alias Cafex.Protocol.DescribeGroups.Request 6 | alias Cafex.Protocol.DescribeGroups.Response 7 | 8 | test "DescribeGroup protocol implementation" do 9 | req = %Request{} 10 | assert DescribeGroups.has_response?(req) == true 11 | assert DescribeGroups.decoder(req) == DescribeGroups 12 | assert DescribeGroups.api_key(req) == Cafex.Protocol.api_key(:describe_groups) 13 | assert DescribeGroups.api_version(req) == 0 14 | end 15 | 16 | test "encode creates a valid DescribeGroups request" do 17 | good_request = <<1 :: 32, 19 :: 16, "cafex_test_consumer" :: binary >> 18 | 19 | request = %Request{groups: ["cafex_test_consumer"]} 20 | assert good_request == Cafex.Protocol.Request.encode(request) 21 | end 22 | 23 | test "parse_response correctly parse a valid response" do 24 | response = << 1 :: 32, 25 | 26 | 0 :: 16, 27 | 19 :: 16, 28 | "cafex_test_consumer" :: binary, 29 | 6 :: 16-signed, 30 | "Stable" :: binary, 31 | 8 :: 16, 32 | "consumer" :: binary, 33 | 5 :: 16, 34 | "cafex" :: binary, 35 | 36 | 1 :: 32, 37 | 38 | 42 :: 16, 39 | "cafex-4e8326b6-66d5-4ba5-877e-9feb4182462e" :: binary, 40 | 5 :: 16, 41 | "cafex" :: binary, 42 | 13 :: 16, 43 | "/192.168.99.1" :: binary, 44 | 22 :: 32, # metadata length 45 | 0 :: 16, 46 | 1 :: 32, 47 | 10 :: 16, 48 | "cafex_test" :: binary, 49 | 0 :: 32, 50 | 51 | 34 :: 32, # assignment length 52 | 0 :: 16, 53 | 1 :: 32, 54 | 10 :: 16, 55 | "cafex_test" :: binary, 56 | 2 :: 32, 57 | 3 :: 32, 58 | 4 :: 32, 59 | 0 :: 32 60 | >> 61 | 62 | expected_response = %Response{ 63 | groups: [ 64 | %{error: :no_error, 65 | group_id: "cafex_test_consumer", 66 | state: "Stable", 67 | protocol_type: "consumer", 68 | protocol: "cafex", 69 | members: [%{ 70 | member_id: "cafex-4e8326b6-66d5-4ba5-877e-9feb4182462e", 71 | client_id: "cafex", 72 | client_host: "/192.168.99.1", 73 | member_metadata: {0, ["cafex_test"], ""}, 74 | member_assignment: {0, [{"cafex_test", [3, 4]}], ""} 75 | }]} 76 | ]} 77 | 78 | assert expected_response == DescribeGroups.decode(response) 79 | end 80 | end 81 | -------------------------------------------------------------------------------- /test/protocol/fetch_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Fetch.Test do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Protocol.Message 5 | alias Cafex.Protocol.Fetch 6 | alias Cafex.Protocol.Fetch.Request 7 | alias Cafex.Protocol.Fetch.Response 8 | 9 | test "Fetch protocol implementation" do 10 | req = %Request{} 11 | assert Fetch.has_response?(req) == true 12 | assert Fetch.decoder(req) == Fetch 13 | assert Fetch.api_key(req) == Cafex.Protocol.api_key(:fetch) 14 | assert Fetch.api_version(req) == 0 15 | end 16 | 17 | test "encode creates a valid fetch request" do 18 | good_request = << -1 :: 32, 10 :: 32, 1 :: 32, 19 | 1 :: 32, 3 :: 16, "bar" :: binary, 20 | 1 :: 32, 0 :: 32, 1 :: 64, 10000 :: 32 >> 21 | request = %Request{ replica_id: -1, 22 | max_wait_time: 10, 23 | min_bytes: 1, 24 | topics: [{"bar", [{0, 1, 10000}]}] } 25 | assert good_request == Fetch.encode(request) 26 | end 27 | 28 | test "parse_response correctly parses a valid response with a key and a value" do 29 | response = << 1 :: 32, 3 :: 16, "bar" :: binary, 30 | 1 :: 32, 0 :: 32, 0 :: 16, 10 :: 64, 31 | 32 :: 32, 32 | 1 :: 64, 20 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, 33 | 3 :: 32, "foo" :: binary, 34 | 3 :: 32, "bar" :: binary >> 35 | 36 | expected_response = %Response{topics: [{"bar", [ 37 | %{error: :no_error, 38 | hwm_offset: 10, 39 | partition: 0, 40 | messages: [%Message{attributes: 0, 41 | key: "foo", 42 | offset: 1, 43 | timestamp_type: nil, 44 | magic_byte: 0, 45 | value: "bar"}] 46 | }]}]} 47 | assert expected_response == Fetch.decode(response) 48 | end 49 | 50 | test "parse_response correctly parses a response with excess bytes" do 51 | response = << 1 :: 32, 4 :: 16, "food" :: binary, 52 | 1 :: 32, 0 :: 32, 0 :: 16, 56 :: 64, 53 | 87 :: 32, 54 | 0 :: 64, 17 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, 55 | -1 :: 32, 3 :: 32, "hey" :: binary, 56 | 1 :: 64, 17 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, 57 | -1 :: 32, 3 :: 32, "hey" :: binary, 58 | 2 :: 64, 17 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, 59 | -1 :: 32, 3 :: 32, "hey" :: binary >> 60 | 61 | expected_response = %Response{topics: [{"food", [ 62 | %{error: :no_error, hwm_offset: 56, partition: 0, messages: [ 63 | %Message{attributes: 0, key: nil, offset: 0, value: "hey", timestamp_type: nil, magic_byte: 0}, 64 | %Message{attributes: 0, key: nil, offset: 1, value: "hey", timestamp_type: nil, magic_byte: 0}, 65 | %Message{attributes: 0, key: nil, offset: 2, value: "hey", timestamp_type: nil, magic_byte: 0} 66 | ]}]}]} 67 | assert expected_response == Fetch.decode(response) 68 | end 69 | 70 | test "parse_response correctly parses a valid response with a nil key and a value" do 71 | response = << 1 :: 32, 3 :: 16, "bar" :: binary, 72 | 1 :: 32, 0 :: 32, 0 :: 16, 10 :: 64, 73 | 29 :: 32, 74 | 1 :: 64, 17 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, -1 :: 32, 3 :: 32, "bar" :: binary >> 75 | expected_response = %Response{topics: [{"bar", [ 76 | %{error: :no_error, hwm_offset: 10, partition: 0, messages: [ 77 | %Message{attributes: 0, key: nil, offset: 1, value: "bar", timestamp_type: nil, magic_byte: 0} 78 | ]}]}]} 79 | assert expected_response == Fetch.decode(response) 80 | end 81 | 82 | test "parse_response correctly parses a empty message set response" do 83 | response = << 1 :: 32, 3 :: 16, "bar" :: binary, 84 | 1 :: 32, 0 :: 32, 0 :: 16, 10 :: 64, 85 | 0 :: 32 >> 86 | 87 | expected_response = %Response{topics: [{"bar", [ 88 | %{error: :no_error, hwm_offset: 10, partition: 0, messages: []}]}]} 89 | assert expected_response == Fetch.decode(response) 90 | end 91 | 92 | test "parse_response incorrectly parses a partial message set response" do 93 | response = << 1 :: 32, 3 :: 16, "bar" :: binary, 94 | 1 :: 32, 0 :: 32, 0 :: 16, 10 :: 64, 95 | 28 :: 32, 96 | 1 :: 64, 17 :: 32, 0 :: 32, 0 :: 8, 0 :: 8, -1 :: 32, 3 :: 32, "ba" :: binary >> 97 | 98 | expected_response = %Response{topics: [{"bar", [ 99 | %{error: :no_error, hwm_offset: 10, partition: 0, messages: []}]}]} 100 | assert expected_response == Fetch.decode(response) 101 | end 102 | end 103 | -------------------------------------------------------------------------------- /test/protocol/group_coordinator_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.GroupCoordinator.Test do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Protocol.GroupCoordinator 5 | alias Cafex.Protocol.GroupCoordinator.Request 6 | alias Cafex.Protocol.GroupCoordinator.Response 7 | 8 | test "GroupCoordinator protocol implementation" do 9 | req = %Request{} 10 | assert GroupCoordinator.has_response?(req) == true 11 | assert GroupCoordinator.decoder(req) == GroupCoordinator 12 | assert GroupCoordinator.api_key(req) == Cafex.Protocol.api_key(:group_coordinator) 13 | assert GroupCoordinator.api_version(req) == 0 14 | end 15 | 16 | test "create_request creates a valid consumer metadata request" do 17 | good_request = << 2 :: 16, "we" >> 18 | 19 | request = %Request{group_id: "we"} 20 | 21 | assert good_request == GroupCoordinator.encode(request) 22 | end 23 | 24 | test "parse_response correctly parses a valid response" do 25 | response = <<0, 0, 0, 0, 192, 6, 0, 14, 49, 57, 50, 46, 49, 54, 56, 46, 53, 57, 46, 49, 48, 51, 0, 0, 192, 6>> 26 | 27 | assert GroupCoordinator.decode(response) == %Response{ coordinator_id: 49158, 28 | coordinator_host: "192.168.59.103", 29 | coordinator_port: 49158, 30 | error: :no_error } 31 | end 32 | end 33 | -------------------------------------------------------------------------------- /test/protocol/heartbeat_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Heartbeat.Test do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Protocol.Heartbeat 5 | alias Cafex.Protocol.Heartbeat.Request 6 | alias Cafex.Protocol.Heartbeat.Response 7 | 8 | test "Heartbeat protocol implementation" do 9 | req = %Request{} 10 | assert Heartbeat.has_response?(req) == true 11 | assert Heartbeat.decoder(req) == Heartbeat 12 | assert Heartbeat.api_key(req) == Cafex.Protocol.api_key(:heartbeat) 13 | assert Heartbeat.api_version(req) == 0 14 | end 15 | 16 | test "encode a valid Heartbeat request" do 17 | good_request = << 19 :: 16-signed, 18 | "cafex_test_consumer" :: binary, 19 | 2 :: 32-signed, 20 | 42 :: 16-signed, 21 | "cafex-e6f5cd5e-116b-49a3-a3ca-8746779f3cc5" :: binary>> 22 | 23 | request = %Request{ 24 | group_id: "cafex_test_consumer", 25 | generation_id: 2, 26 | member_id: "cafex-e6f5cd5e-116b-49a3-a3ca-8746779f3cc5" 27 | } 28 | 29 | assert good_request == Cafex.Protocol.Request.encode(request) 30 | end 31 | 32 | test "parse_response correctly parse a valid response" do 33 | response = <<0 :: 16-signed>> 34 | expected_response = %Response{error: :no_error} 35 | 36 | assert expected_response == Heartbeat.decode(response) 37 | end 38 | end 39 | -------------------------------------------------------------------------------- /test/protocol/join_group_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.JoinGroup.Test do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Protocol.JoinGroup 5 | alias Cafex.Protocol.JoinGroup.Request 6 | alias Cafex.Protocol.JoinGroup.Response 7 | 8 | test "JoinGroup protocol implementation" do 9 | req = %Request{} 10 | assert JoinGroup.has_response?(req) == true 11 | assert JoinGroup.decoder(req) == JoinGroup 12 | assert JoinGroup.api_key(req) == Cafex.Protocol.api_key(:join_group) 13 | assert JoinGroup.api_version(req) == 0 14 | end 15 | 16 | test "encode creates a valid JoinGroup request" do 17 | good_request = << 19 :: 16, 18 | "cafex_test_consumer" :: binary, 19 | 20 | 7000 :: 32, # session_timeout 21 | -1 :: 16-signed, # member_id: nil 22 | 23 | 8 :: 16, 24 | "consumer" :: binary, 25 | 26 | 1 :: 32, # group_protocols length 27 | 28 | 5 :: 16, 29 | "cafex" :: binary, 30 | 31 | 22 :: 32, # group_protocols_metadata length 32 | 0 :: 16, 33 | 1 :: 32, 34 | 10 :: 16, 35 | "test_topic" :: binary, 36 | 0 :: 32-signed>> 37 | 38 | request = %Request{ 39 | group_id: "cafex_test_consumer", 40 | session_timeout: 7000, 41 | protocol_type: "consumer", 42 | group_protocols: [{"cafex", {0, ["test_topic"], ""}}] 43 | } 44 | 45 | assert good_request == Cafex.Protocol.Request.encode(request) 46 | end 47 | 48 | test "parse_response correctly parse a valid non-leader response" do 49 | response = << 0 :: 16, # error_code 50 | 2 :: 32, # generation_id 51 | 5 :: 16, 52 | "cafex", # group_protocol 53 | 54 | 42 :: 16, 55 | "cafex-e6f5cd5e-116b-49a3-a3ca-8746779f3cc5", # leader_id 56 | 57 | 42 :: 16, 58 | "cafex-4e8326b6-66d5-4ba5-877e-9feb4182462e", # member_id 59 | 60 | 0 :: 32-signed # members 61 | >> 62 | 63 | expected_response = %Response{ 64 | error: :no_error, 65 | generation_id: 2, 66 | group_protocol: "cafex", 67 | leader_id: "cafex-e6f5cd5e-116b-49a3-a3ca-8746779f3cc5", 68 | member_id: "cafex-4e8326b6-66d5-4ba5-877e-9feb4182462e", 69 | members: []} 70 | 71 | assert expected_response == JoinGroup.decode(response) 72 | end 73 | 74 | test "parse_response correctly parse a valid leader response" do 75 | response = << 0 :: 16-signed, # error_code 76 | 3 :: 32, # generation_id 77 | 5 :: 16, 78 | "cafex", # group_protocol 79 | 80 | 42 :: 16, 81 | "cafex-4e8326b6-66d5-4ba5-877e-9feb4182462e", # leader_id 82 | 83 | 42 :: 16, 84 | "cafex-4e8326b6-66d5-4ba5-877e-9feb4182462e", # member_id 85 | 86 | 1 :: 32, # members 87 | 88 | 42 :: 16, # member_id 89 | "cafex-4e8326b6-66d5-4ba5-877e-9feb4182462e", 90 | 91 | 27 :: 32, 92 | 0 :: 16, # version 93 | 1 :: 32, # subscriptions 94 | 10 :: 16, 95 | "cafex_test", 96 | 5 :: 32, 97 | "[3,4]" 98 | >> 99 | 100 | expected_response = %Response{ 101 | error: :no_error, 102 | generation_id: 3, 103 | group_protocol: "cafex", 104 | leader_id: "cafex-4e8326b6-66d5-4ba5-877e-9feb4182462e", 105 | member_id: "cafex-4e8326b6-66d5-4ba5-877e-9feb4182462e", 106 | members: [ 107 | {"cafex-4e8326b6-66d5-4ba5-877e-9feb4182462e", {0, ["cafex_test"], "[3,4]"}}]} 108 | 109 | assert expected_response == JoinGroup.decode(response) 110 | end 111 | end 112 | -------------------------------------------------------------------------------- /test/protocol/leave_group_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.LeaveGroup.Test do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Protocol.LeaveGroup 5 | alias Cafex.Protocol.LeaveGroup.Request 6 | alias Cafex.Protocol.LeaveGroup.Response 7 | 8 | test "LeaveGroup protocol implementation" do 9 | req = %Request{} 10 | assert LeaveGroup.has_response?(req) == true 11 | assert LeaveGroup.decoder(req) == LeaveGroup 12 | assert LeaveGroup.api_key(req) == Cafex.Protocol.api_key(:leave_group) 13 | assert LeaveGroup.api_version(req) == 0 14 | end 15 | 16 | test "encode a valid LeaveGroup request" do 17 | good_request = << 19 :: 16-signed, 18 | "cafex_test_consumer" :: binary, 19 | 42 :: 16-signed, 20 | "cafex-e6f5cd5e-116b-49a3-a3ca-8746779f3cc5" :: binary>> 21 | 22 | request = %Request{ 23 | group_id: "cafex_test_consumer", 24 | member_id: "cafex-e6f5cd5e-116b-49a3-a3ca-8746779f3cc5" 25 | } 26 | 27 | assert good_request == Cafex.Protocol.Request.encode(request) 28 | end 29 | 30 | test "parse_response correctly parse a valid response" do 31 | response = <<0 :: 16-signed>> 32 | expected_response = %Response{error: :no_error} 33 | 34 | assert expected_response == LeaveGroup.decode(response) 35 | end 36 | end 37 | -------------------------------------------------------------------------------- /test/protocol/list_groups_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.ListGroups.Test do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Protocol.ListGroups 5 | alias Cafex.Protocol.ListGroups.Request 6 | alias Cafex.Protocol.ListGroups.Response 7 | 8 | test "encode creates a valid ListGroups request" do 9 | good_request = <<>> 10 | 11 | request = %Request{} 12 | assert good_request == Cafex.Protocol.Request.encode(request) 13 | end 14 | 15 | test "parse_response correctly parse a valid response" do 16 | response = << 0 :: 16-signed, 17 | 1 :: 32-signed, 18 | 19 :: 16-signed, 19 | "cafex_test_consumer" :: binary, 20 | 8 :: 16-signed, 21 | "consumer" :: binary >> 22 | 23 | expected_response = %Response{error: :no_error, 24 | groups: [{"cafex_test_consumer", "consumer"}]} 25 | 26 | assert expected_response == ListGroups.decode(response) 27 | end 28 | end 29 | -------------------------------------------------------------------------------- /test/protocol/metadata_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Metadata.Test do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Protocol.Metadata 5 | alias Cafex.Protocol.Metadata.Request 6 | alias Cafex.Protocol.Metadata.Response 7 | 8 | test "Metadata protocol implementation" do 9 | req = %Request{} 10 | assert Metadata.has_response?(req) == true 11 | assert Metadata.decoder(req) == Metadata 12 | assert Metadata.api_key(req) == Cafex.Protocol.api_key(:metadata) 13 | assert Metadata.api_version(req) == 0 14 | end 15 | 16 | test "create_request with no topics creates a valid metadata request" do 17 | good_request = << 0 :: 32 >> 18 | request = %Request{} 19 | assert good_request == Cafex.Protocol.Request.encode(request) 20 | end 21 | 22 | test "create_request with a single topic creates a valid metadata request" do 23 | good_request = << 1 :: 32, 3 :: 16, "bar" :: binary >> 24 | request = %Request{topics: ["bar"]} 25 | assert good_request == Cafex.Protocol.Request.encode(request) 26 | end 27 | 28 | test "create_request with a multiple topics creates a valid metadata request" do 29 | good_request = << 3 :: 32, 3 :: 16, "bar" :: binary, 3 :: 16, "baz" :: binary, 4 :: 16, "food" :: binary >> 30 | request = %Request{topics: ["bar", "baz", "food"]} 31 | assert good_request == Cafex.Protocol.Request.encode(request) 32 | end 33 | 34 | test "parse_response correctly parses a valid response" do 35 | response = << 1 :: 32, 0 :: 32, 3 :: 16, "foo" :: binary, 9092 :: 32, 36 | 1 :: 32, 0 :: 16, 3 :: 16, "bar" :: binary, 37 | 1 :: 32, 0 :: 16, 0 :: 32, 0 :: 32, 0 :: 32, 1 :: 32, 0 :: 32 >> 38 | 39 | expected_response = %Response{ 40 | brokers: [%{host: "foo", node_id: 0, port: 9092}], 41 | topics: [ 42 | %{name: "bar", error: :no_error, partitions: [ 43 | %{error: :no_error, isrs: [0], leader: 0, partition_id: 0, replicas: []} 44 | ]} 45 | ] 46 | } 47 | 48 | assert expected_response == Metadata.decode(response) 49 | end 50 | end 51 | -------------------------------------------------------------------------------- /test/protocol/offset_commit_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.OffsetCommit.Test do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Protocol.OffsetCommit 5 | alias Cafex.Protocol.OffsetCommit.Request 6 | alias Cafex.Protocol.OffsetCommit.Response 7 | 8 | test "OffsetCommit protocol implementation" do 9 | req = %Request{} 10 | assert OffsetCommit.has_response?(req) == true 11 | assert OffsetCommit.decoder(req) == OffsetCommit 12 | assert OffsetCommit.api_key(req) == Cafex.Protocol.api_key(:offset_commit) 13 | assert OffsetCommit.api_version(%{req | api_version: 0}) == 0 14 | assert OffsetCommit.api_version(%{req | api_version: 1}) == 1 15 | end 16 | 17 | test "create_request creates a valid offset commit message with default version 0" do 18 | offset_commit_request_default = %Request{consumer_group: "bar", 19 | topics: [{"foo", [{0, 10, "baz"}]}]} 20 | offset_commit_request_v0 = %Request{api_version: 0, 21 | consumer_group: "bar", 22 | topics: [{"foo", [{0, 10, "baz"}]}]} 23 | 24 | good_request = << 3 :: 16, "bar", 1 :: 32, 3 :: 16, "foo", 1 :: 32, 0 :: 32, 10 :: 64, 3 :: 16, "baz" >> 25 | 26 | request_default = OffsetCommit.encode(offset_commit_request_default) 27 | request_v0 = OffsetCommit.encode(offset_commit_request_v0) 28 | 29 | assert request_default == good_request 30 | assert request_v0 == good_request 31 | end 32 | 33 | test "create_request creates a valid offset commit message with version 1" do 34 | offset_commit_request_v1 = %Request{api_version: 1, 35 | consumer_group: "bar", 36 | consumer_id: "consumer1", 37 | topics: [{"foo", [{0, 10, "baz"}]}]} 38 | 39 | good_request = << 3 :: 16, "bar", -1 :: 32, 9 :: 16, "consumer1", 1 :: 32, 3 :: 16, "foo", 1 :: 32, 0 :: 32, 10 :: 64, -1 :: 64, 3 :: 16, "baz" >> 40 | 41 | request_v1 = OffsetCommit.encode(offset_commit_request_v1) 42 | 43 | assert request_v1 == good_request 44 | end 45 | 46 | test "create_request creates a valid offset commit message with version 2" do 47 | offset_commit_request_v2 = %Request{api_version: 2, 48 | consumer_group: "bar", 49 | consumer_group_generation_id: -1, 50 | consumer_id: "consumer1", 51 | topics: [{"foo", [{0, 10, "baz"}]}]} 52 | 53 | good_request = << 3 :: 16, "bar", -1 :: 32, 9 :: 16, "consumer1", -1 :: 64, 1 :: 32, 3 :: 16, "foo", 1 :: 32, 0 :: 32, 10 :: 64, 3 :: 16, "baz" >> 54 | 55 | request_v2 = OffsetCommit.encode(offset_commit_request_v2) 56 | 57 | assert request_v2 == good_request 58 | end 59 | 60 | test "parse_response correctly parses a valid response" do 61 | response = <<0, 0, 0, 1, 0, 4, 102, 111, 111, 100, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0>> 62 | assert OffsetCommit.decode(response) == %Response{topics: [{"food", [{0, :no_error}]}]} 63 | end 64 | end 65 | -------------------------------------------------------------------------------- /test/protocol/offset_fetch_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.OffsetFetch.Test do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Protocol.OffsetFetch 5 | alias Cafex.Protocol.OffsetFetch.Request 6 | alias Cafex.Protocol.OffsetFetch.Response 7 | 8 | test "OffsetFetch protocol implementation" do 9 | req = %Request{} 10 | assert OffsetFetch.has_response?(req) == true 11 | assert OffsetFetch.decoder(req) == OffsetFetch 12 | assert OffsetFetch.api_key(req) == Cafex.Protocol.api_key(:offset_fetch) 13 | assert OffsetFetch.api_version(%{req | api_version: 0}) == 0 14 | assert OffsetFetch.api_version(%{req | api_version: 1}) == 1 15 | end 16 | 17 | test "create_request creates a valid offset commit message with default version 0" do 18 | offset_commit_request_default = %Request{ consumer_group: "bar", 19 | topics: [{"foo", [0]}] } 20 | offset_commit_request_v0 = %Request{ api_version: 0, 21 | consumer_group: "bar", 22 | topics: [{"foo", [0]}] } 23 | offset_commit_request_v1 = %Request{ api_version: 1, 24 | consumer_group: "bar", 25 | topics: [{"foo", [0]}] } 26 | offset_commit_request_v2 = %Request{ api_version: 2, 27 | consumer_group: "bar", 28 | topics: [{"foo", [0]}] } 29 | good_request = << 3 :: 16, "bar" :: binary, 1 :: 32, 3 :: 16, "foo" :: binary, 1 :: 32, 0 :: 32 >> 30 | request_default = OffsetFetch.encode(offset_commit_request_default) 31 | request_v0 = OffsetFetch.encode(offset_commit_request_v0) 32 | request_v1 = OffsetFetch.encode(offset_commit_request_v1) 33 | request_v2 = OffsetFetch.encode(offset_commit_request_v2) 34 | assert request_default == good_request 35 | assert request_v0 == good_request 36 | assert request_v1 == good_request 37 | assert request_v2 == good_request 38 | end 39 | 40 | test "parse_response correctly parses a valid response" do 41 | response1 = <<0, 0, 0, 1, 0, 4, 102, 111, 111, 100, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 9, 0, 0, 0, 0>> 42 | # response1 = <<1 :: 32, 4 :: 16, "food", 1 :: 32, 0 :: 32, 9 :: 64, 0 :: 16, 0 :: 16>> 43 | assert OffsetFetch.decode(response1) == %Response{topics: [{"food", [{0, 9, "", :no_error}]}]} 44 | 45 | response2 = <<0, 0, 0, 1, 0, 4, 102, 111, 111, 100, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 4, 255, 255, 0, 0>> 46 | # response2 = << 1 :: 32, 4 :: 16, "food", 1 :: 32, 1 :: 32, 4 :: 64, -1 :: 16, 0 :: 16>> 47 | assert OffsetFetch.decode(response2) == %Response{topics: [{"food", [{1, 4, "", :no_error}]}]} 48 | end 49 | end 50 | -------------------------------------------------------------------------------- /test/protocol/offset_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Offset.Test do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Protocol.Offset 5 | alias Cafex.Protocol.Offset.Request 6 | alias Cafex.Protocol.Offset.Response 7 | 8 | test "Offset protocol implementation" do 9 | req = %Request{} 10 | assert Offset.has_response?(req) == true 11 | assert Offset.decoder(req) == Offset 12 | assert Offset.api_key(req) == Cafex.Protocol.api_key(:offset) 13 | assert Offset.api_version(req) == 0 14 | end 15 | 16 | test "decode correctly parses a valid response with an offset" do 17 | response = << 1 :: 32, 3 :: 16, "bar" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 1 :: 32, 10 :: 64 >> 18 | expected_response = %Response{offsets: [{"bar", [%{error: :no_error, offsets: [10], partition: 0}]}]} 19 | 20 | assert expected_response == Offset.decode(response) 21 | end 22 | 23 | test "decode correctly parses a valid response with multiple offsets" do 24 | response = << 1 :: 32, 3 :: 16, "bar" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 2 :: 32, 10 :: 64, 20 :: 64 >> 25 | expected_response = %Response{offsets: [{"bar", [%{error: :no_error, offsets: [10, 20], partition: 0}]}]} 26 | 27 | assert expected_response == Offset.decode(response) 28 | end 29 | 30 | test "decode correctly parses a valid response with multiple partitions" do 31 | response = << 1 :: 32, 3 :: 16, "bar" :: binary, 2 :: 32, 0 :: 32, 0 :: 16, 1 :: 32, 10 :: 64, 1 :: 32, 0 :: 16, 1 :: 32, 20 :: 64 >> 32 | expected_response = %Response{offsets: [{"bar", [ 33 | %{error: :no_error, offsets: [10], partition: 0}, 34 | %{error: :no_error, offsets: [20], partition: 1} 35 | ]}]} 36 | 37 | assert expected_response == Offset.decode(response) 38 | end 39 | 40 | test "decode correctly parses a valid response with multiple topics" do 41 | response = << 2 :: 32, 3 :: 16, "bar" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 1 :: 32, 10 :: 64, 3 :: 16, "baz" :: binary, 1 :: 32, 0 :: 32, 0 :: 16, 1 :: 32, 20 :: 64 >> 42 | expected_response = %Response{offsets: [ 43 | {"bar", [%{error: :no_error, offsets: [10], partition: 0}]}, 44 | {"baz", [%{error: :no_error, offsets: [20], partition: 0}]} 45 | ]} 46 | 47 | assert expected_response == Offset.decode(response) 48 | end 49 | end 50 | -------------------------------------------------------------------------------- /test/protocol/sync_group_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.SyncGroup.Test do 2 | use ExUnit.Case, async: true 3 | 4 | alias Cafex.Protocol.SyncGroup 5 | alias Cafex.Protocol.SyncGroup.Request 6 | alias Cafex.Protocol.SyncGroup.Response 7 | 8 | test "SyncGroup protocol implementation" do 9 | req = %Request{} 10 | assert SyncGroup.has_response?(req) == true 11 | assert SyncGroup.decoder(req) == SyncGroup 12 | assert SyncGroup.api_key(req) == Cafex.Protocol.api_key(:sync_group) 13 | assert SyncGroup.api_version(req) == 0 14 | end 15 | 16 | test "encode creates a valid non-leader SyncGroup request" do 17 | good_request = << 19 :: 16, 18 | "cafex_test_consumer" :: binary, 19 | 1 :: 32, 20 | 42 :: 16, 21 | "cafex-e6f5cd5e-116b-49a3-a3ca-8746779f3cc5", 22 | 0 :: 32>> 23 | 24 | request = %Request{ 25 | group_id: "cafex_test_consumer", 26 | generation_id: 1, 27 | member_id: "cafex-e6f5cd5e-116b-49a3-a3ca-8746779f3cc5", 28 | group_assignment: [] 29 | } 30 | 31 | assert good_request == Cafex.Protocol.Request.encode(request) 32 | end 33 | 34 | test "encode creates a valid leader SyncGroup request" do 35 | good_request = << 19 :: 16, 36 | "cafex_test_consumer" :: binary, 37 | 1 :: 32, 38 | 42 :: 16, 39 | "cafex-e6f5cd5e-116b-49a3-a3ca-8746779f3cc5" :: binary, 40 | 41 | 1 :: 32, 42 | 43 | 42 :: 16, 44 | "cafex-e6f5cd5e-116b-49a3-a3ca-8746779f3cc5" :: binary, 45 | 46 | 46 :: 32, 47 | 48 | 0 :: 16, 49 | 50 | 1 :: 32, 51 | 10 :: 16, 52 | "cafex_test" :: binary, 53 | 54 | 5 :: 32, 55 | 0 :: 32, 56 | 1 :: 32, 57 | 2 :: 32, 58 | 3 :: 32, 59 | 4 :: 32, 60 | 61 | 0 :: 32 62 | >> 63 | 64 | request = %Request{ 65 | group_id: "cafex_test_consumer", 66 | generation_id: 1, 67 | member_id: "cafex-e6f5cd5e-116b-49a3-a3ca-8746779f3cc5", 68 | group_assignment: [ 69 | {"cafex-e6f5cd5e-116b-49a3-a3ca-8746779f3cc5", {0, [{"cafex_test", [0, 1, 2, 3, 4]}], ""}} 70 | ] 71 | } 72 | 73 | assert good_request == Cafex.Protocol.Request.encode(request) 74 | end 75 | 76 | test "parse_response correctly parse a valid response" do 77 | response = << 0 :: 16, 78 | 46 :: 32, 79 | 0 :: 16, 80 | 81 | 1 :: 32, 82 | 10 :: 16, 83 | "cafex_test" :: binary, 84 | 85 | 5 :: 32, 86 | 0 :: 32, 87 | 1 :: 32, 88 | 2 :: 32, 89 | 3 :: 32, 90 | 4 :: 32, 91 | 0 :: 32>> 92 | 93 | expected_response = %Response{ 94 | error: :no_error, 95 | member_assignment: {0, [{"cafex_test", [0, 1, 2, 3, 4]}], ""}} 96 | 97 | assert expected_response == SyncGroup.decode(response) 98 | end 99 | end 100 | -------------------------------------------------------------------------------- /test/protocol_test.exs: -------------------------------------------------------------------------------- 1 | defmodule Cafex.Protocol.Test do 2 | use ExUnit.Case, async: true 3 | 4 | test "api keys" do 5 | assert Cafex.Protocol.api_key(:produce) == 0 6 | assert Cafex.Protocol.api_key(:fetch) == 1 7 | assert Cafex.Protocol.api_key(:offset) == 2 8 | assert Cafex.Protocol.api_key(:metadata) == 3 9 | assert Cafex.Protocol.api_key(:offset_commit) == 8 10 | assert Cafex.Protocol.api_key(:offset_fetch) == 9 11 | assert Cafex.Protocol.api_key(:group_coordinator) == 10 12 | assert Cafex.Protocol.api_key(:join_group) == 11 13 | assert Cafex.Protocol.api_key(:heartbeat) == 12 14 | assert Cafex.Protocol.api_key(:leave_group) == 13 15 | assert Cafex.Protocol.api_key(:sync_group) == 14 16 | assert Cafex.Protocol.api_key(:describe_groups) == 15 17 | assert Cafex.Protocol.api_key(:list_groups) == 16 18 | end 19 | 20 | test "macros compile" do 21 | message = ~r"To use Cafex.Protocol, `api` must be set" 22 | assert_raise CompileError, message, fn -> Code.compile_quoted(quote do 23 | defmodule BadApi do 24 | use Cafex.Protocol 25 | defrequest do 26 | end 27 | 28 | defresponse do 29 | end 30 | end 31 | end) end 32 | 33 | message = ~r"Unsupported api: -1" 34 | assert_raise CompileError, message, fn -> Code.compile_quoted(quote do 35 | defmodule BadApi do 36 | use Cafex.Protocol, api: -1 37 | defrequest do 38 | end 39 | 40 | defresponse do 41 | end 42 | end 43 | end) end 44 | 45 | message = ~r"Use Cafex.Protocol must call `defresponse`" 46 | assert_raise CompileError, message, fn -> Code.compile_quoted(quote do 47 | defmodule BadApi do 48 | use Cafex.Protocol, api: :produce 49 | defrequest do 50 | end 51 | end 52 | end) end 53 | 54 | assert [ 55 | {TestApi.Response, _}, 56 | {Cafex.Protocol.Request.TestApi.Request, _}, 57 | {TestApi.Request, _}, 58 | {TestApi, _} 59 | ] = Code.compile_quoted(quote do 60 | defmodule TestApi do 61 | use Cafex.Protocol, api: :metadata 62 | defresponse do 63 | end 64 | 65 | def decode(_), do: %TestApi.Response{} 66 | end 67 | end) 68 | 69 | assert [ 70 | {Cafex.Protocol.Request.TestApi2.Request, _}, 71 | {TestApi2.Request, _}, 72 | {TestApi2.Response, _}, 73 | {TestApi2, _} 74 | ] = Code.compile_quoted(quote do 75 | defmodule TestApi2 do 76 | use Cafex.Protocol, api: :metadata 77 | 78 | defrequest do 79 | field :name, binary 80 | end 81 | 82 | defresponse do 83 | end 84 | 85 | def decode(_), do: %TestApi2.Response{} 86 | def encode(%TestApi2.Request{}), do: <<>> 87 | end 88 | end) 89 | end 90 | 91 | defmodule TestApi do 92 | use Cafex.Protocol, api: :metadata 93 | 94 | defresponse do 95 | end 96 | 97 | def decode(<<>>), do: %TestApi.Response{} 98 | end 99 | 100 | test "generated kafka API module" do 101 | req = %TestApi.Request{} 102 | res = %TestApi.Response{} 103 | assert TestApi.encode(req) == <<>> 104 | assert TestApi.decode(<<>>) == res 105 | assert TestApi.api_key(req) == Cafex.Protocol.api_key(:metadata) 106 | assert TestApi.api_version(req) == 0 107 | assert TestApi.has_response?(req) == true 108 | assert :ok == Protocol.assert_impl!(Cafex.Protocol.Request, TestApi.Request) 109 | end 110 | end 111 | 112 | -------------------------------------------------------------------------------- /test/test_helper.exs: -------------------------------------------------------------------------------- 1 | ExUnit.start() 2 | 3 | defmodule TestHelper do 4 | def generate_random_string(string_length \\ 20) do 5 | :random.seed(:os.timestamp) 6 | Enum.map(1..string_length, fn _ -> (:random.uniform * 25 + 65) |> round end) |> to_string 7 | end 8 | 9 | def uris do 10 | Mix.Config.read!("config/config.exs") |> hd |> elem(1) |> hd |> elem(1) 11 | end 12 | 13 | def utc_time do 14 | {x, {a,b,c}} = :calendar.local_time |> :calendar.local_time_to_universal_time_dst |> hd 15 | {x, {a,b,c + 60}} 16 | end 17 | end 18 | --------------------------------------------------------------------------------