diff --git a/.gitignore b/.gitignore new file mode 100644 index 0000000..0b34ef0 --- /dev/null +++ b/.gitignore @@ -0,0 +1,7 @@ +build +build.ninja + +.idea +.project +.cproject +.settings diff --git a/CMakeLists.txt b/CMakeLists.txt new file mode 100644 index 0000000..50e4dc7 --- /dev/null +++ b/CMakeLists.txt @@ -0,0 +1,107 @@ +# +# This file is open source software, licensed to you under the terms +# of the Apache License, Version 2.0 (the "License"). See the NOTICE file +# distributed with this work for additional information regarding copyright +# ownership. You may not use this file except in compliance with the License. +# +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +# +# Copyright (C) 2018 Scylladb, Ltd. +# + +cmake_minimum_required (VERSION 3.5) +project (kafka4seastar) + +find_package (Seastar REQUIRED) + +set(HEADER_DIRECTORY include/kafka4seastar) + +set(HEADERS + ${HEADER_DIRECTORY}/connection/connection_manager.hh + ${HEADER_DIRECTORY}/connection/kafka_connection.hh + ${HEADER_DIRECTORY}/connection/tcp_connection.hh + ${HEADER_DIRECTORY}/producer/batcher.hh + ${HEADER_DIRECTORY}/producer/kafka_producer.hh + ${HEADER_DIRECTORY}/producer/producer_properties.hh + ${HEADER_DIRECTORY}/producer/sender.hh + ${HEADER_DIRECTORY}/protocol/kafka_error_code.hh + ${HEADER_DIRECTORY}/protocol/kafka_primitives.hh + ${HEADER_DIRECTORY}/protocol/kafka_records.hh + ${HEADER_DIRECTORY}/protocol/api_versions_request.hh + ${HEADER_DIRECTORY}/protocol/api_versions_response.hh + ${HEADER_DIRECTORY}/protocol/headers.hh + ${HEADER_DIRECTORY}/protocol/metadata_request.hh + ${HEADER_DIRECTORY}/protocol/metadata_response.hh + ${HEADER_DIRECTORY}/protocol/produce_request.hh + ${HEADER_DIRECTORY}/protocol/produce_response.hh + ${HEADER_DIRECTORY}/utils/defaults.hh + ${HEADER_DIRECTORY}/utils/metadata_manager.hh + ${HEADER_DIRECTORY}/utils/partitioner.hh + ${HEADER_DIRECTORY}/utils/retry_helper.hh) + +set(SOURCES + src/connection/connection_manager.cc + src/connection/kafka_connection.cc + src/connection/tcp_connection.cc + src/producer/batcher.cc + src/producer/kafka_producer.cc + src/producer/sender.cc + src/protocol/kafka_error_code.cc + src/protocol/kafka_records.cc + src/protocol/api_versions_request.cc + src/protocol/api_versions_response.cc + src/protocol/headers.cc + src/protocol/metadata_request.cc + src/protocol/metadata_response.cc + src/protocol/produce_request.cc + src/protocol/produce_response.cc + src/utils/defaults.cc + src/utils/metadata_manager.cc + src/utils/partitioner.cc) + +add_library(kafka4seastar STATIC ${SOURCES}) + +target_include_directories(kafka4seastar + PUBLIC + $ + $ + PRIVATE + ${CMAKE_CURRENT_SOURCE_DIR}/src) + +add_subdirectory(tests/unit) + +add_executable (kafka_demo + demo/kafka_demo.cc) + +target_link_libraries (kafka4seastar + Seastar::seastar + Seastar::seastar_testing) + +target_link_libraries(kafka_demo + kafka4seastar) + +include(GNUInstallDirs) + +install(TARGETS kafka4seastar + EXPORT kafka4seastar-export + LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} + ARCHIVE DESTINATION ${CMAKE_INSTALL_LIBDIR}) + +install(EXPORT kafka4seastar-export + FILE + Kafka4seastarTargets.cmake + NAMESPACE + Kafka4seastar:: + DESTINATION + ${CMAKE_INSTALL_LIBDIR}/cmake/kafka4seastar) \ No newline at end of file diff --git a/Kafka4seastarConfig.cmake b/Kafka4seastarConfig.cmake new file mode 100644 index 0000000..afb79fb --- /dev/null +++ b/Kafka4seastarConfig.cmake @@ -0,0 +1,8 @@ +get_filename_component(kafka4seastar_CMAKE_DIR "${CMAKE_CURRENT_LIST_FILE}" PATH) +include(CMakeFindDependencyMacro) + +find_dependency(Seastar REQUIRED) + +if(NOT TARGET Kafka4seastar::kafka4seastar) + include("${kafka4seastar_CMAKE_DIR}/Kafka4seastarTargets.cmake") +endif() diff --git a/demo/kafka_demo.cc b/demo/kafka_demo.cc new file mode 100644 index 0000000..28dda03 --- /dev/null +++ b/demo/kafka_demo.cc @@ -0,0 +1,92 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include +#include +#include +#include +#include +#include +#include + +using namespace seastar; + +namespace bpo = boost::program_options; +namespace k4s = kafka4seastar; + +seastar::future async_stdin_read() { + return seastar::smp::submit_to(1, []{ + sstring res; + std::cin >> res; + return res; + }); +} + +int main(int ac, char** av) { + app_template app; + app.add_options() + ("host", bpo::value()->default_value("172.13.0.1"), "Address of the Kafka broker") + ("port", bpo::value()->default_value(9092), "Port to connect through"); + + return app.run(ac, av, [&app] { + return seastar::async([&app] { + auto&& config = app.configuration(); + std::string host = config["host"].as(); + uint16_t port = config["port"].as(); + (void) port; + + k4s::producer_properties properties; + properties._client_id = "seastar-kafka-demo"; + properties._request_timeout = 1000; + properties._servers = { + {host, port}, + {"172.29.0.1", 9092} + }; + + k4s::kafka_producer producer(std::move(properties)); + producer.init().wait(); + fprint(std::cout, "Producer initialized and ready to send\n\n"); + + sstring topic, key, value; + while (true) { + fprint(std::cout, + "\nType the topic and the message you want to send below. If you want to quit type 'q'\n"); + fprint(std::cout, "Enter topic: "); + topic = async_stdin_read().get0(); + + if (topic.empty() || topic == "q") { + producer.disconnect().wait(); + fprint(std::cout, "Finished succesfully!\n"); + break; + } + + fprint(std::cout, "Enter key: "); + key = async_stdin_read().get0(); + fprint(std::cout, "Enter value: "); + value = async_stdin_read().get0(); + + (void)producer.produce(topic, key, value).handle_exception([key, value](auto ep) { + fprint(std::cout, "Failure sending %s %s: %s.\n", key, value, ep); + }); + } + }); + }); +} diff --git a/include/kafka4seastar/connection/connection_manager.hh b/include/kafka4seastar/connection/connection_manager.hh new file mode 100644 index 0000000..8441dcc --- /dev/null +++ b/include/kafka4seastar/connection/connection_manager.hh @@ -0,0 +1,152 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include +#include + +#include + +using namespace seastar; + +namespace kafka4seastar { + +struct metadata_refresh_exception : public std::runtime_error { +public: + explicit metadata_refresh_exception(const seastar::sstring& message) : runtime_error(message) {} +}; + +class connection_manager { +public: + + using connection_id = std::pair; + using connection_iterator = std::map>::iterator; + +private: + + std::map> _connections; + seastar::sstring _client_id; + + semaphore _send_semaphore; + + future<> _pending_queue; + + future connect(const seastar::sstring& host, uint16_t port, uint32_t timeout); + + template + future> perform_request(connection_iterator& conn, RequestType& request, bool with_response) { + auto send_future = with_response + ? conn->second->send(std::move(request)) + : conn->second->send_without_response(std::move(request)); + + promise<> promise; + auto f = promise.get_future(); + _pending_queue = _pending_queue.then([f = std::move(f)] () mutable { + return std::move(f); + }); + + send_future = send_future.then([promise = std::move(promise)] (auto response) mutable { + promise.set_value(); + return response; + }); + + return make_ready_future(std::move(send_future)); + } + +public: + + explicit connection_manager(seastar::sstring client_id) + : _client_id(std::move(client_id)), + _send_semaphore(1), + _pending_queue(make_ready_future<>()) {} + + future<> init(const std::set& servers, uint32_t request_timeout); + connection_iterator get_connection(const connection_id& connection); + future<> disconnect(const connection_id& connection); + + template + future send(RequestType&& request, const seastar::sstring& host, + uint16_t port, uint32_t timeout, bool with_response=true) { + // In order to preserve ordering of sends, a semaphore with + // count = 1 is used due to its FIFO guarantees. + // + // It is important that connect() and send() are done + // with semaphore, as naive implementation + // + // connect(host, port).then([](auto conn) { conn.send(req1); }); + // connect(host, port).then([](auto conn) { conn.send(req2); }); + // + // could introduce reordering of requests: after both + // connects resolve as ready futures, the continuations (sends) + // are not guaranteed to run with any specific order. + // + // In order to not limit concurrency, send_future is + // returned as future> and "unpacked" + // outside the semaphore - scheduling inside semaphore + // (only 1 at the time) and waiting for result outside it. + return with_semaphore(_send_semaphore, 1, [this, request = std::move(request), host, port, timeout, with_response] () mutable { + auto conn = get_connection({host, port}); + if (conn != _connections.end()) { + return perform_request(conn, request, with_response); + } else { + return connect(host, port, timeout).then([this, request = std::move(request), with_response](connection_iterator conn) mutable { + return perform_request(conn, request, with_response); + }); + } + }).then([](future send_future) { + return send_future; + }).then([this, host, port](typename RequestType::response_type response) { + if (response._error_code == error::kafka_error_code::REQUEST_TIMED_OUT || + response._error_code == error::kafka_error_code::CORRUPT_MESSAGE || + response._error_code == error::kafka_error_code::NETWORK_EXCEPTION) { + _pending_queue = _pending_queue.then([this, host, port] { + return disconnect({host, port}); + }); + } + return response; + }).handle_exception([this, host, port] (std::exception_ptr ep) { + try { + _pending_queue = _pending_queue.then([this, host, port] { + return disconnect({host, port}); + }); + std::rethrow_exception(ep); + } catch (seastar::timed_out_error& e) { + typename RequestType::response_type response; + response._error_code = error::kafka_error_code::REQUEST_TIMED_OUT; + return response; + } catch (...) { + typename RequestType::response_type response; + response._error_code = error::kafka_error_code::NETWORK_EXCEPTION; + return response; + } + }); + } + + future ask_for_metadata(metadata_request&& request); + + future<> disconnect_all(); + +}; + +} diff --git a/include/kafka4seastar/connection/kafka_connection.hh b/include/kafka4seastar/connection/kafka_connection.hh new file mode 100644 index 0000000..f694d13 --- /dev/null +++ b/include/kafka4seastar/connection/kafka_connection.hh @@ -0,0 +1,210 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include +#include +#include + +#include +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +class kafka_connection final { + + tcp_connection _connection; + seastar::sstring _client_id; + int32_t _correlation_id; + api_versions_response _api_versions; + semaphore _send_semaphore; + semaphore _receive_semaphore; + + template + temporary_buffer serialize_request(RequestType request, int32_t correlation_id, int16_t api_version) { + thread_local kafka::output_stream header_stream = kafka::output_stream::resizable_stream(); + header_stream.reset(); + + request_header req_header; + req_header._api_key = RequestType::API_KEY; + req_header._api_version = api_version; + req_header._correlation_id = correlation_id; + req_header._client_id = _client_id; + req_header.serialize(header_stream, 0); + + thread_local kafka::output_stream payload_stream = kafka::output_stream::resizable_stream(); + payload_stream.reset(); + + request.serialize(payload_stream, api_version); + + thread_local kafka::output_stream message_stream = kafka::output_stream::resizable_stream(); + message_stream.reset(); + + kafka_int32_t message_size(header_stream.size() + payload_stream.size()); + message_size.serialize(message_stream, 0); + message_stream.write(header_stream.begin(), header_stream.size()); + message_stream.write(payload_stream.begin(), payload_stream.size()); + + return temporary_buffer{message_stream.begin(), message_stream.size()}; + } + + future<> send_request(temporary_buffer message_buffer) { + return _connection.write(std::move(message_buffer)); + } + + template + future receive_response(int32_t correlation_id, int16_t api_version) { + return _connection.read(4).then([] (temporary_buffer response_size) { + kafka::input_stream response_size_stream(response_size.get(), response_size.size()); + + kafka_int32_t size; + size.deserialize(response_size_stream, 0); + return *size; + }).then([this] (int32_t response_size) { + return _connection.read(response_size); + }).then([correlation_id, api_version] (temporary_buffer response) { + kafka::input_stream response_stream(response.get(), response.size()); + + response_header response_header; + response_header.deserialize(response_stream, 0); + if (*response_header._correlation_id != correlation_id) { + throw parsing_exception("Received invalid correlation id"); + } + + typename RequestType::response_type deserialized_response; + deserialized_response.deserialize(response_stream, api_version); + + return deserialized_response; + }); + } + + future<> init(); + +public: + static future> connect(const seastar::sstring& host, uint16_t port, + const seastar::sstring& client_id, uint32_t timeout_ms); + + kafka_connection(tcp_connection connection, seastar::sstring client_id) : + _connection(std::move(connection)), + _client_id(std::move(client_id)), + _correlation_id(0), + _send_semaphore(1), + _receive_semaphore(1) {} + + kafka_connection(kafka_connection&& other) = default; + kafka_connection(kafka_connection& other) = delete; + + future<> close(); + + template + future send(RequestType request) { + return send(std::move(request), _api_versions.max_version()); + } + + template + future send(RequestType request, int16_t api_version) { + auto correlation_id = _correlation_id++; + auto serialized_message = serialize_request(std::move(request), correlation_id, api_version); + + // In order to preserve ordering of sends, two semaphores with + // count = 1 are used due to its FIFO guarantees. + // + // Send and receive are always queued jointly, + // so that receive will get response from correct + // request. Kafka guarantees that responses will + // be sent in the same order that requests were sent. + // + // Usage of two semaphores makes it possible for + // requests to be sent without waiting for + // the previous response. + auto request_future = with_semaphore(_send_semaphore, 1, + [this, serialized_message = std::move(serialized_message)]() mutable { + return send_request(std::move(serialized_message)); + }).handle_exception([] (std::exception_ptr ep) { + // Ignore exception as it will be handled in response_future + }); + auto response_future = with_semaphore(_receive_semaphore, 1, [this, correlation_id, api_version] { + return receive_response(correlation_id, api_version); + }).handle_exception([] (std::exception_ptr ep) { + try { + std::rethrow_exception(ep); + } catch (seastar::timed_out_error& e) { + typename RequestType::response_type response; + response._error_code = error::kafka_error_code::REQUEST_TIMED_OUT; + return response; + } catch (parsing_exception& e) { + typename RequestType::response_type response; + response._error_code = error::kafka_error_code::CORRUPT_MESSAGE; + return response; + } catch (...) { + typename RequestType::response_type response; + response._error_code = error::kafka_error_code::NETWORK_EXCEPTION; + return response; + } + }); + return response_future; + } + + template + future send_without_response(RequestType request) { + return send_without_response(std::move(request), _api_versions.max_version()); + } + + template + future send_without_response(RequestType request, int16_t api_version) { + auto correlation_id = _correlation_id++; + auto serialized_message = serialize_request(std::move(request), correlation_id, api_version); + + auto request_future = with_semaphore(_send_semaphore, 1, + [this, serialized_message = std::move(serialized_message)]() mutable { + return send_request(std::move(serialized_message)); + }).then([] { + typename RequestType::response_type response; + response._error_code = error::kafka_error_code::NONE; + return response; + }).handle_exception([] (auto ep) { + try { + std::rethrow_exception(ep); + } catch (seastar::timed_out_error& e) { + typename RequestType::response_type response; + response._error_code = error::kafka_error_code::REQUEST_TIMED_OUT; + return response; + } catch (parsing_exception& e) { + typename RequestType::response_type response; + response._error_code = error::kafka_error_code::CORRUPT_MESSAGE; + return response; + } catch (...) { + typename RequestType::response_type response; + response._error_code = error::kafka_error_code::NETWORK_EXCEPTION; + return response; + } + }); + return request_future; + } +}; + +} diff --git a/include/kafka4seastar/connection/tcp_connection.hh b/include/kafka4seastar/connection/tcp_connection.hh new file mode 100644 index 0000000..bbc91a3 --- /dev/null +++ b/include/kafka4seastar/connection/tcp_connection.hh @@ -0,0 +1,78 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include +#include +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +inline struct connected_socket_input_stream_config duperel() { +struct connected_socket_input_stream_config cfg; +cfg.buffer_size = 32; +cfg.min_buffer_size = 1; +cfg.max_buffer_size = 1024; +return cfg; +} + +struct tcp_connection_exception final : public std::runtime_error { + explicit tcp_connection_exception(const seastar::sstring& message) : runtime_error(message) {} +}; + +class tcp_connection final { + + net::inet_address _host; + uint16_t _port; + uint32_t _timeout_ms; + connected_socket _fd; + input_stream _read_buf; + output_stream _write_buf; + +public: + static future connect(const seastar::sstring& host, uint16_t port, uint32_t timeout_ms); + + tcp_connection(const net::inet_address& host, uint16_t port, uint32_t timeout_ms, connected_socket&& fd) noexcept + : _host(host) + , _port(port) + , _timeout_ms(timeout_ms) + , _fd(std::move(fd)) + , _read_buf(_fd.input(duperel())) + , _write_buf(_fd.output(8192 * 512)) { + _fd.set_nodelay(true); +}; + + tcp_connection(tcp_connection&& other) = default; + tcp_connection(tcp_connection& other) = delete; + + future<> write(temporary_buffer buff); + future> read(size_t bytes_to_read); + future<> close(); + +}; + +} diff --git a/include/kafka4seastar/producer/batcher.hh b/include/kafka4seastar/producer/batcher.hh new file mode 100644 index 0000000..3df8c60 --- /dev/null +++ b/include/kafka4seastar/producer/batcher.hh @@ -0,0 +1,71 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include + +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +class batcher { +private: + std::vector _messages; + size_t _messages_byte_size; + uint32_t _buffer_memory; + metadata_manager& _metadata_manager; + connection_manager& _connection_manager; + retry_helper _retry_helper; + ack_policy _acks; + uint32_t _request_timeout; + + bool _keep_refreshing = false; + semaphore _refresh_finished = 0; + abort_source _stop_refresh; + uint32_t _expiration_time; +public: + batcher(metadata_manager& metadata_manager, connection_manager& connection_manager, + uint32_t max_retries, ack_policy acks, uint32_t request_timeout, uint32_t expiration_time, + uint32_t buffer_memory, noncopyable_function(uint32_t)> retry_strategy) + : _messages_byte_size(0), + _buffer_memory(buffer_memory), + _metadata_manager(metadata_manager), + _connection_manager(connection_manager), + _retry_helper(max_retries, std::move(retry_strategy)), + _acks(acks), + _request_timeout(request_timeout), + _expiration_time(expiration_time) {} + + void queue_message(sender_message message); + future<> flush(); + future<> flush_coroutine(std::chrono::milliseconds dur); + + void start_flush(); + future<> stop_flush(); +}; + +} diff --git a/include/kafka4seastar/producer/kafka_producer.hh b/include/kafka4seastar/producer/kafka_producer.hh new file mode 100644 index 0000000..c4056c6 --- /dev/null +++ b/include/kafka4seastar/producer/kafka_producer.hh @@ -0,0 +1,56 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include + +#include +#include + +#include +#include +#include +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +class kafka_producer final { + + producer_properties _properties; + connection_manager _connection_manager; + metadata_manager _metadata_manager; + batcher _batcher; + +public: + explicit kafka_producer(producer_properties&& properties); + seastar::future<> init(); + seastar::future<> produce(seastar::sstring topic_name, seastar::sstring key, seastar::sstring value); + seastar::future<> flush(); + seastar::future<> disconnect(); + +}; + +} diff --git a/include/kafka4seastar/producer/producer_properties.hh b/include/kafka4seastar/producer/producer_properties.hh new file mode 100644 index 0000000..c63d79e --- /dev/null +++ b/include/kafka4seastar/producer/producer_properties.hh @@ -0,0 +1,92 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include +#include +#include + +#include +#include + +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +enum class ack_policy { + NONE = 0, + LEADER = 1, + ALL = -1, +}; + +struct enable_idempotence_tag {}; +using enable_idempotence = bool_class; + +class producer_properties final { + +public: + + // Number of acknowledgments from the server to be waited for + // before considering a request complete. + // NONE -> don't wait + // LEADER -> wait for the leader to acknowledge, no guarantee the record has been replicated + // ALL -> wait for all in-sync replicas to acknowledge receiving the record + ack_policy _acks = ack_policy::LEADER; + + // Enabling this ensures that exactly one copy of each message will be written to the stream. + // CURRENTLY NOT IMPLEMENTED + enable_idempotence _enable_idempotence = enable_idempotence::no; + + // number of ms to wait before sending a request, this allows to wait for potential + // batches to form even when there is no load + uint16_t _linger = 0; + // max bytes stored in one batch + uint32_t _buffer_memory = 32 * 1024 * 1024; + // maximum number of retries to be performed before considering the request as failed + uint32_t _retries = 10; + // max number of requests in one batch + uint32_t _batch_size = 16384; + // number of ms after which the connection attempt is considered to have timed out + uint32_t _request_timeout = 500; + // max time in ms after which a new metadata refresh will be sent, even if no changes have been noticed + uint32_t _metadata_refresh = 300000; + + // Identifier of the created producer instance + seastar::sstring _client_id {}; + // a list of host-port pairs to use for establishing the initial connection to the cluster + std::set> _servers {}; + + // Strategy according to which we should choose the target partition, + // based on the given key (or lack thereof) + std::unique_ptr _partitioner = defaults::round_robin_partitioner(); + // Strategy describing how long to wait between consecutive retries, + // based on how many have already been performed + noncopyable_function(uint32_t)> _retry_backoff_strategy = defaults::exp_retry_backoff(20, 1000); + +}; + +} diff --git a/include/kafka4seastar/producer/sender.hh b/include/kafka4seastar/producer/sender.hh new file mode 100644 index 0000000..d93ca29 --- /dev/null +++ b/include/kafka4seastar/producer/sender.hh @@ -0,0 +1,119 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include +#include +#include + +#include + +#include +#include +#include +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +struct send_exception : public std::runtime_error { +public: + explicit send_exception(const seastar::sstring& message) : runtime_error(message) {} +}; + +struct sender_message { + seastar::sstring _key; + seastar::sstring _value; + + std::chrono::time_point _timestamp; + + seastar::sstring _topic; + int32_t _partition_index; + + kafka_error_code_t _error_code; + promise<> _promise; + + sender_message() : + _timestamp(std::chrono::system_clock::now()), + _partition_index(0), + _error_code(error::kafka_error_code::UNKNOWN_SERVER_ERROR) {} + sender_message(sender_message&& s) = default; + sender_message& operator=(sender_message&& s) = default; + sender_message(sender_message& s) = delete; + + size_t size() const noexcept { + return _key.size() + _value.size(); + } +}; + +class sender { +public: + using connection_id = std::pair; + using topic_partition = std::pair; + +private: + connection_manager& _connection_manager; + metadata_manager& _metadata_manager; + std::vector _messages; + + std::map>>> _messages_split_by_broker_topic_partition; + std::map> _messages_split_by_topic_partition; + std::vector>> _responses; + + uint32_t _connection_timeout; + + ack_policy _acks; + + std::optional broker_for_topic_partition(const seastar::sstring& topic, int32_t partition_index); + connection_id broker_for_id(int32_t id); + + void set_error_code_for_broker(const connection_id& broker, const error::kafka_error_code& error_code); + void set_success_for_broker(const connection_id& broker); + void set_error_code_for_topic_partition(const seastar::sstring& topic, int32_t partition_index, + const error::kafka_error_code& error_code); + void set_success_for_topic_partition(const seastar::sstring& topic, int32_t partition_index); + + void split_messages(); + void queue_requests(); + + void set_error_codes_for_responses(std::vector>>& responses); + void filter_messages(); + future<> process_messages_errors(); + +public: + sender(connection_manager& connection_manager, metadata_manager& metadata_manager, + uint32_t connection_timeout, ack_policy acks); + + void move_messages(std::vector& messages); + size_t messages_size() const; + bool messages_empty() const; + + void send_requests(); + future<> receive_responses(); + void close(); +}; + +} diff --git a/include/kafka4seastar/protocol/api_versions_request.hh b/include/kafka4seastar/protocol/api_versions_request.hh new file mode 100644 index 0000000..fa79849 --- /dev/null +++ b/include/kafka4seastar/protocol/api_versions_request.hh @@ -0,0 +1,46 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include + +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +class api_versions_request { +public: + using response_type = api_versions_response; + static constexpr int16_t API_KEY = 18; + static constexpr int16_t MIN_SUPPORTED_VERSION = 0; + static constexpr int16_t MAX_SUPPORTED_VERSION = 2; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +} diff --git a/include/kafka4seastar/protocol/api_versions_response.hh b/include/kafka4seastar/protocol/api_versions_response.hh new file mode 100644 index 0000000..68a609c --- /dev/null +++ b/include/kafka4seastar/protocol/api_versions_response.hh @@ -0,0 +1,79 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +struct unsupported_version_exception : public std::runtime_error { +public: + unsupported_version_exception(const seastar::sstring& message) : runtime_error(message) {} +}; + +class api_versions_response_key { +public: + kafka_int16_t _api_key; + kafka_int16_t _min_version; + kafka_int16_t _max_version; + + bool operator<(const api_versions_response_key& other) const noexcept; + bool operator<(int16_t api_key) const noexcept; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +class api_versions_response { +public: + kafka_error_code_t _error_code; + kafka_array_t _api_keys; + kafka_int32_t _throttle_time_ms; + + template + int16_t max_version() const { + auto broker_versions = (*this)[RequestType::API_KEY]; + if (*broker_versions._api_key == -1) { + throw unsupported_version_exception("Broker does not support specific request"); + } + if (*broker_versions._max_version < RequestType::MIN_SUPPORTED_VERSION) { + throw unsupported_version_exception("Broker is too old"); + } + if (*broker_versions._min_version > RequestType::MAX_SUPPORTED_VERSION) { + throw unsupported_version_exception("Broker is too new"); + } + return std::min(*broker_versions._max_version, RequestType::MAX_SUPPORTED_VERSION); + } + bool contains(int16_t api_key) const; + api_versions_response_key operator[](int16_t api_key) const; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +} diff --git a/include/kafka4seastar/protocol/headers.hh b/include/kafka4seastar/protocol/headers.hh new file mode 100644 index 0000000..92aebf0 --- /dev/null +++ b/include/kafka4seastar/protocol/headers.hh @@ -0,0 +1,52 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include + +using namespace seastar; + +namespace kafka4seastar { + +class request_header { +public: + kafka_int16_t _api_key; + kafka_int16_t _api_version; + kafka_int32_t _correlation_id; + kafka_nullable_string_t _client_id; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +class response_header { +public: + kafka_int32_t _correlation_id; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +} diff --git a/include/kafka4seastar/protocol/kafka_error_code.hh b/include/kafka4seastar/protocol/kafka_error_code.hh new file mode 100644 index 0000000..143a39b --- /dev/null +++ b/include/kafka4seastar/protocol/kafka_error_code.hh @@ -0,0 +1,152 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +namespace error { + +struct is_retriable_tag {}; +using is_retriable = bool_class; + +struct invalidates_metadata_tag {}; +using invalidates_metadata = bool_class; + +class kafka_error_code { + +public: + + int16_t _error_code; + seastar::sstring _error_message; + is_retriable _is_retriable; + invalidates_metadata _invalidates_metadata; + + kafka_error_code( + int16_t error_code, + seastar::sstring error_message, + is_retriable is_retriable, + invalidates_metadata is_invalid_metadata); + + static const kafka_error_code& get_error(int16_t value); + + static const kafka_error_code UNKNOWN_SERVER_ERROR; + static const kafka_error_code NONE; + static const kafka_error_code OFFSET_OUT_OF_RANGE; + static const kafka_error_code CORRUPT_MESSAGE; + static const kafka_error_code UNKNOWN_TOPIC_OR_PARTITION; + static const kafka_error_code INVALID_FETCH_SIZE; + static const kafka_error_code LEADER_NOT_AVAILABLE; + static const kafka_error_code NOT_LEADER_FOR_PARTITION; + static const kafka_error_code REQUEST_TIMED_OUT; + static const kafka_error_code BROKER_NOT_AVAILABLE; + static const kafka_error_code REPLICA_NOT_AVAILABLE; + static const kafka_error_code MESSAGE_TOO_LARGE; + static const kafka_error_code STALE_CONTROLLER_EPOCH; + static const kafka_error_code OFFSET_METADATA_TOO_LARGE; + static const kafka_error_code NETWORK_EXCEPTION; + static const kafka_error_code COORDINATOR_LOAD_IN_PROGRESS; + static const kafka_error_code COORDINATOR_NOT_AVAILABLE; + static const kafka_error_code NOT_COORDINATOR; + static const kafka_error_code INVALID_TOPIC_EXCEPTION; + static const kafka_error_code RECORD_LIST__TOO_LARGE; + static const kafka_error_code NOT_ENOUGH_REPLICAS; + static const kafka_error_code NOT_ENOUGH_REPLICAS_AFTER_APPEND; + static const kafka_error_code INVALID_REQUIRED_ACKS; + static const kafka_error_code ILLEGAL_GENERATION; + static const kafka_error_code INCONSISTENT_PROTOCOL; + static const kafka_error_code INVALID_GROUP_ID; + static const kafka_error_code UNKNOWN_MEMBER_ID; + static const kafka_error_code INVALID_SESSION_TIMEOUT; + static const kafka_error_code REBALANCE_IN_PROGRESS; + static const kafka_error_code INVALID_COMMIT_OFFSET_SIZE; + static const kafka_error_code TOPIC_AUTHORIZATION_FAILED; + static const kafka_error_code GROUP_AUTHORIZATION_FAILED; + static const kafka_error_code CLUSTER_AUTHORIZATION_FAILED; + static const kafka_error_code INVALID_TIMESTAMP; + static const kafka_error_code UNSUPPORTED_SASL_MECHANISM; + static const kafka_error_code ILLEGAL_SASL_STATE; + static const kafka_error_code UNSUPPORTED_VERSION; + static const kafka_error_code TOPIC_ALREADY_EXISTS; + static const kafka_error_code INVALID_PARTITIONS; + static const kafka_error_code INVALID_REPLICATION_FACTOR; + static const kafka_error_code INVALID_REPLICA_ASSIGNMENT; + static const kafka_error_code INVALID_CONFIG; + static const kafka_error_code NOT_CONTROLLER; + static const kafka_error_code INVALID_REQUEST; + static const kafka_error_code UNSUPPORTED_FOR_MESSAGE_FORMAT; + static const kafka_error_code POLICY_VIOLATION; + static const kafka_error_code OUT_OF_ORDER_SEQUENCE_NUMBER; + static const kafka_error_code DUPLICATE_SEQUENCE_NUMBER; + static const kafka_error_code INVALID_PRODUCER_EPOCH; + static const kafka_error_code INVALID_TXN_STATE; + static const kafka_error_code INVALID_PRODUCER_ID_MAPPING; + static const kafka_error_code INVALID_TRANSACTION_TIMEOUT; + static const kafka_error_code CONCURRENT_TRANSACTIONS; + static const kafka_error_code TRANSACTION_COORDINATOR_FENCED; + static const kafka_error_code TRANSACTIONAL_ID_AUTHORIZATION_FAILED; + static const kafka_error_code SECURITY_DISABLED; + static const kafka_error_code OPERATION_NOT_ATTEMPTED; + static const kafka_error_code KAFKA_STORAGE_ERROR; + static const kafka_error_code LOG_DIR_NOT_FOUND; + static const kafka_error_code SASL_AUTHENTICATION_FAILED; + static const kafka_error_code UNKNOWN_PRODUCER_ID; + static const kafka_error_code REASSIGNMENT_IN_PROGRESS; + static const kafka_error_code DELEGATION_TOKEN_AUTH_DISABLED; + static const kafka_error_code DELEGATION_TOKEN_NOT_FOUND; + static const kafka_error_code DELEGATION_TOKEN_OWNER_MISMATCH; + static const kafka_error_code DELEGATION_TOKEN_REQUEST_NOT_ALLOWED; + static const kafka_error_code DELEGATION_TOKEN_AUTHORIZATION_FAILED; + static const kafka_error_code DELEGATION_TOKEN_EXPIRED; + static const kafka_error_code INVALID_PRINCIPAL_TYPE; + static const kafka_error_code NON_EMPTY_GROUP; + static const kafka_error_code GROUP_ID_NOT_FOUND; + static const kafka_error_code FETCH_SESSION_ID_NOT_FOUND; + static const kafka_error_code INVALID_FETCH_SESSION_EPOCH; + static const kafka_error_code LISTENER_NOT_FOUND; + static const kafka_error_code TOPIC_DELETION_DISABLED; + static const kafka_error_code FENCED_LEADER_EPOCH; + static const kafka_error_code UNKNOWN_LEADER_EPOCH; + static const kafka_error_code UNSUPPORTED_COMPRESSION_TYPE; + static const kafka_error_code STALE_BROKER_EPOCH; + static const kafka_error_code OFFSET_NOT_AVAILABLE; + static const kafka_error_code MEMBER_ID_REQUIRED; + static const kafka_error_code PREFERRED_LEADER_NOT_AVAILABLE; + static const kafka_error_code GROUP_MAX_SIZE_REACHED; + static const kafka_error_code FENCED_INSTANCE_ID; + static const kafka_error_code ELIGIBLE_LEADERS_NOT_AVAILABLE; + static const kafka_error_code ELECTION_NOT_NEEDED; + static const kafka_error_code NO_REASSIGNMENT_IN_PROGRESS; + static const kafka_error_code GROUP_SUBSCRIBED_TO_TOPIC; + static const kafka_error_code INVALID_RECORD; +}; + +} + +} diff --git a/include/kafka4seastar/protocol/kafka_primitives.hh b/include/kafka4seastar/protocol/kafka_primitives.hh new file mode 100644 index 0000000..43f9f12 --- /dev/null +++ b/include/kafka4seastar/protocol/kafka_primitives.hh @@ -0,0 +1,486 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +template +class kafka_number_t { +private: + NumberType _value; + static constexpr auto NUMBER_SIZE = sizeof(NumberType); + +public: + kafka_number_t() noexcept : kafka_number_t(0) {} + + explicit kafka_number_t(NumberType value) noexcept : _value(value) {} + + [[nodiscard]] const NumberType& operator*() const noexcept { return _value; } + + [[nodiscard]] NumberType& operator*() noexcept { return _value; } + + kafka_number_t& operator=(NumberType value) noexcept { + _value = value; + return *this; + } + + void serialize(kafka::output_stream& os, int16_t api_version) const { + std::array buffer{}; + auto value = net::hton(_value); + auto value_pointer = reinterpret_cast(&value); + std::copy(value_pointer, value_pointer + NUMBER_SIZE, buffer.begin()); + + os.write(buffer.data(), NUMBER_SIZE); + } + + void deserialize(kafka::input_stream& is, int16_t api_version) { + std::array buffer{}; + is.read(buffer.data(), NUMBER_SIZE); + if (is.gcount() != NUMBER_SIZE) { + throw parsing_exception("Stream ended prematurely when reading number"); + } + _value = net::ntoh(*reinterpret_cast(buffer.data())); + } + + [[nodiscard]] size_t serialized_length() const noexcept { + return NUMBER_SIZE; + } +}; + +class kafka_error_code_t { +private: + int16_t _value; + static constexpr auto NUMBER_SIZE = sizeof(int16_t); + +public: + kafka_error_code_t() noexcept : _value(0) {} + kafka_error_code_t(const error::kafka_error_code& error) noexcept : _value(error._error_code) {} + + [[nodiscard]] const error::kafka_error_code& operator*() const noexcept { + return error::kafka_error_code::get_error(_value); + } + + [[nodiscard]] const error::kafka_error_code* operator->() const noexcept { + return &error::kafka_error_code::get_error(_value); + } + + kafka_error_code_t& operator=(const error::kafka_error_code& error) noexcept { + _value = error._error_code; + return *this; + } + + void serialize(kafka::output_stream& os, int16_t api_version) const { + std::array buffer{}; + auto value = net::hton(_value); + auto value_pointer = reinterpret_cast(&value); + std::copy(value_pointer, value_pointer + NUMBER_SIZE, buffer.begin()); + + os.write(buffer.data(), NUMBER_SIZE); + } + + void deserialize(kafka::input_stream& is, int16_t api_version) { + std::array buffer{}; + is.read(buffer.data(), NUMBER_SIZE); + if (is.gcount() != NUMBER_SIZE) { + throw parsing_exception("Stream ended prematurely when reading number"); + } + _value = net::ntoh(*reinterpret_cast(buffer.data())); +/* try { + error::kafka_error_code::get_error(_value); + } catch (const std::out_of_range& e) { + throw parsing_exception("Error with such code does not exist"); + }*/ + } + + bool operator==(const error::kafka_error_code& other) const { + return other._error_code == this->_value; + } + + bool operator!=(const error::kafka_error_code& other) const { + return ! (*this == other); + } + + [[nodiscard]] size_t serialized_length() const noexcept { + return NUMBER_SIZE; + } +}; + +using kafka_int8_t = kafka_number_t; +using kafka_int16_t = kafka_number_t; +using kafka_int32_t = kafka_number_t; +using kafka_int64_t = kafka_number_t; +using kafka_uint32_t = kafka_number_t; +using kafka_bool_t = kafka_number_t; + +class kafka_varint_t { +private: + int32_t _value; +public: + kafka_varint_t() noexcept : kafka_varint_t(0) {} + + explicit kafka_varint_t(int32_t value) noexcept : _value(value) {} + + [[nodiscard]] const int32_t& operator*() const noexcept { return _value; } + + [[nodiscard]] int32_t& operator*() noexcept { return _value; } + + kafka_varint_t& operator=(int32_t value) noexcept { + _value = value; + return *this; + } + + void serialize(kafka::output_stream& os, int16_t api_version) const { + auto current_value = (static_cast(_value) << 1) ^ static_cast(_value >> 31); + do { + uint8_t current_byte = current_value & 0x7F; + current_value >>= 7; + if (current_value != 0) { + current_byte |= 0x80; + } + os.write(reinterpret_cast(¤t_byte), 1); + } while (current_value != 0); + } + + [[nodiscard]] size_t serialized_length() const noexcept { + size_t result = 0; + auto current_value = (static_cast(_value) << 1) ^ static_cast(_value >> 31); + do { + current_value >>= 7; + result++; + } while (current_value != 0); + return result; + } + + void deserialize(kafka::input_stream& is, int16_t api_version) { + uint32_t current_value = 0; + int32_t current_offset = 0; + char current_byte = 0; + do { + is.read(¤t_byte, 1); + if (is.gcount() != 1) { + throw parsing_exception("Stream ended prematurely when reading varint"); + } + if (current_byte == 0) { + break; + } + auto max_bit_write = current_offset + 32 - __builtin_clz(static_cast(current_byte)); + if (max_bit_write > 32) { + throw parsing_exception("Deserialized varint is larger than 32 bits"); + } + current_value |= static_cast(current_byte & 0x7F) << current_offset; + current_offset += 7; + } while (current_byte & 0x80); + current_value = (current_value >> 1) ^ -(current_value & 1); + _value = current_value; + } +}; + +template +class kafka_buffer_t { +private: + seastar::sstring _value; +public: + kafka_buffer_t() noexcept = default; + + explicit kafka_buffer_t(seastar::sstring value) : _value(std::move(value)) {} + + [[nodiscard]] const seastar::sstring& operator*() const noexcept { return _value; } + + [[nodiscard]] seastar::sstring& operator*() noexcept { return _value; } + + [[nodiscard]] const seastar::sstring* operator->() const noexcept { return &_value; } + + [[nodiscard]] seastar::sstring* operator->() noexcept { return &_value; } + + kafka_buffer_t& operator=(const seastar::sstring& value) { + _value = value; + return *this; + } + + kafka_buffer_t& operator=(seastar::sstring&& value) noexcept { + _value = std::move(value); + return *this; + } + + void serialize(kafka::output_stream& os, int16_t api_version) const { + SizeType length(_value.size()); + length.serialize(os, api_version); + + os.write(_value.data(), _value.size()); + } + + [[nodiscard]] size_t serialized_length() const noexcept { + size_t result = 0; + SizeType length(_value.size()); + result += length.serialized_length(); + result += _value.size(); + return result; + } + + void deserialize(kafka::input_stream& is, int16_t api_version) { + SizeType length; + length.deserialize(is, api_version); + // TODO: Max length check + if (*length < 0) { + throw parsing_exception("Length of buffer is negative"); + } + + seastar::sstring value; + value.resize(*length); + is.read(value.data(), *length); + + if (is.gcount() != *length) { + throw parsing_exception("Stream ended prematurely when reading buffer"); + } + _value.swap(value); + } +}; + +template +class kafka_nullable_buffer_t { +private: + seastar::sstring _value; + bool _is_null; +public: + kafka_nullable_buffer_t() noexcept : _is_null(true) {} + + explicit kafka_nullable_buffer_t(seastar::sstring value) : _value(std::move(value)), _is_null(false) {} + + [[nodiscard]] bool is_null() const noexcept { return _is_null; } + + void set_null() noexcept { + _value.reset(); + _is_null = true; + } + + [[nodiscard]] const seastar::sstring& operator*() const { + if (_is_null) { + throw std::domain_error("Object is null."); + } + return _value; + } + + [[nodiscard]] seastar::sstring& operator*() { + if (_is_null) { + throw std::domain_error("Object is null."); + } + return _value; + } + + [[nodiscard]] const seastar::sstring* operator->() const { + if (_is_null) { + throw std::domain_error("Object is null."); + } + return &_value; + } + + [[nodiscard]] seastar::sstring* operator->() { + if (_is_null) { + throw std::domain_error("Object is null."); + } + return &_value; + } + + kafka_nullable_buffer_t& operator=(const seastar::sstring& value) { + _value = value; + _is_null = false; + return *this; + } + + kafka_nullable_buffer_t& operator=(seastar::sstring&& value) noexcept { + _value = std::move(value); + _is_null = false; + return *this; + } + + void serialize(kafka::output_stream& os, int16_t api_version) const { + if (_is_null) { + SizeType null_indicator(-1); + null_indicator.serialize(os, api_version); + } else { + SizeType length(_value.size()); + length.serialize(os, api_version); + os.write(_value.data(), _value.size()); + } + } + + + [[nodiscard]] size_t serialized_length() const noexcept { + size_t result = 0; + SizeType length(_is_null ? -1 : _value.size()); + result += length.serialized_length(); + result += (is_null ? 0 :_value.size()); + return result; + } + + + void deserialize(kafka::input_stream& is, int16_t api_version) { + SizeType length; + length.deserialize(is, api_version); + if (*length >= 0) { + seastar::sstring value; + // TODO: Max length check + value.resize(*length); + is.read(value.data(), *length); + + if (is.gcount() != *length) { + throw parsing_exception("Stream ended prematurely when reading nullable buffer"); + } + _value.swap(value); + _is_null = false; + } else if (*length == -1) { + set_null(); + } else { + throw parsing_exception("Length of buffer is invalid"); + } + } +}; + +using kafka_string_t = kafka_buffer_t; +using kafka_nullable_string_t = kafka_nullable_buffer_t; + +using kafka_bytes_t = kafka_buffer_t; +using kafka_nullable_bytes_t = kafka_nullable_buffer_t; + +template +class kafka_array_t { +public: + using container_type = std::vector; +private: + container_type _elems; + bool _is_null; +public: + kafka_array_t() noexcept : _is_null(true) {} + + explicit kafka_array_t(container_type elems) noexcept + : _elems(std::move(elems)), _is_null(false) {} + + [[nodiscard]] bool is_null() const noexcept { return _is_null; } + + [[nodiscard]] ElementType& operator[](size_t i) { + if (_is_null) { + throw std::domain_error("Object is null."); + } + return _elems[i]; + } + + [[nodiscard]] const ElementType& operator[](size_t i) const { + if (_is_null) { + throw std::domain_error("Object is null."); + } + return _elems[i]; + } + + [[nodiscard]] const container_type& operator*() const { + if (_is_null) { + throw std::domain_error("Object is null."); + } + return _elems; + } + + [[nodiscard]] container_type& operator*() { + if (_is_null) { + throw std::domain_error("Object is null."); + } + return _elems; + } + + [[nodiscard]] const container_type* operator->() const { + if (_is_null) { + throw std::domain_error("Object is null."); + } + return &_elems; + } + + [[nodiscard]] container_type* operator->() { + if (_is_null) { + throw std::domain_error("Object is null."); + } + return &_elems; + } + + kafka_array_t& operator=(const container_type& elems) { + _elems = elems; + _is_null = false; + return *this; + } + + kafka_array_t& operator=(container_type&& elems) noexcept { + _elems = std::move(elems); + _is_null = false; + return *this; + } + + void set_null() noexcept { + _elems.clear(); + _is_null = true; + } + + void serialize(kafka::output_stream& os, int16_t api_version) const { + if (_is_null) { + ElementCountType null_indicator(-1); + null_indicator.serialize(os, api_version); + } else { + ElementCountType length(_elems.size()); + length.serialize(os, api_version); + for (const auto& elem : _elems) { + elem.serialize(os, api_version); + } + } + } + + void deserialize(kafka::input_stream& is, int16_t api_version) { + ElementCountType length; + length.deserialize(is, api_version); + if (*length >= 0) { + // TODO: Max length check + container_type elems(*length); + for (int32_t i = 0; i < *length; i++) { + elems[i].deserialize(is, api_version); + } + _elems.swap(elems); + _is_null = false; + } else if (*length == -1) { + set_null(); + } else { + throw parsing_exception("Length of array is invalid"); + } + } +}; + +} diff --git a/include/kafka4seastar/protocol/kafka_records.hh b/include/kafka4seastar/protocol/kafka_records.hh new file mode 100644 index 0000000..7c0375a --- /dev/null +++ b/include/kafka4seastar/protocol/kafka_records.hh @@ -0,0 +1,97 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include + +#include +using namespace seastar; + +namespace kafka4seastar { + +class kafka_record_header { +public: + seastar::sstring _header_key; + seastar::sstring _value; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); + + [[nodiscard]] size_t serialized_length() const noexcept; +}; + +class kafka_record { +public: + kafka_varint_t _timestamp_delta; + kafka_varint_t _offset_delta; + seastar::sstring _key; + seastar::sstring _value; + std::vector _headers; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +enum class kafka_record_compression_type { + NO_COMPRESSION = 0, GZIP = 1, SNAPPY = 2, LZ4 = 3, ZSTD = 4 +}; + +enum class kafka_record_timestamp_type { + CREATE_TIME = 0, LOG_APPEND_TIME = 1 +}; + +class kafka_record_batch { +public: + kafka_int64_t _base_offset; + kafka_int32_t _partition_leader_epoch; + kafka_int8_t _magic; + + kafka_record_compression_type _compression_type; + kafka_record_timestamp_type _timestamp_type; + bool _is_transactional; + bool _is_control_batch; + + kafka_int64_t _first_timestamp; + kafka_int64_t _producer_id; + kafka_int16_t _producer_epoch; + kafka_int32_t _base_sequence; + + std::vector _records; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +class kafka_records { +public: + std::vector _record_batches; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +} diff --git a/include/kafka4seastar/protocol/metadata_request.hh b/include/kafka4seastar/protocol/metadata_request.hh new file mode 100644 index 0000000..af23cf9 --- /dev/null +++ b/include/kafka4seastar/protocol/metadata_request.hh @@ -0,0 +1,58 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +class metadata_request_topic { +public: + kafka_string_t _name; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +class metadata_request { +public: + using response_type = metadata_response; + static constexpr int16_t API_KEY = 3; + static constexpr int16_t MIN_SUPPORTED_VERSION = 1; // Kafka 0.10.0.0 + static constexpr int16_t MAX_SUPPORTED_VERSION = 8; + + kafka_array_t _topics; + kafka_bool_t _allow_auto_topic_creation; + kafka_bool_t _include_cluster_authorized_operations; + kafka_bool_t _include_topic_authorized_operations; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +} diff --git a/include/kafka4seastar/protocol/metadata_response.hh b/include/kafka4seastar/protocol/metadata_response.hh new file mode 100644 index 0000000..139ebbd --- /dev/null +++ b/include/kafka4seastar/protocol/metadata_response.hh @@ -0,0 +1,86 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include + +using namespace seastar; + +namespace kafka4seastar { + +class metadata_response_broker { +public: + kafka_int32_t _node_id; + kafka_string_t _host; + kafka_int32_t _port; + kafka_nullable_string_t _rack; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +class metadata_response_partition { +public: + kafka_error_code_t _error_code; + kafka_int32_t _partition_index; + kafka_int32_t _leader_id; + kafka_int32_t _leader_epoch; + kafka_array_t _replica_nodes; + kafka_array_t _isr_nodes; + kafka_array_t _offline_replicas; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +class metadata_response_topic { +public: + kafka_error_code_t _error_code; + kafka_string_t _name; + kafka_bool_t _is_internal; + kafka_array_t _partitions; + kafka_int32_t _topic_authorized_operations; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +class metadata_response { +public: + kafka_int32_t _throttle_time_ms; + kafka_array_t _brokers; + kafka_nullable_string_t _cluster_id; + kafka_int32_t _controller_id; + kafka_array_t _topics; + kafka_int32_t _cluster_authorized_operations; + kafka_error_code_t _error_code; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +} diff --git a/include/kafka4seastar/protocol/produce_request.hh b/include/kafka4seastar/protocol/produce_request.hh new file mode 100644 index 0000000..d32939a --- /dev/null +++ b/include/kafka4seastar/protocol/produce_request.hh @@ -0,0 +1,70 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +class produce_request_partition_produce_data { +public: + kafka_int32_t _partition_index; + kafka_records _records; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +class produce_request_topic_produce_data { +public: + kafka_string_t _name; + kafka_array_t _partitions; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +class produce_request { +public: + using response_type = produce_response; + static constexpr int16_t API_KEY = 0; + static constexpr int16_t MIN_SUPPORTED_VERSION = 2; // Kafka 0.10.0.0 + static constexpr int16_t MAX_SUPPORTED_VERSION = 8; + + kafka_nullable_string_t _transactional_id; + kafka_int16_t _acks; + kafka_int32_t _timeout_ms; + kafka_array_t _topics; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +} diff --git a/include/kafka4seastar/protocol/produce_response.hh b/include/kafka4seastar/protocol/produce_response.hh new file mode 100644 index 0000000..2af7ea5 --- /dev/null +++ b/include/kafka4seastar/protocol/produce_response.hh @@ -0,0 +1,81 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include + +using namespace seastar; + +namespace kafka4seastar { + +class produce_response_batch_index_and_error_message { +private: + kafka_int32_t _batch_index; + kafka_nullable_string_t _batch_index_error_message; +public: + [[nodiscard]] const kafka_int32_t& get_batch_index() const; + + [[nodiscard]] const kafka_nullable_string_t& get_batch_index_error_message() const; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +class produce_response_partition_produce_response { +public: + kafka_int32_t _partition_index; + kafka_error_code_t _error_code; + kafka_int64_t _base_offset; + kafka_int64_t _log_append_time_ms; + kafka_int64_t _log_start_offset; + kafka_array_t _record_errors; + kafka_nullable_string_t _error_message; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +class produce_response_topic_produce_response { +public: + kafka_string_t _name; + kafka_array_t _partitions; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +class produce_response { +public: + kafka_array_t _responses; + kafka_int32_t _throttle_time_ms; + kafka_error_code_t _error_code; + + void serialize(kafka::output_stream& os, int16_t api_version) const; + + void deserialize(kafka::input_stream& is, int16_t api_version); +}; + +} diff --git a/include/kafka4seastar/protocol/streams.hh b/include/kafka4seastar/protocol/streams.hh new file mode 100644 index 0000000..7233e41 --- /dev/null +++ b/include/kafka4seastar/protocol/streams.hh @@ -0,0 +1,186 @@ +#ifndef __STREAMS_HH__ +#define __STREAMS_HH__ + +#include +#include +#include +#include +#include + +using std::vector; + +namespace kafka4seastar { + struct parsing_exception : public std::runtime_error { + public: + parsing_exception(const std::string& message) : runtime_error(message) {} + }; + + namespace kafka { + + class input_stream { + private: + const char *_data; + int32_t _length; + int32_t _current_position = 0; + int32_t _gcount = 0; + + public: + input_stream(const char *data, int32_t length) noexcept + : _data(data) + , _length(length) + {} + + inline void read(char *destination, int32_t length) { + if(_current_position + length > _length){ + throw kafka4seastar::parsing_exception ("Attempted to read more than the remaining length of the input stream."); + } + std::copy((_data + _current_position), (_data + _current_position + length), destination); + _current_position += length; + _gcount = length; + } + + inline int32_t gcount() const { + return _gcount; + } + + inline const char *get() const { + if(_current_position == _length) { + throw kafka4seastar::parsing_exception("Input stream is exhausted."); + } + return _data + _current_position; + } + + inline int32_t get_position() const noexcept { + return _current_position; + } + + inline void set_position(int32_t new_position) { + if(new_position >= _length || new_position < 0) { + throw kafka4seastar::parsing_exception("Attempted to set input stream's position outside its data."); + } + else { + _current_position = new_position; + } + } + + inline void move_position(int32_t delta) { + set_position(_current_position + delta); + } + + int32_t size() const noexcept { + return _length; + } + + inline const char * begin() const { + return _data; + } + }; + + class output_stream { + private: + char* _data; + int _size; + int _capacity; + + int32_t _current_position = 0; + bool _is_resizable; + + output_stream(bool is_resizable, int32_t size) noexcept + :_is_resizable(is_resizable), + _capacity(size), + _data((char*)malloc(size+5)), + _size(size) + {} + + public: + + ~output_stream() { free(_data); } + + static output_stream fixed_size_stream(int32_t size) { + return output_stream(false, size); + } + + static output_stream resizable_stream() { + return output_stream(true, 0); + } + + inline void write(const char* source, int32_t length) { + if (length + _current_position > _size) { + if (_is_resizable) { + if (length + _current_position > _capacity) { + _capacity = std::max(_capacity * 2, length + _current_position); + _data = (char*)realloc(_data, _capacity); } + _size = length + _current_position ; + } else { + throw kafka4seastar::parsing_exception("This output stream won't that many bytes"); + } + } + memcpy(_data + _current_position, source, length); + _current_position += length; + } + + + inline char *get() { + if(_current_position == _size) { + throw kafka4seastar::parsing_exception("Output stream is full."); + } + return _data + _current_position; + } + + inline const char *get() const { + if(_current_position == static_cast(_size)) { + throw kafka4seastar::parsing_exception("Output stream is full."); + } + return _data + _current_position; + } + + inline const char * begin() const { + return _data; + } + + inline int32_t get_position() const noexcept { + return _current_position; + } + + inline void set_position(int32_t new_position) { + if(new_position < 0) { + throw kafka4seastar::parsing_exception("Cannot set position to negative value."); + } + + if(_is_resizable) { + if (new_position >= _size) { + if (new_position + 1 > _capacity) { + _capacity = std::max(_capacity * 2, new_position + 1); + _data = (char*)realloc(_data, _capacity);} + _size = new_position + 1; + + } + } + else { + if(new_position >= static_cast(_size)) { + kafka4seastar::parsing_exception("Attempted to set fixed output stream's position past its data."); + } + } + + _current_position = new_position; + } + + inline void move_position(int32_t delta) { + set_position(_current_position + delta); + } + + int32_t size() const noexcept { + return _size; + } + + void reset() noexcept { + _current_position = 0; + _size = 0; + } + }; + +} // namespace kafka + +} // namespace seastar + +#endif // __STREAMS_HH__ diff --git a/include/kafka4seastar/utils/defaults.hh b/include/kafka4seastar/utils/defaults.hh new file mode 100644 index 0000000..b2649f0 --- /dev/null +++ b/include/kafka4seastar/utils/defaults.hh @@ -0,0 +1,46 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include + +#include +#include + +#include + +using namespace seastar; + +namespace kafka4seastar { + +namespace defaults { + +noncopyable_function(uint32_t)> exp_retry_backoff(uint32_t base_ms, uint32_t max_backoff_ms); + +std::unique_ptr round_robin_partitioner(); +std::unique_ptr random_partitioner(); + +} + +} diff --git a/include/kafka4seastar/utils/metadata_manager.hh b/include/kafka4seastar/utils/metadata_manager.hh new file mode 100644 index 0000000..d8db7e4 --- /dev/null +++ b/include/kafka4seastar/utils/metadata_manager.hh @@ -0,0 +1,85 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include + +#include +#include +#include +#include + +using namespace seastar; + +namespace std { + + template <> + struct hash> + { + std::size_t operator()(const std::pair& k) const + { + using std::size_t; + using std::hash; + using std::string; + return hash()(k.first) ^ hash()(k.second); + } + }; + +} + +namespace kafka4seastar { + +class metadata_manager { +public: + using broker_id = std::pair; + +private: + connection_manager& _connection_manager; + metadata_response _metadata; + bool _keep_refreshing = false; + semaphore _refresh_finished = 0; + abort_source _stop_refresh; + uint32_t _expiration_time; + + std::unordered_map _brokers; + + void parse_new_metadata(); + + seastar::future<> refresh_coroutine(std::chrono::milliseconds dur); + +public: + explicit metadata_manager(connection_manager& manager, uint32_t expiration_time) + : _connection_manager(manager), _expiration_time(expiration_time) {} + + seastar::future<> refresh_metadata(); + void start_refresh(); + future<> stop_refresh(); + // Capturing resulting metadata response object is forbidden, + // it can be destroyed any time. + metadata_response& get_metadata(); + + std::optional get_broker(broker_id& id); +}; + +} diff --git a/include/kafka4seastar/utils/partitioner.hh b/include/kafka4seastar/utils/partitioner.hh new file mode 100644 index 0000000..1b1145e --- /dev/null +++ b/include/kafka4seastar/utils/partitioner.hh @@ -0,0 +1,50 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +class partitioner { +public: + virtual const metadata_response_partition& get_partition(const seastar::sstring& key, const kafka_array_t& partitions) = 0; + virtual ~partitioner() = default; +}; + +class basic_partitioner : public partitioner { +public: + const metadata_response_partition& get_partition(const seastar::sstring& key, const kafka_array_t& partitions) override; +}; + +class rr_partitioner : public partitioner { +public: + const metadata_response_partition& get_partition(const seastar::sstring& key, const kafka_array_t& partitions) override; +private: + uint32_t counter = 0; +}; + +} diff --git a/include/kafka4seastar/utils/retry_helper.hh b/include/kafka4seastar/utils/retry_helper.hh new file mode 100644 index 0000000..1023c8f --- /dev/null +++ b/include/kafka4seastar/utils/retry_helper.hh @@ -0,0 +1,72 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include +#include + +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +struct do_retry_tag { }; +using do_retry = bool_class; + +class retry_helper { +private: + uint32_t _max_retry_count; + + noncopyable_function(uint32_t)> _backoff; + + template + future<> with_retry(AsyncAction&& action, uint32_t retry_number) { + if (retry_number >= _max_retry_count) { + return make_ready_future<>(); + } + return _backoff(retry_number) + .then([this, action = std::forward(action), retry_number]() mutable { + return futurize_apply(action) + .then([this, action = std::forward(action), retry_number](bool_class do_retry_val) mutable { + if (do_retry_val == do_retry::yes) { + return with_retry(std::forward(action), retry_number + 1); + } else { + return make_ready_future<>(); + } + }); + }); + } + +public: + retry_helper(uint32_t max_retry_count, noncopyable_function(uint32_t)> backoff) + : _max_retry_count(max_retry_count), _backoff(std::move(backoff)) {} + + template + future<> with_retry(AsyncAction&& action) { + return with_retry(std::forward(action), 0); + } +}; + +} diff --git a/src/connection/connection_manager.cc b/src/connection/connection_manager.cc new file mode 100644 index 0000000..6604132 --- /dev/null +++ b/src/connection/connection_manager.cc @@ -0,0 +1,107 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include +#include +#include + +#include + +using namespace seastar; + +namespace kafka4seastar { + +future connection_manager::connect(const seastar::sstring& host, uint16_t port, uint32_t timeout) { + auto conn = _connections.find({host, port}); + return conn != _connections.end() + ? make_ready_future(conn) + : kafka_connection::connect(host, port, _client_id, timeout) + .then([this, host, port] (std::unique_ptr conn) { + return make_ready_future(_connections.emplace(std::make_pair<>(host, port), std::move(conn)).first); + }); +} + +future<> connection_manager::init(const std::set& servers, uint32_t request_timeout) { + std::vector> fs; + + fs.reserve(servers.size()); + + for (auto& server : servers) { + fs.push_back(connect(server.first, server.second, request_timeout).discard_result()); + } + + return when_all_succeed(fs.begin(), fs.end()).discard_result(); +} + +connection_manager::connection_iterator connection_manager::get_connection(const connection_id& connection) { + return _connections.find(connection); +} + +future<> connection_manager::disconnect(const connection_id& connection) { + auto conn = _connections.find(connection); + if (conn != _connections.end()) { + auto conn_ptr = std::move(conn->second); + _connections.erase(conn); + auto f = conn_ptr->close(); + return f.finally([conn_ptr = std::move(conn_ptr)]{}); + } + return make_ready_future(); +} + +future connection_manager::ask_for_metadata(metadata_request&& request) { + auto conn_id = std::optional(); + return seastar::do_with(metadata_response(), [this, request = std::move(request), conn_id = std::move(conn_id)] (metadata_response& metadata) mutable { + return seastar::repeat([this, request = std::move(request), conn_id = std::move(conn_id), &metadata] () mutable { + auto it = !conn_id ? _connections.begin() : _connections.upper_bound(*conn_id); + if (it == _connections.end()) { + throw metadata_refresh_exception("No brokers responded."); + } + conn_id = it->first; + return it->second->send(request).then([this, &metadata](metadata_response res) mutable { + if (res._error_code == error::kafka_error_code::NONE) { + metadata = std::move(res); + return seastar::stop_iteration::yes; + } + else { + return seastar::stop_iteration::no; + } + }); + }).then([&metadata] () mutable { + return std::move(metadata); + }); + }); + +} + +future<> connection_manager::disconnect_all() { + while (_connections.begin() != _connections.end()) { + auto it = _connections.begin(); + auto fut = disconnect({it->first.first, it->first.second}); + _pending_queue = _pending_queue.then([fut=std::move(fut)]() mutable { + return std::move(fut); + }); + } + + return _pending_queue.discard_result(); +} + +} diff --git a/src/connection/kafka_connection.cc b/src/connection/kafka_connection.cc new file mode 100644 index 0000000..d212411 --- /dev/null +++ b/src/connection/kafka_connection.cc @@ -0,0 +1,54 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include + +using namespace seastar; + +namespace kafka4seastar { + +future> kafka_connection::connect(const seastar::sstring& host, uint16_t port, + const seastar::sstring& client_id, uint32_t timeout_ms) { + return tcp_connection::connect(host, port, timeout_ms) + .then([client_id] (tcp_connection connection) { + return std::make_unique(std::move(connection), client_id); + }).then([] (std::unique_ptr connection) { + auto f = connection->init(); + return f.then([connection = std::move(connection)] () mutable { + return std::move(connection); + }); + }); +} + +future<> kafka_connection::init() { + api_versions_request request; + return send(request, api_versions_request::MAX_SUPPORTED_VERSION) + .then([this](api_versions_response response) { + _api_versions = response; + }); +} + +future<> kafka_connection::close() { + return _connection.close(); +} + +} diff --git a/src/connection/tcp_connection.cc b/src/connection/tcp_connection.cc new file mode 100644 index 0000000..10b5663 --- /dev/null +++ b/src/connection/tcp_connection.cc @@ -0,0 +1,76 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +static auto timeout_end(uint32_t timeout_ms) { + return std::chrono::steady_clock::now() + std::chrono::milliseconds(timeout_ms); +} + +future tcp_connection::connect(const seastar::sstring& host, uint16_t port, + uint32_t timeout_ms) { + net::inet_address target_host = net::inet_address{host}; + sa_family_t family = target_host.is_ipv4() ? sa_family_t(AF_INET) : sa_family_t(AF_INET6); + socket_address socket = socket_address(::sockaddr_in{family, INADDR_ANY, {0}}); + auto f = target_host.is_ipv4() + ? engine().net().connect(ipv4_addr{target_host, port}, socket, transport::TCP) + : engine().net().connect(ipv6_addr{target_host, port}, socket, transport::TCP); + auto f_timeout = seastar::with_timeout(timeout_end(timeout_ms), std::move(f)); + return f_timeout.then([target_host = std::move(target_host), timeout_ms, port] (connected_socket fd) { + return tcp_connection(target_host, port, timeout_ms, std::move(fd)); + } + ); +} + +future> tcp_connection::read(size_t bytes_to_read) { + auto f = _read_buf.read_exactly(bytes_to_read) + .then([this, bytes_to_read](temporary_buffer data) { + if (data.size() != bytes_to_read) { + _fd.shutdown_input(); + _fd.shutdown_output(); + throw tcp_connection_exception("Connection ended prematurely"); + } + return data; + }); + return seastar::with_timeout(timeout_end(_timeout_ms), std::move(f)); +} + +future<> tcp_connection::write(temporary_buffer buff) { + auto f = _write_buf.write(std::move(buff)).then([this] { + return _write_buf.flush(); + }); + return seastar::with_timeout(timeout_end(_timeout_ms), std::move(f)); +} + +future<> tcp_connection::close() { + return when_all_succeed(_read_buf.close(), _write_buf.close()) + .discard_result().handle_exception([](std::exception_ptr ep) { + // Ignore close exceptions. + }); +} + +} diff --git a/src/producer/batcher.cc b/src/producer/batcher.cc new file mode 100644 index 0000000..2d94020 --- /dev/null +++ b/src/producer/batcher.cc @@ -0,0 +1,103 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include + +#include + +using namespace seastar; + +namespace kafka4seastar { + +void batcher::queue_message(sender_message message) { + _messages.emplace_back(std::move(message)); + _messages_byte_size += message.size(); +// if (_expiration_time == 0 || _messages_byte_size > _buffer_memory) { + // (void) flush(); + // } +} + +future<> batcher::flush() { + return do_with(sender(_connection_manager, _metadata_manager, _request_timeout, _acks), [this](sender& sender) { + bool is_batch_loaded = false; + return _retry_helper.with_retry([this, &sender, is_batch_loaded]() mutable { + // It is important to move messages from current batch + // into sender and send requests in the same continuation, + // in order to preserve correct order of messages. + if (!is_batch_loaded) { + int size_moved = 0; + for (auto& message : _messages) { + size_moved += message.size(); + } + sender.move_messages(_messages); + _messages_byte_size -= size_moved; + is_batch_loaded = true; + } + sender.send_requests(); + + return sender.receive_responses().then([&sender] { + return sender.messages_empty() ? do_retry::no : do_retry::yes; + }); + }).finally([&sender] { + return sender.close(); + }); + }); +} + +future<> batcher::flush_coroutine(std::chrono::milliseconds dur) { + return seastar::do_until([this] { return !_keep_refreshing; }, [this, dur]{ + return seastar::sleep_abortable(dur, _stop_refresh).then([this] { + return flush(); + }).handle_exception([] (std::exception_ptr ep) { + try { + std::rethrow_exception(ep); + } catch (seastar::sleep_aborted& e) { + return make_ready_future(); + } catch (...) { + // no other exception should happen here, + // if they do, they have to be handled individually + std::rethrow_exception(ep); + } + }); + }).finally([this]{ + _refresh_finished.signal(); + }); +} + +void batcher::start_flush() { + if (_expiration_time == 0) { + return; + } + _keep_refreshing = true; + (void) flush_coroutine(std::chrono::milliseconds(_expiration_time)); +} + +future<> batcher::stop_flush() { + if (_expiration_time == 0) { + return make_ready_future(); + } + _keep_refreshing = false; + _stop_refresh.request_abort(); + return _refresh_finished.wait(1); +} + +} diff --git a/src/producer/kafka_producer.cc b/src/producer/kafka_producer.cc new file mode 100644 index 0000000..ed98b11 --- /dev/null +++ b/src/producer/kafka_producer.cc @@ -0,0 +1,99 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include +#include +#include + +#include +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +kafka_producer::kafka_producer(producer_properties&& properties) + : _properties(std::move(properties)), + _connection_manager(_properties._client_id), + _metadata_manager(_connection_manager, _properties._metadata_refresh), + _batcher(_metadata_manager, _connection_manager, _properties._retries, + _properties._acks, _properties._request_timeout, _properties._linger, + _properties._buffer_memory, std::move(_properties._retry_backoff_strategy)) {} + +seastar::future<> kafka_producer::init() { + return _connection_manager.init(_properties._servers, _properties._request_timeout).then([this] { + _metadata_manager.start_refresh(); + return _metadata_manager.refresh_metadata(); + }).then([this] { + _batcher.start_flush(); + }); +} + +seastar::future<> kafka_producer::produce(seastar::sstring topic_name, seastar::sstring key, seastar::sstring value) { + auto& metadata =_metadata_manager.get_metadata(); + auto partition_index = 0; + for (const auto& topic : *metadata._topics) { + if (*topic._name == topic_name) { + partition_index = *_properties._partitioner->get_partition(key, topic._partitions)._partition_index; + break; + } + } + + sender_message message; + message._topic = std::move(topic_name); + message._key = std::move(key); + message._value = std::move(value); + message._partition_index = partition_index; + + auto send_future = message._promise.get_future(); + _batcher.queue_message(std::move(message)); + return send_future; +} + +seastar::future<> kafka_producer::flush() { + return _batcher.flush(); +} + +seastar::future<> kafka_producer::disconnect() { + return _batcher.stop_flush().then([this] { + return _metadata_manager.stop_refresh(); + }).then([this] () { + return _connection_manager.disconnect_all(); + }); +} + +} diff --git a/src/producer/sender.cc b/src/producer/sender.cc new file mode 100644 index 0000000..53600a4 --- /dev/null +++ b/src/producer/sender.cc @@ -0,0 +1,285 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include + +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +sender::sender(connection_manager& connection_manager, + metadata_manager& metadata_manager, + uint32_t connection_timeout, + ack_policy acks) + : _connection_manager(connection_manager), + _metadata_manager(metadata_manager), + _connection_timeout(connection_timeout), + _acks(acks) {} + +std::optional sender::broker_for_topic_partition(const seastar::sstring& topic, int32_t partition_index) { + //auto& metadata = _metadata_manager.get_metadata(); + metadata_manager::broker_id key{topic, partition_index}; + return _metadata_manager.get_broker(key); +/* + auto topic_candidate = std::lower_bound(metadata._topics->begin(), metadata._topics->end(), topic, [](auto& a, auto& b) { + return *a._name < b; + }); + + if (topic_candidate != metadata._topics->end() && *topic_candidate->_name == topic && topic_candidate->_error_code == error::kafka_error_code::NONE) { + auto it = std::lower_bound(topic_candidate->_partitions->begin(), topic_candidate->_partitions->end(), partition_index, [](auto& a, auto& b) { + return *a._partition_index < b; + }); + + if (it != topic_candidate->_partitions->end() && *it->_partition_index == partition_index && it->_error_code == error::kafka_error_code::NONE) { + return broker_for_id(*it->_leader_id); + } + } + + return std::nullopt;*/ +} + +sender::connection_id sender::broker_for_id(int32_t id) { + auto& metadata = _metadata_manager.get_metadata(); + auto it = std::lower_bound(metadata._brokers->begin(), metadata._brokers->end(), id, [] (auto& a, auto& b) { + return *a._node_id < b; + }); + + if (*it->_node_id == id) { + return {*it->_host, *it->_port}; + } + + return {}; +} + +void sender::split_messages() { + _messages_split_by_topic_partition.clear(); + _messages_split_by_broker_topic_partition.clear(); + for (auto& message : _messages) { + auto broker = broker_for_topic_partition(message._topic, message._partition_index); + if (broker) { + _messages_split_by_broker_topic_partition[*broker][message._topic][message._partition_index].push_back(&message); + _messages_split_by_topic_partition[{message._topic, message._partition_index}].push_back(&message); + } else { + // TODO: Differentiate between unknown topic, leader not available etc. + message._error_code = error::kafka_error_code::UNKNOWN_TOPIC_OR_PARTITION; + } + } +} + +void sender::queue_requests() { + _responses.clear(); + _responses.reserve(_messages_split_by_broker_topic_partition.size()); + for (auto& [broker, messages_by_topic_partition] : _messages_split_by_broker_topic_partition) { + produce_request req; + req._acks = static_cast(_acks); + req._timeout_ms = _connection_timeout; + + kafka_array_t topics{ + kafka_array_t::container_type()}; + req._topics = std::move(topics); // NEW + + for (auto& [topic, messages_by_partition] : messages_by_topic_partition) { + produce_request_topic_produce_data topic_data; + topic_data._name = topic; + + kafka_array_t partitions{ + kafka_array_t::container_type()}; + topic_data._partitions = std::move(partitions); // NEW + + for (auto& [partition, messages] : messages_by_partition) { + produce_request_partition_produce_data partition_data; + partition_data._partition_index = partition; + + kafka_records records; + kafka_record_batch record_batch; + + record_batch._base_offset = 0; + record_batch._partition_leader_epoch = -1; + record_batch._magic = 2; + record_batch._compression_type = kafka_record_compression_type::NO_COMPRESSION; + record_batch._timestamp_type = kafka_record_timestamp_type::CREATE_TIME; + + auto first_timestamp = std::chrono::duration_cast(messages[0]->_timestamp.time_since_epoch()).count(); + record_batch._first_timestamp = first_timestamp; + record_batch._producer_id = -1; + record_batch._producer_epoch = -1; + record_batch._base_sequence = -1; + record_batch._is_transactional = false; + record_batch._is_control_batch = false; + + for (size_t i = 0; i < messages.size(); i++) { + auto current_timestamp = std::chrono::duration_cast(messages[i]->_timestamp.time_since_epoch()).count(); + + kafka_record record; + record._timestamp_delta = current_timestamp - first_timestamp; + record._offset_delta = i; + record._key = messages[i]->_key; + record._value = messages[i]->_value; + record_batch._records.emplace_back(std::move(record)); + } + + records._record_batches.emplace_back(std::move(record_batch)); + partition_data._records = std::move(records); // NEW + + topic_data._partitions->emplace_back(std::move(partition_data)); // NEW + } + req._topics->emplace_back(std::move(topic_data)); + } + + auto with_response = _acks != ack_policy::NONE; + _responses.emplace_back(_connection_manager.send(std::move(req), broker.first, broker.second, _connection_timeout, with_response) + .then([broker](auto response) { + return std::make_pair(broker, response); + })); + } +} + +void sender::set_error_code_for_broker(const sender::connection_id& broker, const error::kafka_error_code& error_code) { + for (auto& [topic, messages_by_partition] : _messages_split_by_broker_topic_partition[broker]) { + for (auto& [partition, messages] : messages_by_partition) { + for (auto& message : messages) { + (void)topic; (void)partition; + message->_error_code = error_code; + } + } + } +} + +void sender::set_success_for_broker(const sender::connection_id& broker) { + for (auto& [topic, messages_by_partition] : _messages_split_by_broker_topic_partition[broker]) { + for (auto& [partition, messages] : messages_by_partition) { + for (auto& message : messages) { + (void)topic; (void)partition; + message->_error_code = error::kafka_error_code::NONE; + message->_promise.set_value(); + } + } + } +} + +void sender::set_error_code_for_topic_partition(const seastar::sstring& topic, int32_t partition_index, + const error::kafka_error_code& error_code) { + for (auto& message : _messages_split_by_topic_partition[{topic, partition_index}]) { + message->_error_code = error_code; + } +} + +void sender::set_success_for_topic_partition(const seastar::sstring& topic, int32_t partition_index) { + for (auto& message : _messages_split_by_topic_partition[{topic, partition_index}]) { + message->_error_code = error::kafka_error_code::NONE; + message->_promise.set_value(); + } +} + +void sender::move_messages(std::vector& messages) { + _messages.reserve(_messages.size() + messages.size()); + _messages.insert(_messages.end(), std::make_move_iterator(messages.begin()), + std::make_move_iterator(messages.end())); + messages.clear(); +} + +size_t sender::messages_size() const { + return _messages.size(); +} + +bool sender::messages_empty() const { + return _messages.empty(); +} + +void sender::send_requests() { + split_messages(); + queue_requests(); +} + +future<> sender::receive_responses() { + return when_all(_responses.begin(), _responses.end()).then( + [this](std::vector>> responses) { + set_error_codes_for_responses(responses); + filter_messages(); + return process_messages_errors(); + }); +} + +future<> sender::process_messages_errors() { + auto should_refresh_metadata = false; + for (auto& message : _messages) { + if (message._error_code->_invalidates_metadata) { + should_refresh_metadata = true; + break; + } + } + if (should_refresh_metadata) { + return _metadata_manager.refresh_metadata().discard_result(); + } else { + return make_ready_future<>(); + } +} + +void sender::filter_messages() { + _messages.erase(std::remove_if(_messages.begin(), _messages.end(), [](auto& message) { + if (message._error_code == error::kafka_error_code::NONE) { + return true; + } + if (!message._error_code->_is_retriable) { + message._promise.set_exception(send_exception(message._error_code->_error_message)); + return true; + } + return false; + }), _messages.end()); +} + +void sender::set_error_codes_for_responses(std::vector>>& responses) { + for (auto& response : responses) { + auto [broker, response_message] = response.get0(); + if (response_message._error_code != error::kafka_error_code::NONE) { + set_error_code_for_broker(broker, *response_message._error_code); + continue; + } + if (response_message._responses.is_null()) { + // No detailed information (when ack_policy::NONE) so set success for the broker. + set_success_for_broker(broker); + continue; + } + for (auto& topic_response : *response_message._responses) { + for (auto& partition_response : *topic_response._partitions) { + if (partition_response._error_code == error::kafka_error_code::NONE) { + set_success_for_topic_partition(*topic_response._name, *partition_response._partition_index); + } else { + set_error_code_for_topic_partition(*topic_response._name, + *partition_response._partition_index, *partition_response._error_code); + } + } + } + } +} + +void sender::close() { + for (auto& message : _messages) { + message._promise.set_exception(send_exception(message._error_code->_error_message)); + } + _messages.clear(); +} + +} diff --git a/src/protocol/api_versions_request.cc b/src/protocol/api_versions_request.cc new file mode 100644 index 0000000..8bb9980 --- /dev/null +++ b/src/protocol/api_versions_request.cc @@ -0,0 +1,27 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include + +void kafka4seastar::api_versions_request::serialize(kafka::output_stream& os, int16_t api_version) const {} + +void kafka4seastar::api_versions_request::deserialize(kafka::input_stream& is, int16_t api_version) {} diff --git a/src/protocol/api_versions_response.cc b/src/protocol/api_versions_response.cc new file mode 100644 index 0000000..f8583a1 --- /dev/null +++ b/src/protocol/api_versions_response.cc @@ -0,0 +1,83 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include + +#include + +using namespace seastar; + +namespace kafka4seastar { + +void api_versions_response_key::serialize(kafka::output_stream& os, int16_t api_version) const { + _api_key.serialize(os, api_version); + _min_version.serialize(os, api_version); + _max_version.serialize(os, api_version); +} + +void api_versions_response_key::deserialize(kafka::input_stream& is, int16_t api_version) { + _api_key.deserialize(is, api_version); + _min_version.deserialize(is, api_version); + _max_version.deserialize(is, api_version); +} + +bool api_versions_response_key::operator<(const api_versions_response_key& other) const noexcept { + return *_api_key < *other._api_key; +} + +bool api_versions_response_key::operator<(int16_t api_key) const noexcept { + return *_api_key < api_key; +} + +api_versions_response_key api_versions_response::operator[](int16_t api_key) const { + auto it = std::lower_bound(_api_keys->begin(), _api_keys->end(), api_key); + if (it != _api_keys->end() && *it->_api_key == api_key) { + return *it; + } + api_versions_response_key null_response; + null_response._api_key = -1; + return null_response; +} + +bool api_versions_response::contains(int16_t api_key) const { + auto it = std::lower_bound(_api_keys->begin(), _api_keys->end(), api_key); + return it != _api_keys->end() && *it->_api_key == api_key; +} + +void api_versions_response::serialize(kafka::output_stream& os, int16_t api_version) const { + _error_code.serialize(os, api_version); + _api_keys.serialize(os, api_version); + if (api_version >= 1) { + _throttle_time_ms.serialize(os, api_version); + } +} + +void api_versions_response::deserialize(kafka::input_stream& is, int16_t api_version) { + _error_code.deserialize(is, api_version); + _api_keys.deserialize(is, api_version); + std::sort(_api_keys->begin(), _api_keys->end()); + if (api_version >= 1) { + _throttle_time_ms.deserialize(is, api_version); + } +} + +} diff --git a/src/protocol/headers.cc b/src/protocol/headers.cc new file mode 100644 index 0000000..259ef1e --- /dev/null +++ b/src/protocol/headers.cc @@ -0,0 +1,51 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include + +using namespace seastar; + +namespace kafka4seastar { + +void request_header::serialize(kafka::output_stream& os, int16_t api_version) const { + _api_key.serialize(os, api_version); + _api_version.serialize(os, api_version); + _correlation_id.serialize(os, api_version); + _client_id.serialize(os, api_version); +} + +void request_header::deserialize(kafka::input_stream& is, int16_t api_version) { + _api_key.deserialize(is, api_version); + _api_version.deserialize(is, api_version); + _correlation_id.deserialize(is, api_version); + _client_id.deserialize(is, api_version); +} + +void response_header::serialize(kafka::output_stream& os, int16_t api_version) const { + _correlation_id.serialize(os, api_version); +} + +void response_header::deserialize(kafka::input_stream& is, int16_t api_version) { + _correlation_id.deserialize(is, api_version); +} + +} diff --git a/src/protocol/kafka_error_code.cc b/src/protocol/kafka_error_code.cc new file mode 100644 index 0000000..0c5adc5 --- /dev/null +++ b/src/protocol/kafka_error_code.cc @@ -0,0 +1,616 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +namespace error { + +static std::unordered_map errors; + +kafka_error_code::kafka_error_code ( + int16_t error_code, + seastar::sstring error_message, + is_retriable is_retriable, + invalidates_metadata invalidates_metadata) + : _error_code(error_code), + _error_message(error_message), + _is_retriable(is_retriable), + _invalidates_metadata(invalidates_metadata) { + errors.insert(std::pair(error_code, *this)); +} + +const kafka_error_code& kafka_error_code::get_error(int16_t value) { + return errors.at(value); +} + +const kafka_error_code kafka_error_code::kafka_error_code::UNKNOWN_SERVER_ERROR( + -1, + "The server experienced an unexpected error when processing the request.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::kafka_error_code::NONE( + 0, + "", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::OFFSET_OUT_OF_RANGE ( + 1, + "The requested offset is not within the range of offsets maintained by the server.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::CORRUPT_MESSAGE ( + 2, + "This message failed its CRC checksum, exceeds the valid size, " + "has a null key for a compacted topic, or is otherwise corrupt.", + is_retriable::yes, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::UNKNOWN_TOPIC_OR_PARTITION ( + 3, + "This server does not host this topic-partition.", + is_retriable::yes, + invalidates_metadata::yes +); +const kafka_error_code kafka_error_code::INVALID_FETCH_SIZE ( + 4, + "The requested fetch size is invalid.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::LEADER_NOT_AVAILABLE ( + 5, + "There is no leader for this topic-partition " + "as we are in the middle of leadership election.", + is_retriable::yes, + invalidates_metadata::yes +); +const kafka_error_code kafka_error_code::NOT_LEADER_FOR_PARTITION ( + 6, + "This server is not the leader for that topic-partition.", + is_retriable::yes, + invalidates_metadata::yes +); +const kafka_error_code kafka_error_code::REQUEST_TIMED_OUT ( + 7, + "The request timed out.", + is_retriable::yes, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::BROKER_NOT_AVAILABLE ( + 8, + "The broker is not available.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::REPLICA_NOT_AVAILABLE ( + 9, + "The replica is not available for the requested topic partition.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::MESSAGE_TOO_LARGE ( + 10, + "The request included a message larger than " + "the max message size the server will accept.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::STALE_CONTROLLER_EPOCH ( + 11, + "The controller moved to another broker.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::OFFSET_METADATA_TOO_LARGE ( + 12, + "The metadata field of the offset request was too large.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::NETWORK_EXCEPTION ( + 13, + "The server disconnected before a response was retrieved.", + is_retriable::yes, + invalidates_metadata::yes +); +const kafka_error_code kafka_error_code::COORDINATOR_LOAD_IN_PROGRESS ( + 14, + "The coordinator is loading and hence can't process requests.", + is_retriable::yes, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::COORDINATOR_NOT_AVAILABLE ( + 15, + "The coordinator is not available.", + is_retriable::yes, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::NOT_COORDINATOR ( + 16, + "This is not the correct coordinator.", + is_retriable::yes, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_TOPIC_EXCEPTION ( + 17, + "The request attempted to perform an operation on an invalid topic.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::RECORD_LIST__TOO_LARGE ( + 18, + "The request included message batch larger than the configured segment size on the server.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::NOT_ENOUGH_REPLICAS ( + 19, + "Messages are rejected since there are fewer in-sync replicas than required.", + is_retriable::yes, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::NOT_ENOUGH_REPLICAS_AFTER_APPEND ( + 20, + "Messages are written to the log, but to fewer in-sync replicas than required.", + is_retriable::yes, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_REQUIRED_ACKS ( + 21, + "Produce request specified an invalid value for required acks.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::ILLEGAL_GENERATION ( + 22, + "Specified group generation id is not valid.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INCONSISTENT_PROTOCOL ( + 23, + "The group member's supported protocols are incompatible with those of existing members " + "or first group member tried to join with empty protocol type or empty protocol list.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_GROUP_ID ( + 24, + "The configured groupId is invalid.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::UNKNOWN_MEMBER_ID ( + 25, + "The coordinator is not aware of this member.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_SESSION_TIMEOUT ( + 26, + "The session timeout is not within the range allowed by the broker " + "(as configured by group.min.session.timeout.ms and group.max.session.timeout.ms).", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::REBALANCE_IN_PROGRESS ( + 27, + "The group is rebalancing, so a rejoin is needed.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_COMMIT_OFFSET_SIZE ( + 28, + "The committing offset data size is not valid.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::TOPIC_AUTHORIZATION_FAILED ( + 29, + "Topic authorization failed.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::GROUP_AUTHORIZATION_FAILED ( + 30, + "Group authorization failed.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::CLUSTER_AUTHORIZATION_FAILED ( + 31, + "Cluster authorization failed.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_TIMESTAMP ( + 32, + "The timestamp of the message is out of acceptable range.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::UNSUPPORTED_SASL_MECHANISM ( + 33, + "The broker does not support the requested SASL mechanism.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::ILLEGAL_SASL_STATE ( + 34, + "Request is not valid given the current SASL state.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::UNSUPPORTED_VERSION ( + 35, + "The version of API is not supported.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::TOPIC_ALREADY_EXISTS ( + 36, + "Topic with this name already exists.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_PARTITIONS ( + 37, + "Number of partitions is below 1.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_REPLICATION_FACTOR ( + 38, + "Replication factor is below 1 " + "or larger than the number of available brokers.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_REPLICA_ASSIGNMENT ( + 39, + "Replica assignment is invalid.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_CONFIG ( + 40, + "Configuration is invalid.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::NOT_CONTROLLER ( + 41, + "This is not the correct controller for this cluster.", + is_retriable::yes, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_REQUEST ( + 42, + "This most likely occurs because of a request being malformed by the " + "client library or the message was sent to an incompatible broker. " + "See the broker logs for more details.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::UNSUPPORTED_FOR_MESSAGE_FORMAT ( + 43, + "The message format version on the broker does not support the request.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::POLICY_VIOLATION ( + 44, + "Request parameters do not satisfy the configured policy.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::OUT_OF_ORDER_SEQUENCE_NUMBER ( + 45, + "The broker received an out of order sequence number.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::DUPLICATE_SEQUENCE_NUMBER ( + 46, + "The broker received a duplicate sequence number.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_PRODUCER_EPOCH ( + 47, + "Producer attempted an operation with an old epoch. Either there is a newer producer " + "with the same transactionalId, or the producer's transaction has been expired by the broker.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_TXN_STATE ( + 48, + "The producer attempted a transactional operation in an invalid state.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_PRODUCER_ID_MAPPING ( + 49, + "The producer attempted to use a producer id " + "which is not currently assigned to its transactional id.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_TRANSACTION_TIMEOUT ( + 50, + "The transaction timeout is larger than the maximum value allowed by " + "the broker (as configured by transaction.max.timeout.ms).", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::CONCURRENT_TRANSACTIONS ( + 51, + "The producer attempted to update a transaction " + "while another concurrent operation on the same transaction was ongoing.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::TRANSACTION_COORDINATOR_FENCED ( + 52, + "Indicates that the transaction coordinator sending a WriteTxnMarker " + "is no longer the current coordinator for a given producer.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::TRANSACTIONAL_ID_AUTHORIZATION_FAILED ( + 53, + "Transactional Id authorization failed.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::SECURITY_DISABLED ( + 54, + "Security features are disabled.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::OPERATION_NOT_ATTEMPTED ( + 55, + "The broker did not attempt to execute this operation. " + "This may happen for batched RPCs " + "where some operations in the batch failed, " + "causing the broker to respond without " + "trying the rest.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::KAFKA_STORAGE_ERROR ( + 56, + "Disk error when trying to access log file on the disk.", + is_retriable::yes, + invalidates_metadata::yes +); +const kafka_error_code kafka_error_code::LOG_DIR_NOT_FOUND ( + 57, + "The user-specified log directory is not found in the broker config.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::SASL_AUTHENTICATION_FAILED ( + 58, + "SASL Authentication failed.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::UNKNOWN_PRODUCER_ID ( + 59, + "This exception is raised by the broker if it could not locate the producer metadata " + "associated with the producerId in question. " + "This could happen if, for instance, the producer's records " + "were deleted because their retention time had elapsed. " + "Once the last records of the producerId are removed, " + "the producer's metadata is removed from the broker, " + "and future appends by the producer will return this exception.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::REASSIGNMENT_IN_PROGRESS ( + 60, + "A partition reassignment is in progress.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::DELEGATION_TOKEN_AUTH_DISABLED ( + 61, + "Delegation Token feature is not enabled.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::DELEGATION_TOKEN_NOT_FOUND ( + 62, + "Delegation Token is not found on server.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::DELEGATION_TOKEN_OWNER_MISMATCH ( + 63, + "Specified Principal is not valid Owner/Renewer.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::DELEGATION_TOKEN_REQUEST_NOT_ALLOWED ( + 64, + "Delegation Token requests are not allowed on PLAINTEXT/1-way SSL " + "channels and on delegation token authenticated channels.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::DELEGATION_TOKEN_AUTHORIZATION_FAILED ( + 65, + "Delegation Token authorization failed.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::DELEGATION_TOKEN_EXPIRED ( + 66, + "Delegation Token is expired.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_PRINCIPAL_TYPE ( + 67, + "Supplied principalType is not supported.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::NON_EMPTY_GROUP ( + 68, + "The group is not empty.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::GROUP_ID_NOT_FOUND ( + 69, + "The group id does not exist.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::FETCH_SESSION_ID_NOT_FOUND ( + 70, + "The fetch session ID was not found.", + is_retriable::yes, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_FETCH_SESSION_EPOCH ( + 71, + "The fetch session epoch is invalid.", + is_retriable::yes, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::LISTENER_NOT_FOUND ( + 72, + "There is no listener on the leader broker that matches the listener " + "on which metadata request was processed.", + is_retriable::yes, + invalidates_metadata::yes +); +const kafka_error_code kafka_error_code::TOPIC_DELETION_DISABLED ( + 73, + "Topic deletion is disabled.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::FENCED_LEADER_EPOCH ( + 74, + "The leader epoch in the request is older than the epoch on the broker.", + is_retriable::yes, + invalidates_metadata::yes +); +const kafka_error_code kafka_error_code::UNKNOWN_LEADER_EPOCH ( + 75, + "The leader epoch in the request is newer than the epoch on the broker.", + is_retriable::yes, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::UNSUPPORTED_COMPRESSION_TYPE ( + 76, + "The requesting client does not support the compression type of given partition.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::STALE_BROKER_EPOCH ( + 77, + "Broker epoch has changed.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::OFFSET_NOT_AVAILABLE ( + 78, + "The leader high watermark has not caught up from a recent leader election" + " so the offsets cannot be guaranteed to be monotonically increasing.", + is_retriable::yes, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::MEMBER_ID_REQUIRED ( + 79, + "The group member needs to have a valid member id " + "before actually entering a consumer group.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::PREFERRED_LEADER_NOT_AVAILABLE ( + 80, + "The preferred leader was not available.", + is_retriable::yes, + invalidates_metadata::yes +); +const kafka_error_code kafka_error_code::GROUP_MAX_SIZE_REACHED ( + 81, + "The consumer group has reached its max size.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::FENCED_INSTANCE_ID ( + 82, + "The broker rejected this consumer since " + "another consumer with the same group.instance.id has registered " + "with a different member.id.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::ELIGIBLE_LEADERS_NOT_AVAILABLE ( + 83, + "Eligible topic partition leaders are not available.", + is_retriable::yes, + invalidates_metadata::yes +); +const kafka_error_code kafka_error_code::ELECTION_NOT_NEEDED ( + 84, + "Leader election not needed for topic partition.", + is_retriable::yes, + invalidates_metadata::yes +); +const kafka_error_code kafka_error_code::NO_REASSIGNMENT_IN_PROGRESS ( + 85, + "No partition reassignment is in progress.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::GROUP_SUBSCRIBED_TO_TOPIC ( + 86, + "Deleting offsets of a topic is forbidden " + "while the consumer group is actively subscribed to it.", + is_retriable::no, + invalidates_metadata::no +); +const kafka_error_code kafka_error_code::INVALID_RECORD ( + 87, + "This record has failed the validation on broker and hence be rejected.", + is_retriable::no, + invalidates_metadata::no +); +} + +} diff --git a/src/protocol/kafka_records.cc b/src/protocol/kafka_records.cc new file mode 100644 index 0000000..dbd9d75 --- /dev/null +++ b/src/protocol/kafka_records.cc @@ -0,0 +1,385 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include + +#include +#include +#include +#include +#include +#include + +std::uint32_t crc32c(const char* first, const char* last) +{ + std::uint32_t code = ~0U; + + for ( ; first < last; /* inline */) + { + if (reinterpret_cast(first) % 8 == 0 && first + 8 <= last) + { + code = _mm_crc32_u64(code, *reinterpret_cast(first)); + first += 8; + } + else if (reinterpret_cast(first) % 4 == 0 && first + 4 <= last) + { + code = _mm_crc32_u32(code, *reinterpret_cast(first)); + first += 4; + } + else if (reinterpret_cast(first) % 2 == 0 && first + 2 <= last) + { + code = _mm_crc32_u16(code, *reinterpret_cast(first)); + first += 2; + } + else // if (reinterpret_cast(first) % 1 == 0 && first + 1 <= last) + { + code = _mm_crc32_u8(code, *reinterpret_cast(first)); + first += 1; + } + } + + return ~code; +} + +using namespace seastar; + +namespace kafka4seastar { + +void kafka_record_header::serialize(kafka::output_stream& os, int16_t api_version) const { + kafka_varint_t header_key_length(_header_key.size()); + header_key_length.serialize(os, api_version); + os.write(_header_key.data(), _header_key.size()); + + kafka_varint_t header_value_length(_value.size()); + header_value_length.serialize(os, api_version); + os.write(_value.data(), _value.size()); +} + +void kafka_record_header::deserialize(kafka::input_stream& is, int16_t api_version) { + kafka_buffer_t header_key; + header_key.deserialize(is, api_version); + _header_key.swap(*header_key); + + kafka_buffer_t value; + value.deserialize(is, api_version); + _value.swap(*value); +} + +size_t kafka_record_header::serialized_length() const noexcept { + size_t result = 0; + + kafka_varint_t header_key_length(_header_key.size()); + result += header_key_length.serialized_length(); + result += _header_key.size(); + + kafka_varint_t header_value_length(_value.size()); + result += header_value_length.serialized_length(); + result += _value.size(); + + return result; +} + +void kafka_record::serialize(kafka::output_stream& os, int16_t api_version) const { + size_t length_value = 0; + kafka_int8_t attributes(0); + length_value += attributes.serialized_length(); + + length_value += _timestamp_delta.serialized_length(); + length_value += _offset_delta.serialized_length(); + + kafka_varint_t key_length(_key.size()); + length_value += key_length.serialized_length(); + length_value += _key.size(); + + kafka_varint_t value_length(_value.size()); + length_value += value_length.serialized_length(); + length_value += _value.size(); + + kafka_varint_t header_count(_headers.size()); + length_value += header_count.serialized_length(); + + for (const auto& header : _headers) { + length_value += header.serialized_length(); + } + + kafka_varint_t length(length_value); + length.serialize(os, api_version); + + attributes.serialize(os, api_version); + _timestamp_delta.serialize(os, api_version); + _offset_delta.serialize(os, api_version); + + key_length.serialize(os, api_version); + os.write(_key.data(), _key.size()); + + value_length.serialize(os, api_version); + os.write(_value.data(), _value.size()); + + header_count.serialize(os, api_version); + for (const auto& header : _headers) { + header.serialize(os, api_version); + } +} + +void kafka_record::deserialize(kafka::input_stream& is, int16_t api_version) { + kafka_varint_t length; + length.deserialize(is, api_version); + if (*length < 0) { + throw parsing_exception("Length of record is invalid"); + } + + auto expected_end_of_record = is.get_position(); + expected_end_of_record += *length; + + kafka_int8_t attributes; + attributes.deserialize(is, api_version); + + _timestamp_delta.deserialize(is, api_version); + _offset_delta.deserialize(is, api_version); + + kafka_buffer_t key; + key.deserialize(is, api_version); + _key.swap(*key); + + kafka_buffer_t value; + value.deserialize(is, api_version); + _value.swap(*value); + + kafka_array_t headers; + headers.deserialize(is, api_version); + _headers.swap(*headers); + + if (is.get_position() != expected_end_of_record) { + throw parsing_exception("Stream ended prematurely when reading record"); + } +} + +void kafka_record_batch::serialize(kafka::output_stream& os, int16_t api_version) const { + if (*_magic != 2) { + // TODO: Implement parsing of versions 0, 1. + throw parsing_exception("Unsupported version of record batch"); + } + + // Payload stores the data after CRC field. + thread_local kafka::output_stream payload_stream = kafka::output_stream::resizable_stream(); + payload_stream.reset(); + + kafka_int16_t attributes(0); + attributes = *attributes | static_cast(_compression_type); + attributes = *attributes | (static_cast(_timestamp_type) << 3); + if (_is_transactional) { + attributes = *attributes | 0x10; + } + if (_is_control_batch) { + attributes = *attributes | 0x20; + } + + attributes.serialize(payload_stream, api_version); + + kafka_int32_t last_offset_delta(0); + if (!_records.empty()) { + last_offset_delta = *_records.back()._offset_delta; + } + + last_offset_delta.serialize(payload_stream, api_version); + + _first_timestamp.serialize(payload_stream, api_version); + + int32_t max_timestamp_delta = 0; + for (const auto& record : _records) { + max_timestamp_delta = std::max(max_timestamp_delta, *record._timestamp_delta); + } + kafka_int64_t max_timestamp(*_first_timestamp + max_timestamp_delta); + max_timestamp.serialize(payload_stream, api_version); + + _producer_id.serialize(payload_stream, api_version); + + _producer_epoch.serialize(payload_stream, api_version); + + _base_sequence.serialize(payload_stream, api_version); + + if (_compression_type != kafka_record_compression_type::NO_COMPRESSION) { + // TODO: Add support for compression. + throw parsing_exception("Unsupported compression type"); + } + + kafka_int32_t records_count(_records.size()); + records_count.serialize(payload_stream, api_version); + + for (const auto& record : _records) { + record.serialize(payload_stream, api_version); + } + + _base_offset.serialize(os, api_version); + + kafka_int32_t batch_length(0); + batch_length = *batch_length + payload_stream.size(); + // fields before the CRC field. + batch_length = *batch_length + 4 + 4 + 1; + batch_length.serialize(os, api_version); + + _partition_leader_epoch.serialize(os, api_version); + + _magic.serialize(os, api_version); + + kafka_int32_t crc(crc32c(payload_stream.begin(), payload_stream.begin() + payload_stream.size())); + crc.serialize(os, api_version); + + os.write(payload_stream.begin(), payload_stream.size()); +} + +void kafka_record_batch::deserialize(kafka::input_stream& is, int16_t api_version) { + // Move to magic byte, read it and return back to start. + auto start_position = is.get_position(); + is.set_position(8 + 4 + 4 + start_position); + _magic.deserialize(is, api_version); + is.set_position(start_position); + + if (*_magic != 2) { + // TODO: Implement parsing of versions 0, 1. + throw parsing_exception("Unsupported record batch version"); + } + + _base_offset.deserialize(is, api_version); + + kafka_int32_t batch_length; + batch_length.deserialize(is, api_version); + + auto expected_end_of_batch = is.get_position(); + expected_end_of_batch += *batch_length; + + _partition_leader_epoch.deserialize(is, api_version); + + _magic.deserialize(is, api_version); + + kafka_int32_t crc; + crc.deserialize(is, api_version); + + // TODO: Missing validation of returned CRC value. + + kafka_int16_t attributes; + attributes.deserialize(is, api_version); + + auto compression_type = *attributes & 0x7; + switch (compression_type) { + case 0: + _compression_type = kafka_record_compression_type::NO_COMPRESSION; + break; + case 1: + _compression_type = kafka_record_compression_type::GZIP; + break; + case 2: + _compression_type = kafka_record_compression_type::SNAPPY; + break; + case 3: + _compression_type = kafka_record_compression_type::LZ4; + break; + case 4: + _compression_type = kafka_record_compression_type::ZSTD; + break; + default: + throw parsing_exception("Unsupported compression type"); + } + + _timestamp_type = (*attributes & 0x8) ? + kafka_record_timestamp_type::LOG_APPEND_TIME + : _timestamp_type = kafka_record_timestamp_type::CREATE_TIME; + + _is_transactional = bool(*attributes & 0x10); + _is_control_batch = bool(*attributes & 0x20); + + kafka_int32_t last_offset_delta; + last_offset_delta.deserialize(is, api_version); + + _first_timestamp.deserialize(is, api_version); + + kafka_int64_t max_timestamp; + max_timestamp.deserialize(is, api_version); + + _producer_id.deserialize(is, api_version); + + _producer_epoch.deserialize(is, api_version); + + _base_sequence.deserialize(is, api_version); + + kafka_int32_t records_count; + records_count.deserialize(is, api_version); + + if (*records_count < 0) { + throw parsing_exception("Record count in batch is invalid"); + } + _records.resize(*records_count); + + auto remaining_bytes = expected_end_of_batch - is.get_position(); + std::vector records_payload(remaining_bytes); + + is.read(records_payload.data(), remaining_bytes); + if (is.gcount() != remaining_bytes) { + throw parsing_exception("Stream ended prematurely when reading record batch"); + } + + kafka::input_stream records_stream(records_payload.data(), records_payload.size()); + for (auto& record : _records) { + record.deserialize(records_stream, api_version); + } + + if (records_stream.get_position() != remaining_bytes) { + throw parsing_exception("Stream ended prematurely when reading record batch"); + } +} + +void kafka_records::serialize(kafka::output_stream& os, int16_t api_version) const { + thread_local kafka::output_stream serialized_batches_stream = kafka::output_stream::resizable_stream(); + serialized_batches_stream.reset(); + + for (const auto& batch : _record_batches) { + batch.serialize(serialized_batches_stream, api_version); + } + + kafka_int32_t records_length(serialized_batches_stream.size()); + records_length.serialize(os, api_version); + + os.write(serialized_batches_stream.begin(), serialized_batches_stream.size()); +} + +void kafka_records::deserialize(kafka::input_stream& is, int16_t api_version) { + kafka_int32_t records_length; + records_length.deserialize(is, api_version); + if (*records_length < 0) { + throw parsing_exception("Records length is invalid"); + } + + auto expected_end_of_records = is.get_position(); + expected_end_of_records += *records_length; + + _record_batches.clear(); + while (is.get_position() < expected_end_of_records) { + _record_batches.emplace_back(); + _record_batches.back().deserialize(is, api_version); + } + + if (is.get_position() != expected_end_of_records) { + throw parsing_exception("Stream ended prematurely when reading records"); + } +} + +} diff --git a/src/protocol/metadata_request.cc b/src/protocol/metadata_request.cc new file mode 100644 index 0000000..341e369 --- /dev/null +++ b/src/protocol/metadata_request.cc @@ -0,0 +1,59 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include + +using namespace seastar; + +namespace kafka4seastar { + +void metadata_request_topic::serialize(kafka::output_stream& os, int16_t api_version) const { + _name.serialize(os, api_version); +} + +void metadata_request_topic::deserialize(kafka::input_stream& is, int16_t api_version) { + _name.deserialize(is, api_version); +} + +void metadata_request::serialize(kafka::output_stream& os, int16_t api_version) const { + _topics.serialize(os, api_version); + if (api_version >= 4) { + _allow_auto_topic_creation.serialize(os, api_version); + } + if (api_version >= 8) { + _include_cluster_authorized_operations.serialize(os, api_version); + _include_topic_authorized_operations.serialize(os, api_version); + } +} + +void metadata_request::deserialize(kafka::input_stream& is, int16_t api_version) { + _topics.deserialize(is, api_version); + if (api_version >= 4) { + _allow_auto_topic_creation.deserialize(is, api_version); + } + if (api_version >= 8) { + _include_cluster_authorized_operations.deserialize(is, api_version); + _include_topic_authorized_operations.deserialize(is, api_version); + } +} + +} diff --git a/src/protocol/metadata_response.cc b/src/protocol/metadata_response.cc new file mode 100644 index 0000000..2b21498 --- /dev/null +++ b/src/protocol/metadata_response.cc @@ -0,0 +1,133 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include + +using namespace seastar; + +namespace kafka4seastar { + +void metadata_response_broker::serialize(kafka::output_stream& os, int16_t api_version) const { + _node_id.serialize(os, api_version); + _host.serialize(os, api_version); + _port.serialize(os, api_version); + if (api_version >= 1) { + _rack.serialize(os, api_version); + } +} + +void metadata_response_broker::deserialize(kafka::input_stream& is, int16_t api_version) { + _node_id.deserialize(is, api_version); + _host.deserialize(is, api_version); + _port.deserialize(is, api_version); + if (api_version >= 1) { + _rack.deserialize(is, api_version); + } +} + +void metadata_response_partition::serialize(kafka::output_stream& os, int16_t api_version) const { + _error_code.serialize(os, api_version); + _partition_index.serialize(os, api_version); + _leader_id.serialize(os, api_version); + if (api_version >= 7) { + _leader_epoch.serialize(os, api_version); + } + _replica_nodes.serialize(os, api_version); + _isr_nodes.serialize(os, api_version); + if (api_version >= 5) { + _offline_replicas.serialize(os, api_version); + } +} + +void metadata_response_partition::deserialize(kafka::input_stream& is, int16_t api_version) { + _error_code.deserialize(is, api_version); + _partition_index.deserialize(is, api_version); + _leader_id.deserialize(is, api_version); + if (api_version >= 7) { + _leader_epoch.deserialize(is, api_version); + } + _replica_nodes.deserialize(is, api_version); + _isr_nodes.deserialize(is, api_version); + if (api_version >= 5) { + _offline_replicas.deserialize(is, api_version); + } +} + +void metadata_response_topic::serialize(kafka::output_stream& os, int16_t api_version) const { + _error_code.serialize(os, api_version); + _name.serialize(os, api_version); + if (api_version >= 1) { + _is_internal.serialize(os, api_version); + } + _partitions.serialize(os, api_version); + if (api_version >= 8) { + _topic_authorized_operations.serialize(os, api_version); + } +} + +void metadata_response_topic::deserialize(kafka::input_stream& is, int16_t api_version) { + _error_code.deserialize(is, api_version); + _name.deserialize(is, api_version); + if (api_version >= 1) { + _is_internal.deserialize(is, api_version); + } + _partitions.deserialize(is, api_version); + if (api_version >= 8) { + _topic_authorized_operations.deserialize(is, api_version); + } +} + +void metadata_response::serialize(kafka::output_stream& os, int16_t api_version) const { + if (api_version >= 3) { + _throttle_time_ms.serialize(os, api_version); + } + _brokers.serialize(os, api_version); + if (api_version >= 2) { + _cluster_id.serialize(os, api_version); + } + if (api_version >= 1) { + _controller_id.serialize(os, api_version); + } + _topics.serialize(os, api_version); + if (api_version >= 8) { + _cluster_authorized_operations.serialize(os, api_version); + } +} + +void metadata_response::deserialize(kafka::input_stream& is, int16_t api_version) { + if (api_version >= 3) { + _throttle_time_ms.deserialize(is, api_version); + } + _brokers.deserialize(is, api_version); + if (api_version >= 2) { + _cluster_id.deserialize(is, api_version); + } + if (api_version >= 1) { + _controller_id.deserialize(is, api_version); + } + _topics.deserialize(is, api_version); + if (api_version >= 8) { + _cluster_authorized_operations.deserialize(is, api_version); + } +} + +} diff --git a/src/protocol/produce_request.cc b/src/protocol/produce_request.cc new file mode 100644 index 0000000..8d83ac7 --- /dev/null +++ b/src/protocol/produce_request.cc @@ -0,0 +1,67 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include + +using namespace seastar; + +namespace kafka4seastar { + +void produce_request_partition_produce_data::serialize(kafka::output_stream& os, int16_t api_version) const { + _partition_index.serialize(os, api_version); + _records.serialize(os, api_version); +} + +void produce_request_partition_produce_data::deserialize(kafka::input_stream& is, int16_t api_version) { + _partition_index.deserialize(is, api_version); + _records.deserialize(is, api_version); +} + +void produce_request_topic_produce_data::serialize(kafka::output_stream& os, int16_t api_version) const { + _name.serialize(os, api_version); + _partitions.serialize(os, api_version); +} + +void produce_request_topic_produce_data::deserialize(kafka::input_stream& is, int16_t api_version) { + _name.deserialize(is, api_version); + _partitions.deserialize(is, api_version); +} + +void produce_request::serialize(kafka::output_stream& os, int16_t api_version) const { + if (api_version >= 3) { + _transactional_id.serialize(os, api_version); + } + _acks.serialize(os, api_version); + _timeout_ms.serialize(os, api_version); + _topics.serialize(os, api_version); +} + +void produce_request::deserialize(kafka::input_stream& is, int16_t api_version) { + if (api_version >= 3) { + _transactional_id.deserialize(is, api_version); + } + _acks.deserialize(is, api_version); + _timeout_ms.deserialize(is, api_version); + _topics.deserialize(is, api_version); +} + +} diff --git a/src/protocol/produce_response.cc b/src/protocol/produce_response.cc new file mode 100644 index 0000000..2a1963d --- /dev/null +++ b/src/protocol/produce_response.cc @@ -0,0 +1,96 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +void produce_response_batch_index_and_error_message::serialize(kafka::output_stream& os, int16_t api_version) const { + _batch_index.serialize(os, api_version); + _batch_index_error_message.serialize(os, api_version); +} + +void produce_response_batch_index_and_error_message::deserialize(kafka::input_stream& is, int16_t api_version) { + _batch_index.deserialize(is, api_version); + _batch_index_error_message.deserialize(is, api_version); +} + +void produce_response_partition_produce_response::serialize(kafka::output_stream& os, int16_t api_version) const { + _partition_index.serialize(os, api_version); + _error_code.serialize(os, api_version); + _base_offset.serialize(os, api_version); + if (api_version >= 2) { + _log_append_time_ms.serialize(os, api_version); + } + if (api_version >= 5) { + _log_start_offset.serialize(os, api_version); + } + if (api_version >= 8) { + _record_errors.serialize(os, api_version); + _error_message.serialize(os, api_version); + } +} + +void produce_response_partition_produce_response::deserialize(kafka::input_stream& is, int16_t api_version) { + _partition_index.deserialize(is, api_version); + _error_code.deserialize(is, api_version); + _base_offset.deserialize(is, api_version); + if (api_version >= 2) { + _log_append_time_ms.deserialize(is, api_version); + } + if (api_version >= 5) { + _log_start_offset.deserialize(is, api_version); + } + if (api_version >= 8) { + _record_errors.deserialize(is, api_version); + _error_message.deserialize(is, api_version); + } +} + +void produce_response_topic_produce_response::serialize(kafka::output_stream& os, int16_t api_version) const { + _name.serialize(os, api_version); + _partitions.serialize(os, api_version); +} + +void produce_response_topic_produce_response::deserialize(kafka::input_stream& is, int16_t api_version) { + _name.deserialize(is, api_version); + _partitions.deserialize(is, api_version); +} + +void produce_response::serialize(kafka::output_stream& os, int16_t api_version) const { + _responses.serialize(os, api_version); + if (api_version >= 1) { + _throttle_time_ms.serialize(os, api_version); + } +} + +void produce_response::deserialize(kafka::input_stream& is, int16_t api_version) { + _responses.deserialize(is, api_version); + if (api_version >= 1) { + _throttle_time_ms.deserialize(is, api_version); + } +} + +} diff --git a/src/utils/defaults.cc b/src/utils/defaults.cc new file mode 100644 index 0000000..3f4e8b5 --- /dev/null +++ b/src/utils/defaults.cc @@ -0,0 +1,66 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include +#include +#include + +#include + +#include + +using namespace seastar; + +namespace kafka4seastar { + +namespace defaults { + +noncopyable_function(uint32_t)> exp_retry_backoff(uint32_t base_ms, uint32_t max_backoff_ms) { + std::random_device rd; + return [base_ms, max_backoff_ms, mt = std::mt19937(rd())] (uint32_t retry_number) mutable { + if (retry_number == 0) { + return make_ready_future<>(); + } + + // Exponential backoff with (full) jitter + auto backoff_time = base_ms * std::pow(2.0f, retry_number - 1); + auto backoff_time_discrete = static_cast(std::round(backoff_time)); + + auto capped_backoff_time = std::min(max_backoff_ms, backoff_time_discrete); + std::uniform_int_distribution dist(0, capped_backoff_time); + + auto jittered_backoff = dist(mt); + return seastar::sleep(std::chrono::milliseconds(jittered_backoff)); + }; +} + +std::unique_ptr round_robin_partitioner() { + return std::make_unique(); +} + +std::unique_ptr random_partitioner() { + return std::make_unique(); +} + +} + +} diff --git a/src/utils/metadata_manager.cc b/src/utils/metadata_manager.cc new file mode 100644 index 0000000..1744d0c --- /dev/null +++ b/src/utils/metadata_manager.cc @@ -0,0 +1,134 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include +#include +#include + +#include + +using namespace seastar; + +namespace kafka4seastar { + + void metadata_manager::parse_new_metadata() { + std::unordered_map from_id; + for (auto& broker : *_metadata._brokers) { + from_id.insert({*broker._node_id, broker}); + } + + + std::unordered_map new_brokers; + for (auto& topic : *_metadata._topics) { + for (auto& partition : *topic._partitions) { + auto& node = from_id[*partition._leader_id]; + new_brokers.insert({{*topic._name, *partition._partition_index}, {*node._host, *node._port}}); + } + } + + _brokers = std::move(new_brokers); + } + + std::optional metadata_manager::get_broker(metadata_manager::broker_id& id) { + auto it = _brokers.find(id); + if (it != _brokers.end()) { + auto val = it->second; + return std::optional(std::move(val)); + } + return std::nullopt; + } + + seastar::future<> metadata_manager::refresh_metadata() { + metadata_request req; + + req._allow_auto_topic_creation = true; + req._include_cluster_authorized_operations = true; + req._include_topic_authorized_operations = true; + + return _connection_manager.ask_for_metadata(std::move(req)).then([this] (metadata_response metadata) { + std::sort(metadata._brokers->begin(), metadata._brokers->end(), [] (auto& a, auto& b) { + return *a._node_id < *b._node_id; + }); + std::sort(metadata._topics->begin(), metadata._topics->end(), [] (auto& a, auto& b) { + if (*a._name == *b._name) { + return a._error_code == error::kafka_error_code::NONE; + } else { + return *a._name < *b._name; + } + }); + for (auto& topic : *metadata._topics) { + std::sort(topic._partitions->begin(), topic._partitions->end(), [] (auto& a, auto& b) { + if (*a._partition_index == *b._partition_index) { + return a._error_code == error::kafka_error_code::NONE; + } else { + return *a._partition_index < *b._partition_index; + } + }); + } + _metadata = std::move(metadata); + parse_new_metadata(); + }).handle_exception([] (std::exception_ptr ep) { + try { + std::rethrow_exception(ep); + } catch (metadata_refresh_exception& e) { + // Ignore metadata_refresh_exception and preserve the old metadata. + return; + } + }); + } + + seastar::future<> metadata_manager::refresh_coroutine(std::chrono::milliseconds dur) { + return seastar::do_until([this]{return ! _keep_refreshing;}, [this, dur]{ + return seastar::sleep_abortable(dur, _stop_refresh).then([this] { + return refresh_metadata(); + }).handle_exception([this] (std::exception_ptr ep) { + try { + std::rethrow_exception(ep); + } catch (seastar::sleep_aborted& e) { + return make_ready_future(); + } catch (...) { + // no other exception should happen here, + // if they do, they have to be handled individually + std::rethrow_exception(ep); + } + }); + }).finally([this]{ + _refresh_finished.signal(); + return; + }); + } + + metadata_response& metadata_manager::get_metadata() { + return _metadata; + } + + void metadata_manager::start_refresh() { + _keep_refreshing = true; + (void) refresh_coroutine(std::chrono::milliseconds(_expiration_time)); + } + + future<> metadata_manager::stop_refresh() { + _keep_refreshing = false; + _stop_refresh.request_abort(); + return _refresh_finished.wait(1); + } +} diff --git a/src/utils/partitioner.cc b/src/utils/partitioner.cc new file mode 100644 index 0000000..1fd9a1c --- /dev/null +++ b/src/utils/partitioner.cc @@ -0,0 +1,45 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include +#include + +using namespace seastar; + +namespace kafka4seastar { + +const metadata_response_partition& basic_partitioner::get_partition(const seastar::sstring& key, const kafka_array_t& partitions) { + size_t index = std::rand() % partitions->size(); + return partitions[index]; +} + +const metadata_response_partition& rr_partitioner::get_partition(const seastar::sstring& key, const kafka_array_t& partitions) { + if (!key.empty()) { + auto hash_value = std::hash()(key); + return partitions[hash_value % partitions->size()]; + } + else { + return partitions[(counter++) % partitions->size()]; + } +} + +} diff --git a/tests/kafka_e2e_local/test_base_producer.sh b/tests/kafka_e2e_local/test_base_producer.sh new file mode 100755 index 0000000..e023711 --- /dev/null +++ b/tests/kafka_e2e_local/test_base_producer.sh @@ -0,0 +1,144 @@ +#!/usr/bin/env bash + +if [ -z ${KAFKA_DEMO_DIR+x} ]; then + tput setaf 1 + echo "KAFKA_DEMO_DIR not set. Set environment variable KAFKA_DEMO_DIR to a directory of kafka_demo executable." + tput sgr0 + exit 1 +fi + +if [ -z ${KAFKA_BIN_DIR+x} ]; then + tput setaf 1 + echo "KAFKA_BIN_DIR not set. Set environment variable KAFKA_BIN_DIR to Kafka bin directory from Kafka sources," + echo "which contains console consumer and other tools." + tput sgr0 + exit 1 +fi + +function init_kafka() { + tput setaf 4 + echo "Starting Kafka." + tput sgr0 + + cd ../kafkadev_local || exit + + terraform init >/dev/null 2>&1 + terraform apply --var "kafka_count=$1" --var "network_cidr=$2" --auto-approve >/dev/null 2>&1 + + cd "$OLDPWD" + + # Wait 10s for Kafka to start up. + sleep 10s + + tput setaf 3 + printf "Started Kafka.\n\n" + tput sgr0 +} + +function init_topic() { + tput setaf 4 + echo "Creating topic: $4." + tput sgr0 + + "$KAFKA_BIN_DIR"/kafka-topics.sh --create --bootstrap-server "$1" --replication-factor "$2" \ + --partitions "$3" --topic "$4" >/dev/null 2>&1 +} + +function init_producer() { + # Wait 15s for topics to init their leaders, etc. + tput setaf 4 + echo "Waiting 15s for Kafka start up to fully finish." + tput sgr0 + sleep 15s + + touch .kafka_producer_input + + # Set up netcat at port 7777 to pipe input into. + (nc -k -l 7777 | "$KAFKA_DEMO_DIR"/kafka_demo --host "$1" >/dev/null 2>&1) & + KAFKA_DEMO_PID=$! + + tput setaf 3 + printf "Started producer.\n\n" + tput sgr0 +} + +function init_consumer() { + "$KAFKA_BIN_DIR"/kafka-console-consumer.sh --bootstrap-server "$1" \ + --whitelist "$2" > .kafka_consumer_output 2>/dev/null & + KAFKA_CONSUMER_PID=$! + + tput setaf 4 + echo "Starting consumer. Waiting 10s for it to start up." + tput sgr0 + + # Wait for consumer (and producer) to start up. + # Mitigates against possible race conditions, Kafka leader not elected. + sleep 10s + + tput setaf 3 + printf "Started consumer.\n\n" + tput sgr0 + + tput setaf 11 + echo "Starting test!" + tput sgr0 +} + +function write_random() { + RANDOM_LINE=$(head /dev/urandom | tr -dc A-Za-z0-9 | head -c 30 ; echo '') + echo "$1" | nc localhost 7777 + echo "$RANDOM_LINE" | nc localhost 7777 + echo "$RANDOM_LINE" | nc localhost 7777 + echo "$RANDOM_LINE" >> .kafka_producer_input +} + +function invoke_docker() { + CONTAINERS=$(cd ../kafkadev_local; terraform output -json kafka_name | cut -c2- | rev | cut -c2- | rev | tr ',' '\n') + NTH=$(echo $CONTAINERS | tr ' ' '\n' | sed -n "$1p" | tr -d '"') + + tput setaf 4 + echo "Invoking docker: docker $2 $NTH." + tput sgr0 + + docker $2 $NTH >/dev/null 2>&1 +} + +function end_test() { + tput setaf 11 + echo "Ending test!" + tput sgr0 + + tput setaf 4 + echo "Waiting 10s for processes to wind up." + tput sgr0 + + sleep 5s + echo "q" | nc localhost 7777 + sleep 5s + + pkill -P $KAFKA_DEMO_PID >/dev/null 2>&1 + pkill -P $KAFKA_CONSUMER_PID >/dev/null 2>&1 + + # Sort files, because partitions can change order of messages. + sort -o .kafka_consumer_output .kafka_consumer_output + sort -o .kafka_producer_input .kafka_producer_input + + if cmp -s .kafka_consumer_output .kafka_producer_input; then + tput setaf 2 + echo "TEST PASSED! Consumer has received all messages from producer." + tput sgr0 + else + tput setaf 1 + echo "TEST FAILED! Consumer has NOT received all messages from producer." + tput sgr0 + fi + + rm .kafka_producer_input + rm .kafka_consumer_output + + cd ../kafkadev_local || exit + + terraform destroy --var "kafka_count=$1" --var "network_cidr=$2" --auto-approve >/dev/null 2>&1 + + cd "$OLDPWD" +} diff --git a/tests/kafka_e2e_local/test_producer1.sh b/tests/kafka_e2e_local/test_producer1.sh new file mode 100755 index 0000000..6796303 --- /dev/null +++ b/tests/kafka_e2e_local/test_producer1.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +# Producer test 1. +# Create 1 broker cluster and a single topic. +# Write a few messages to this topic. + +source "./test_base_producer.sh" + +init_kafka "1" "172.14.0.0/16" +init_topic "172.14.0.1:9092" "1" "1" "single" +init_producer "172.14.0.1" +init_consumer "172.14.0.1:9092" "single" + +write_random "single" +write_random "single" +write_random "single" +end_test "1" "172.14.0.0/16" diff --git a/tests/kafka_e2e_local/test_producer2.sh b/tests/kafka_e2e_local/test_producer2.sh new file mode 100755 index 0000000..da6f45d --- /dev/null +++ b/tests/kafka_e2e_local/test_producer2.sh @@ -0,0 +1,19 @@ +#!/usr/bin/env bash + +# Producer test 2. +# Create 3 broker cluster and two topics with 3 partitions. +# Write a few messages to both topics. + +source "./test_base_producer.sh" + +init_kafka "3" "172.14.0.0/16" +init_topic "172.14.0.1:9092" "3" "3" "triple" +init_topic "172.14.0.1:9092" "3" "3" "triple2" +init_producer "172.14.0.1" +init_consumer "172.14.0.1:9092" "triple|triple2" + +write_random "triple" +write_random "triple" +write_random "triple" +write_random "triple2" +end_test "3" "172.14.0.0/16" diff --git a/tests/kafka_e2e_local/test_producer3.sh b/tests/kafka_e2e_local/test_producer3.sh new file mode 100755 index 0000000..69ef147 --- /dev/null +++ b/tests/kafka_e2e_local/test_producer3.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +# Producer test 3. +# Create 3 broker cluster and two topics with 3 partitions. +# Write 200 messages total to both topics. + +source "./test_base_producer.sh" + +init_kafka "3" "172.14.0.0/16" +init_topic "172.14.0.1:9092" "3" "3" "triple" +init_topic "172.14.0.1:9092" "3" "3" "triple2" +init_producer "172.14.0.1" +init_consumer "172.14.0.1:9092" "triple|triple2" + +for i in {1..100}; do + write_random "triple" + write_random "triple2" +done + +end_test "3" "172.14.0.0/16" diff --git a/tests/kafka_e2e_local/test_producer4.sh b/tests/kafka_e2e_local/test_producer4.sh new file mode 100755 index 0000000..badee58 --- /dev/null +++ b/tests/kafka_e2e_local/test_producer4.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash + +# Producer test 4. +# Create 3 broker cluster and two topics with 3 partitions. +# Write a few messages to both topics. +# Stop second broker. +# Write a few messages to both topics. +# Start second broker. + +source "./test_base_producer.sh" + +init_kafka "3" "172.14.0.0/16" +init_topic "172.14.0.1:9092" "3" "3" "triple" +init_topic "172.14.0.1:9092" "3" "3" "triple2" +init_producer "172.14.0.1" +init_consumer "172.14.0.1:9092" "triple|triple2" + +for i in {1..10}; do + write_random "triple" + write_random "triple2" +done + +sleep 5s +invoke_docker "2" "stop" +sleep 10s + +for i in {1..10}; do + write_random "triple" + write_random "triple2" +done + +sleep 5s +invoke_docker "2" "start" +sleep 10s + +end_test "3" "172.14.0.0/16" diff --git a/tests/kafka_e2e_local/test_producer5.sh b/tests/kafka_e2e_local/test_producer5.sh new file mode 100755 index 0000000..2f1c42e --- /dev/null +++ b/tests/kafka_e2e_local/test_producer5.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash + +# Producer test 5. +# Create 3 broker cluster and two topics with 3 partitions. +# Write a few messages to both topics. +# Stop first broker. +# Write a few messages to both topics. +# Start first broker. + +source "./test_base_producer.sh" + +init_kafka "3" "172.14.0.0/16" +init_topic "172.14.0.1:9092" "3" "3" "triple" +init_topic "172.14.0.1:9092" "3" "3" "triple2" +init_producer "172.14.0.1" +init_consumer "172.14.0.1:9092" "triple|triple2" + +for i in {1..10}; do + write_random "triple" + write_random "triple2" +done + +sleep 5s +invoke_docker "1" "stop" +sleep 10s + +for i in {1..10}; do + write_random "triple" + write_random "triple2" +done + +sleep 5s +invoke_docker "1" "start" +sleep 10s + +end_test "3" "172.14.0.0/16" diff --git a/tests/unit/CMakeLists.txt b/tests/unit/CMakeLists.txt new file mode 100644 index 0000000..1fde8aa --- /dev/null +++ b/tests/unit/CMakeLists.txt @@ -0,0 +1,58 @@ +## +## This file is open source software, licensed to you under the terms +## of the Apache License, Version 2.0 (the "License"). See the NOTICE file +## distributed with this work for additional information regarding copyright +## ownership. You may not use this file except in compliance with the License. +## +## You may obtain a copy of the License at +## +## http://www.apache.org/licenses/LICENSE-2.0 +## +## Unless required by applicable law or agreed to in writing, +## software distributed under the License is distributed on an +## "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +## KIND, either express or implied. See the License for the +## specific language governing permissions and limitations +## under the License. +## +# +## +## Copyright (C) 2018 Scylladb, Ltd. +## +# + +add_custom_target(unit_tests) + +function(add_kafka_test name) + + cmake_parse_arguments (parsed_args + "" + "" + "SOURCES;LIBRARIES" + ${ARGN}) + + set(target test_unit_${name}) + + set(libraries ${parsed_args_LIBRARIES}) + list(APPEND libraries + kafka4seastar) + + add_executable(${target} ${parsed_args_SOURCES}) + target_link_libraries(${target} + PRIVATE ${libraries}) + + add_dependencies(unit_tests ${target}) + + target_compile_definitions (${target} + PRIVATE SEASTAR_TESTING_MAIN) + +endfunction() + +add_kafka_test(kafka_connection + SOURCES kafka_connection_test.cc) + +add_kafka_test(kafka_protocol + SOURCES kafka_protocol_test.cc) + +add_kafka_test(kafka_retry_helper + SOURCES kafka_retry_helper_test.cc) diff --git a/tests/unit/kafka_connection_test.cc b/tests/unit/kafka_connection_test.cc new file mode 100644 index 0000000..9a965a6 --- /dev/null +++ b/tests/unit/kafka_connection_test.cc @@ -0,0 +1,85 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include +#include +#include + +using namespace seastar; +namespace k4s = kafka4seastar; + +// All of the tests below assume that there is a Kafka broker running +// on address BROKER_ADDRESS +constexpr char BROKER_ADDRESS[] = "172.13.0.1"; +constexpr uint16_t PORT = 9092; +constexpr auto TIMEOUT = 1000; + +constexpr char message_str[] = "\x00\x00\x00\x0E\x00\x12\x00\x02\x00\x00\x00\x00\x00\x04\x74\x65\x73\x74"; +constexpr size_t message_len = sizeof(message_str); + +SEASTAR_THREAD_TEST_CASE(kafka_establish_connection_test) { + k4s::tcp_connection::connect(BROKER_ADDRESS, PORT, TIMEOUT).get(); +} + +SEASTAR_THREAD_TEST_CASE(kafka_connection_write_without_errors_test) { + temporary_buffer message {message_str, message_len}; + + auto conn = k4s::tcp_connection::connect(BROKER_ADDRESS, PORT, TIMEOUT).get0(); + conn.write(message.clone()).get(); + conn.close().get(); +} + +SEASTAR_THREAD_TEST_CASE(kafka_connection_read_without_errors_test) { + return; +} + +SEASTAR_THREAD_TEST_CASE(kafka_connection_successful_write_read_routine_test) { + const std::string correct_response {"\x00\x00\x01\x1C\x00\x00\x00\x00\x00\x00\x00\x00\x00\x2d\x00\x00" + "\x00\x00\x00\x07\x00\x01\x00\x00\x00\x0b\x00\x02\x00\x00\x00\x05" + "\x00\x03\x00\x00\x00\x08\x00\x04\x00\x00\x00\x02\x00\x05\x00\x00" + "\x00\x01\x00\x06\x00\x00\x00\x05\x00\x07\x00\x00\x00\x02\x00\x08" + "\x00\x00\x00\x07\x00\x09\x00\x00\x00\x05\x00\x0a\x00\x00\x00\x02" + "\x00\x0b\x00\x00\x00\x05\x00\x0c\x00\x00\x00\x03\x00\x0d\x00\x00" + "\x00\x02\x00\x0e\x00\x00\x00\x03\x00\x0f\x00\x00\x00\x03\x00\x10" + "\x00\x00\x00\x02\x00\x11\x00\x00\x00\x01\x00\x12\x00\x00\x00\x02" + "\x00\x13\x00\x00\x00\x03\x00\x14\x00\x00\x00\x03\x00\x15\x00\x00" + "\x00\x01\x00\x16\x00\x00\x00\x01\x00\x17\x00\x00\x00\x03\x00\x18" + "\x00\x00\x00\x01\x00\x19\x00\x00\x00\x01\x00\x1a\x00\x00\x00\x01" + "\x00\x1b\x00\x00\x00\x00\x00\x1c\x00\x00\x00\x02\x00\x1d\x00\x00" + "\x00\x01\x00\x1e\x00\x00\x00\x01\x00\x1f\x00\x00\x00\x01\x00\x20" + "\x00\x00\x00\x02\x00\x21\x00\x00\x00\x01\x00\x22\x00\x00\x00\x01" + "\x00\x23\x00\x00\x00\x01\x00\x24\x00\x00\x00\x01\x00\x25\x00\x00" + "\x00\x01\x00\x26\x00\x00\x00\x01\x00\x27\x00\x00\x00\x01\x00\x28" + "\x00\x00\x00\x01\x00\x29\x00\x00\x00\x01\x00\x2a\x00\x00\x00\x01" + "\x00\x2b\x00\x00\x00\x00\x00\x2c\x00\x00\x00\x00\x00\x00\x00\x00", + 18 * 16 + }; + + temporary_buffer message {message_str, message_len}; + + auto conn = k4s::tcp_connection::connect(BROKER_ADDRESS, PORT, TIMEOUT).get0(); + conn.write(message.clone()).get(); + auto buff = conn.read(correct_response.length()).get0(); + std::string response {buff.get(), buff.size()}; + BOOST_CHECK_EQUAL(response, correct_response); + conn.close().get(); +} diff --git a/tests/unit/kafka_protocol_test.cc b/tests/unit/kafka_protocol_test.cc new file mode 100644 index 0000000..7f0ef19 --- /dev/null +++ b/tests/unit/kafka_protocol_test.cc @@ -0,0 +1,495 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +/* + * Copyright (C) 2019 ScyllaDB + */ + +#define BOOST_TEST_MODULE kafka + +#include + +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace seastar; +namespace k4s = kafka4seastar; + +template +void test_deserialize_serialize(std::vector data, + KafkaType &kafka_value, int16_t api_version) { + boost::iostreams::stream input_stream(reinterpret_cast(data.data()), + data.size()); + + kafka_value.deserialize(input_stream, api_version); + + std::vector output(data.size()); + boost::iostreams::stream output_stream(reinterpret_cast(output.data()), + output.size()); + kafka_value.serialize(output_stream, api_version); + + BOOST_REQUIRE(!output_stream.bad()); + + BOOST_REQUIRE_EQUAL(output_stream.tellp(), output.size()); + + BOOST_TEST(output == data, boost::test_tools::per_element()); +} + +template +void test_deserialize_throw(std::vector data, + KafkaType &kafka_value, int16_t api_version) { + boost::iostreams::stream input_stream(reinterpret_cast(data.data()), + data.size()); + + BOOST_REQUIRE_THROW(kafka_value.deserialize(input_stream, api_version), k4s::parsing_exception); +} + +BOOST_AUTO_TEST_CASE(kafka_primitives_number_test) { + k4s::kafka_number_t number(15); + BOOST_REQUIRE_EQUAL(*number, 15); + + test_deserialize_serialize({0x12, 0x34, 0x56, 0x78}, number, 0); + BOOST_REQUIRE_EQUAL(*number, 0x12345678); + + test_deserialize_throw({0x17, 0x27}, number, 0); + BOOST_REQUIRE_EQUAL(*number, 0x12345678); +} + +BOOST_AUTO_TEST_CASE(kafka_primitives_varint_test) { + k4s::kafka_varint_t number(155); + + test_deserialize_serialize({0x00}, number, 0); + BOOST_REQUIRE_EQUAL(*number, 0); + + test_deserialize_serialize({0x08}, number, 0); + BOOST_REQUIRE_EQUAL(*number, 4); + + test_deserialize_serialize({0x07}, number, 0); + BOOST_REQUIRE_EQUAL(*number, -4); + + test_deserialize_serialize({0xAC, 0x02}, number, 0); + BOOST_REQUIRE_EQUAL(*number, 150); + + test_deserialize_serialize({0xAB, 0x02}, number, 0); + BOOST_REQUIRE_EQUAL(*number, -150); + + test_deserialize_throw({0xAC}, number, 0); + BOOST_REQUIRE_EQUAL(*number, -150); + + test_deserialize_serialize({0xFF, 0xFF, 0xFF, 0xFF, 0xF}, number, 0); + BOOST_REQUIRE_EQUAL(*number, -2147483648); + + test_deserialize_throw({0xFF, 0xFF, 0xFF, 0xFF, 0x1F}, number, 0); + BOOST_REQUIRE_EQUAL(*number, -2147483648); +} + +BOOST_AUTO_TEST_CASE(kafka_primitives_string_test) { + k4s::kafka_string_t string("321"); + BOOST_REQUIRE_EQUAL(*string, "321"); + + test_deserialize_serialize({0, 5, 'a', 'b', 'c', 'd', 'e'}, string, 0); + BOOST_REQUIRE_EQUAL(*string, "abcde"); + BOOST_REQUIRE_EQUAL(string->size(), 5); + + test_deserialize_throw({0, 4, 'a', 'b', 'c'}, string, 0); + BOOST_REQUIRE_EQUAL(*string, "abcde"); + BOOST_REQUIRE_EQUAL(string->size(), 5); + + test_deserialize_throw({0}, string, 0); +} + +BOOST_AUTO_TEST_CASE(kafka_primitives_nullable_string_test) { + k4s::kafka_nullable_string_t string; + BOOST_REQUIRE(string.is_null()); + BOOST_REQUIRE_THROW((void) *string, std::exception); + + test_deserialize_serialize({0, 5, 'a', 'b', 'c', 'd', 'e'}, string, 0); + BOOST_REQUIRE_EQUAL(*string, "abcde"); + BOOST_REQUIRE_EQUAL(string->size(), 5); + + test_deserialize_serialize({0xFF, 0xFF}, string, 0); + BOOST_REQUIRE(string.is_null()); +} + +BOOST_AUTO_TEST_CASE(kafka_primitives_bytes_test) { + k4s::kafka_bytes_t bytes; + + test_deserialize_serialize({0, 0, 0, 5, 'a', 'b', 'c', 'd', 'e'}, bytes, 0); + BOOST_REQUIRE_EQUAL(*bytes, "abcde"); + BOOST_REQUIRE_EQUAL(bytes->size(), 5); +} + +BOOST_AUTO_TEST_CASE(kafka_primitives_array_test) { + k4s::kafka_array_t strings; + + test_deserialize_serialize({0, 0, 0, 2, 0, 5, 'a', 'b', 'c', 'd', 'e', 0, 2, 'f', 'g'}, strings, 0); + + BOOST_REQUIRE_EQUAL(strings->size(), 2); + BOOST_REQUIRE_EQUAL(*strings[0], "abcde"); + BOOST_REQUIRE_EQUAL(*strings[1], "fg"); + + test_deserialize_throw({0, 0, 0, 2, 0, 5, 'A', 'B', 'C', 'D', 'E', 0, 2, 'F'}, strings, 0); + BOOST_REQUIRE_EQUAL(strings->size(), 2); + BOOST_REQUIRE_EQUAL(*strings[0], "abcde"); + BOOST_REQUIRE_EQUAL(*strings[1], "fg"); +} + + +BOOST_AUTO_TEST_CASE(kafka_request_header_parsing_test) { + k4s::request_header header; + test_deserialize_serialize({ + 0x00, 0x05, 0x00, 0x01, 0x00, 0x00, 0x00, 0x42, 0x00, 0x05, 0x61, 0x62, 0x63, 0x64, 0x65 + }, header, 0); + + BOOST_REQUIRE_EQUAL(*header._api_key, 5); + BOOST_REQUIRE_EQUAL(*header._api_version, 1); + BOOST_REQUIRE_EQUAL(*header._correlation_id, 0x42); + BOOST_REQUIRE_EQUAL(*header._client_id, "abcde"); +} + +BOOST_AUTO_TEST_CASE(kafka_response_header_parsing_test) { + k4s::response_header header; + test_deserialize_serialize({ + 0x00, 0x05, 0x00, 0x01 + }, header, 0); + + BOOST_REQUIRE_EQUAL(*header._correlation_id, 0x50001); +} + +BOOST_AUTO_TEST_CASE(kafka_primitives_error_code_test) { + const k4s::error::kafka_error_code &error = + k4s::error::kafka_error_code::INVALID_FETCH_SIZE; + k4s::kafka_error_code_t error_code(error); + BOOST_REQUIRE_EQUAL((*error_code)._error_code, 4); + BOOST_REQUIRE_EQUAL(error_code == error, true); + BOOST_REQUIRE_EQUAL(error_code != error, false); + test_deserialize_serialize({0x00, 0x04}, error_code, 0); + BOOST_REQUIRE_EQUAL((*error_code)._error_code, 4); + test_deserialize_throw({0xAC, 0x02}, error_code, 0); +} + +BOOST_AUTO_TEST_CASE(kafka_api_versions_response_parsing_test) { + k4s::api_versions_response response; + test_deserialize_serialize({ + 0x00, 0x00, 0x00, 0x00, 0x00, 0x2d, 0x00, 0x00, 0x00, 0x00, 0x00, 0x07, 0x00, 0x01, 0x00, 0x00, + 0x00, 0x0b, 0x00, 0x02, 0x00, 0x00, 0x00, 0x05, 0x00, 0x03, 0x00, 0x00, 0x00, 0x08, 0x00, 0x04, + 0x00, 0x00, 0x00, 0x02, 0x00, 0x05, 0x00, 0x00, 0x00, 0x01, 0x00, 0x06, 0x00, 0x00, 0x00, 0x05, + 0x00, 0x07, 0x00, 0x00, 0x00, 0x02, 0x00, 0x08, 0x00, 0x00, 0x00, 0x07, 0x00, 0x09, 0x00, 0x00, + 0x00, 0x05, 0x00, 0x0a, 0x00, 0x00, 0x00, 0x02, 0x00, 0x0b, 0x00, 0x00, 0x00, 0x05, 0x00, 0x0c, + 0x00, 0x00, 0x00, 0x03, 0x00, 0x0d, 0x00, 0x00, 0x00, 0x02, 0x00, 0x0e, 0x00, 0x00, 0x00, 0x03, + 0x00, 0x0f, 0x00, 0x00, 0x00, 0x03, 0x00, 0x10, 0x00, 0x00, 0x00, 0x02, 0x00, 0x11, 0x00, 0x00, + 0x00, 0x01, 0x00, 0x12, 0x00, 0x00, 0x00, 0x02, 0x00, 0x13, 0x00, 0x00, 0x00, 0x03, 0x00, 0x14, + 0x00, 0x00, 0x00, 0x03, 0x00, 0x15, 0x00, 0x00, 0x00, 0x01, 0x00, 0x16, 0x00, 0x00, 0x00, 0x01, + 0x00, 0x17, 0x00, 0x00, 0x00, 0x03, 0x00, 0x18, 0x00, 0x00, 0x00, 0x01, 0x00, 0x19, 0x00, 0x00, + 0x00, 0x01, 0x00, 0x1a, 0x00, 0x00, 0x00, 0x01, 0x00, 0x1b, 0x00, 0x00, 0x00, 0x00, 0x00, 0x1c, + 0x00, 0x00, 0x00, 0x02, 0x00, 0x1d, 0x00, 0x00, 0x00, 0x01, 0x00, 0x1e, 0x00, 0x00, 0x00, 0x01, + 0x00, 0x1f, 0x00, 0x00, 0x00, 0x01, 0x00, 0x20, 0x00, 0x00, 0x00, 0x02, 0x00, 0x21, 0x00, 0x00, + 0x00, 0x01, 0x00, 0x22, 0x00, 0x00, 0x00, 0x01, 0x00, 0x23, 0x00, 0x00, 0x00, 0x01, 0x00, 0x24, + 0x00, 0x00, 0x00, 0x01, 0x00, 0x25, 0x00, 0x00, 0x00, 0x01, 0x00, 0x26, 0x00, 0x00, 0x00, 0x01, + 0x00, 0x27, 0x00, 0x00, 0x00, 0x01, 0x00, 0x28, 0x00, 0x00, 0x00, 0x01, 0x00, 0x29, 0x00, 0x00, + 0x00, 0x01, 0x00, 0x2a, 0x00, 0x00, 0x00, 0x01, 0x00, 0x2b, 0x00, 0x00, 0x00, 0x00, 0x00, 0x2c, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 + }, response, 2); + + BOOST_REQUIRE_EQUAL(*response._throttle_time_ms, 0); + BOOST_REQUIRE( + response._error_code == + k4s::error::kafka_error_code::NONE + ); + BOOST_REQUIRE_EQUAL(response._api_keys->size(), 45); + BOOST_REQUIRE_EQUAL(*response._api_keys[0]._api_key, 0); + BOOST_REQUIRE_EQUAL(*response._api_keys[0]._min_version, 0); + BOOST_REQUIRE_EQUAL(*response._api_keys[0]._max_version, 7); + BOOST_REQUIRE_EQUAL(*response._api_keys[1]._api_key, 1); + BOOST_REQUIRE_EQUAL(*response._api_keys[1]._min_version, 0); + BOOST_REQUIRE_EQUAL(*response._api_keys[1]._max_version, 11); +} + +BOOST_AUTO_TEST_CASE(kafka_metadata_request_parsing_test) { + k4s::metadata_request request; + test_deserialize_serialize({ + 0x00, 0x00, 0x00, 0x01, 0x00, 0x05, 0x74, 0x65, 0x73, 0x74, 0x35, 0x01, 0x00, 0x00 + }, request, 8); + + BOOST_REQUIRE_EQUAL(request._topics->size(), 1); + BOOST_REQUIRE_EQUAL(*request._topics[0]._name, "test5"); + BOOST_REQUIRE(*request._allow_auto_topic_creation); + BOOST_REQUIRE(!*request._include_cluster_authorized_operations); + BOOST_REQUIRE(!*request._include_topic_authorized_operations); +} + +BOOST_AUTO_TEST_CASE(kafka_metadata_response_parsing_test) { + k4s::metadata_response response; + test_deserialize_serialize({ + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x03, 0xe9, 0x00, 0x0a, 0x31, 0x37, + 0x32, 0x2e, 0x31, 0x33, 0x2e, 0x30, 0x2e, 0x31, 0x00, 0x00, 0x23, 0x84, 0xff, 0xff, 0x00, 0x16, + 0x6b, 0x4c, 0x5a, 0x35, 0x6a, 0x50, 0x76, 0x44, 0x52, 0x30, 0x43, 0x77, 0x31, 0x79, 0x34, 0x31, + 0x41, 0x66, 0x35, 0x48, 0x55, 0x67, 0x00, 0x00, 0x03, 0xe9, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, + 0x00, 0x05, 0x74, 0x65, 0x73, 0x74, 0x35, 0x00, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x03, 0xe9, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, + 0x03, 0xe9, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x03, 0xe9, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 + }, response, 8); + + BOOST_REQUIRE_EQUAL(*response._throttle_time_ms, 0); + BOOST_REQUIRE_EQUAL(response._brokers->size(), 1); + BOOST_REQUIRE_EQUAL(*response._brokers[0]._node_id, 0x3e9); + BOOST_REQUIRE_EQUAL(*response._brokers[0]._host, "172.13.0.1"); + BOOST_REQUIRE_EQUAL(*response._brokers[0]._port, 0x2384); + BOOST_REQUIRE(response._brokers[0]._rack.is_null()); + BOOST_REQUIRE_EQUAL(*response._cluster_id, "kLZ5jPvDR0Cw1y41Af5HUg"); + BOOST_REQUIRE_EQUAL(*response._controller_id, 0x3e9); + BOOST_REQUIRE_EQUAL(response._topics->size(), 1); + BOOST_REQUIRE( + response._topics[0]._error_code == + k4s::error::kafka_error_code::NONE + ); + BOOST_REQUIRE_EQUAL(*response._topics[0]._name, "test5"); + BOOST_REQUIRE(!*response._topics[0]._is_internal); + BOOST_REQUIRE_EQUAL(response._topics[0]._partitions->size(), 1); + BOOST_REQUIRE( + response._topics[0]._partitions[0]._error_code == + k4s::error::kafka_error_code::NONE + ); + BOOST_REQUIRE_EQUAL(*response._topics[0]._partitions[0]._partition_index, 0); + BOOST_REQUIRE_EQUAL(*response._topics[0]._partitions[0]._leader_id, 0x3e9); + BOOST_REQUIRE_EQUAL(*response._topics[0]._partitions[0]._leader_epoch, 0); + BOOST_REQUIRE_EQUAL(response._topics[0]._partitions[0]._replica_nodes->size(), 1); + BOOST_REQUIRE_EQUAL(*response._topics[0]._partitions[0]._replica_nodes[0], 0x3e9); + BOOST_REQUIRE_EQUAL(response._topics[0]._partitions[0]._isr_nodes->size(), 1); + BOOST_REQUIRE_EQUAL(*response._topics[0]._partitions[0]._isr_nodes[0], 0x3e9); + BOOST_REQUIRE_EQUAL(*response._topics[0]._topic_authorized_operations, 0); + BOOST_REQUIRE_EQUAL(*response._cluster_authorized_operations, 0); +} + +BOOST_AUTO_TEST_CASE(kafka_record_parsing_test) { + k4s::kafka_record record; + test_deserialize_serialize({ + 0x20, 0x00, 0x00, 0x00, 0x08, 0x00, 0x00, 0x00, 0x01, 0x0c, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00 + }, record, 0); + BOOST_REQUIRE_EQUAL(*record._timestamp_delta, 0); + BOOST_REQUIRE_EQUAL(*record._offset_delta, 0); + std::string expected_key{"\x00\x00\x00\x01", 4}; + BOOST_REQUIRE_EQUAL(record._key, expected_key); + std::string expected_value{"\x00\x00\x00\x00\x00\x00", 6}; + BOOST_REQUIRE_EQUAL(record._value, expected_value); + BOOST_REQUIRE_EQUAL(record._headers.size(), 0); + + k4s::kafka_record record2; + test_deserialize_serialize({ + 0x10, 0x00, 0x00, 0x00, 0x02, 0x34, 0x02, 0x36, 0x00 + }, record2, 0); + BOOST_REQUIRE_EQUAL(*record2._timestamp_delta, 0); + BOOST_REQUIRE_EQUAL(*record2._offset_delta, 0); + BOOST_REQUIRE_EQUAL(record2._key, "4"); + BOOST_REQUIRE_EQUAL(record2._value, "6"); + BOOST_REQUIRE_EQUAL(record2._headers.size(), 0); +} + +BOOST_AUTO_TEST_CASE(kafka_record_batch_parsing_test) { + k4s::kafka_record_batch batch; + test_deserialize_serialize({ + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x04, 0x00, 0x00, 0x00, 0x3a, 0x00, 0x00, 0x00, 0x00, + 0x02, 0x6f, 0x51, 0x95, 0x17, 0x00, 0x10, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x6e, 0xb3, + 0x2b, 0x03, 0x41, 0x00, 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x03, 0x41, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x03, 0x00, 0x00, 0x00, 0x01, 0x10, 0x00, 0x00, + 0x00, 0x02, 0x34, 0x02, 0x34, 0x00 + }, batch, 0); + + BOOST_REQUIRE_EQUAL(*batch._base_offset, 4); + BOOST_REQUIRE_EQUAL(*batch._partition_leader_epoch, 0); + BOOST_REQUIRE_EQUAL(*batch._magic, 2); + BOOST_REQUIRE(batch._compression_type == k4s::kafka_record_compression_type::NO_COMPRESSION); + BOOST_REQUIRE(batch._timestamp_type == k4s::kafka_record_timestamp_type::CREATE_TIME); + BOOST_REQUIRE(batch._is_transactional); + BOOST_REQUIRE(!batch._is_control_batch); + BOOST_REQUIRE_EQUAL(*batch._first_timestamp, 0x16eb32b0341); + BOOST_REQUIRE_EQUAL(*batch._producer_id, 0); + BOOST_REQUIRE_EQUAL(*batch._producer_epoch, 0); + BOOST_REQUIRE_EQUAL(*batch._base_sequence, 3); + BOOST_REQUIRE_EQUAL(batch._records.size(), 1); + BOOST_REQUIRE_EQUAL(*batch._records[0]._timestamp_delta, 0); + BOOST_REQUIRE_EQUAL(*batch._records[0]._offset_delta, 0); + BOOST_REQUIRE_EQUAL(batch._records[0]._key, "4"); + BOOST_REQUIRE_EQUAL(batch._records[0]._value, "4"); + BOOST_REQUIRE_EQUAL(batch._records[0]._headers.size(), 0); +} + +BOOST_AUTO_TEST_CASE(kafka_records_parsing_test) { + k4s::kafka_records records; + test_deserialize_serialize({ + 0x00, 0x00, 0x02, 0x8e, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x3a, + 0x00, 0x00, 0x00, 0x00, 0x02, 0xc6, 0x4c, 0x35, 0x56, 0x00, 0x10, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x01, 0x4b, 0x00, 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x01, 0x4b, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x01, 0x10, 0x00, 0x00, 0x00, 0x02, 0x31, 0x02, 0x31, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x01, 0x00, 0x00, 0x00, 0x3a, 0x00, 0x00, 0x00, 0x00, 0x02, 0x90, 0xe7, 0x99, 0x55, 0x00, + 0x10, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x02, 0x0a, 0x00, 0x00, 0x01, + 0x6e, 0xb3, 0x2b, 0x02, 0x0a, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x01, 0x10, 0x00, 0x00, 0x00, 0x02, 0x32, 0x02, 0x32, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0x00, 0x00, 0x00, 0x42, 0x00, 0x00, 0x00, 0x00, + 0x02, 0xb2, 0x80, 0xcd, 0x9a, 0x00, 0x30, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x6e, 0xb3, + 0x2b, 0x02, 0xbb, 0x00, 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x02, 0xbb, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, 0xff, 0xff, 0xff, 0x00, 0x00, 0x00, 0x01, 0x20, 0x00, 0x00, + 0x00, 0x08, 0x00, 0x00, 0x00, 0x01, 0x0c, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x03, 0x00, 0x00, 0x00, 0x3a, 0x00, 0x00, 0x00, 0x00, 0x02, 0x25, + 0x73, 0x58, 0xe3, 0x00, 0x10, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x02, + 0xf0, 0x00, 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x02, 0xf0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0x00, 0x00, 0x00, 0x01, 0x10, 0x00, 0x00, 0x00, 0x02, + 0x33, 0x02, 0x33, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x04, 0x00, 0x00, 0x00, 0x3a, + 0x00, 0x00, 0x00, 0x00, 0x02, 0x6f, 0x51, 0x95, 0x17, 0x00, 0x10, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x03, 0x41, 0x00, 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x03, 0x41, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x03, 0x00, 0x00, 0x00, + 0x01, 0x10, 0x00, 0x00, 0x00, 0x02, 0x34, 0x02, 0x34, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x05, 0x00, 0x00, 0x00, 0x42, 0x00, 0x00, 0x00, 0x00, 0x02, 0xfb, 0x4e, 0xb4, 0x07, 0x00, + 0x30, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x03, 0x96, 0x00, 0x00, 0x01, + 0x6e, 0xb3, 0x2b, 0x03, 0x96, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, + 0xff, 0xff, 0xff, 0x00, 0x00, 0x00, 0x01, 0x20, 0x00, 0x00, 0x00, 0x08, 0x00, 0x00, 0x00, 0x01, + 0x0c, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x06, + 0x00, 0x00, 0x00, 0x3a, 0x00, 0x00, 0x00, 0x00, 0x02, 0xa0, 0xdd, 0x37, 0x0b, 0x00, 0x10, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x03, 0xf5, 0x00, 0x00, 0x01, 0x6e, 0xb3, + 0x2b, 0x03, 0xf5, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x04, 0x00, 0x00, 0x00, 0x01, 0x10, 0x00, 0x00, 0x00, 0x02, 0x35, 0x02, 0x35, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x07, 0x00, 0x00, 0x00, 0x3a, 0x00, 0x00, 0x00, 0x00, 0x02, 0x8b, + 0x5e, 0xf1, 0x92, 0x00, 0x10, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x04, + 0x28, 0x00, 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x04, 0x28, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x05, 0x00, 0x00, 0x00, 0x01, 0x10, 0x00, 0x00, 0x00, 0x02, + 0x36, 0x02, 0x36, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x08, 0x00, 0x00, 0x00, 0x42, + 0x00, 0x00, 0x00, 0x00, 0x02, 0xa3, 0x73, 0x3e, 0xe0, 0x00, 0x30, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x04, 0x6a, 0x00, 0x00, 0x01, 0x6e, 0xb3, 0x2b, 0x04, 0x6a, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, 0xff, 0xff, 0xff, 0x00, 0x00, 0x00, + 0x01, 0x20, 0x00, 0x00, 0x00, 0x08, 0x00, 0x00, 0x00, 0x00, 0x0c, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00 + }, records, 0); + + BOOST_REQUIRE_EQUAL(records._record_batches.size(), 9); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._base_offset, 0); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._partition_leader_epoch, 0); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._magic, 2); + BOOST_REQUIRE(records._record_batches[0]._compression_type == k4s::kafka_record_compression_type::NO_COMPRESSION); + BOOST_REQUIRE(records._record_batches[0]._timestamp_type == k4s::kafka_record_timestamp_type::CREATE_TIME); + BOOST_REQUIRE(records._record_batches[0]._is_transactional); + BOOST_REQUIRE(!records._record_batches[0]._is_control_batch); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._first_timestamp, 0x16eb32b014b); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._producer_id, 0); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._producer_epoch, 0); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._base_sequence, 0); + BOOST_REQUIRE_EQUAL(records._record_batches[0]._records.size(), 1); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._records[0]._timestamp_delta, 0); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._records[0]._offset_delta, 0); + BOOST_REQUIRE_EQUAL(records._record_batches[0]._records[0]._key, "1"); + BOOST_REQUIRE_EQUAL(records._record_batches[0]._records[0]._value, "1"); + BOOST_REQUIRE_EQUAL(records._record_batches[0]._records[0]._headers.size(), 0); + BOOST_REQUIRE_EQUAL(*records._record_batches[2]._base_offset, 2); + BOOST_REQUIRE_EQUAL(*records._record_batches[2]._partition_leader_epoch, 0); + BOOST_REQUIRE_EQUAL(*records._record_batches[2]._magic, 2); + BOOST_REQUIRE(records._record_batches[2]._compression_type == k4s::kafka_record_compression_type::NO_COMPRESSION); + BOOST_REQUIRE(records._record_batches[2]._timestamp_type == k4s::kafka_record_timestamp_type::CREATE_TIME); + BOOST_REQUIRE(records._record_batches[2]._is_transactional); + BOOST_REQUIRE(records._record_batches[2]._is_control_batch); + BOOST_REQUIRE_EQUAL(*records._record_batches[2]._first_timestamp, 0x16eb32b02bb); + BOOST_REQUIRE_EQUAL(*records._record_batches[2]._producer_id, 0); + BOOST_REQUIRE_EQUAL(*records._record_batches[2]._producer_epoch, 0); + BOOST_REQUIRE_EQUAL(*records._record_batches[2]._base_sequence, -1); + BOOST_REQUIRE_EQUAL(records._record_batches[2]._records.size(), 1); + BOOST_REQUIRE_EQUAL(*records._record_batches[2]._records[0]._timestamp_delta, 0); + BOOST_REQUIRE_EQUAL(*records._record_batches[2]._records[0]._offset_delta, 0); + std::string expected_key{"\x00\x00\x00\x01", 4}; + std::string expected_value{"\x00\x00\x00\x00\x00\x00", 6}; + BOOST_REQUIRE_EQUAL(records._record_batches[2]._records[0]._key, expected_key); + BOOST_REQUIRE_EQUAL(records._record_batches[2]._records[0]._value, expected_value); + BOOST_REQUIRE_EQUAL(records._record_batches[2]._records[0]._headers.size(), 0); +} + +BOOST_AUTO_TEST_CASE(kafka_produce_request_parsing_test) { + k4s::produce_request request; + test_deserialize_serialize({ + 0xff, 0xff, 0xff, 0xff, 0x00, 0x00, 0x75, 0x30, 0x00, 0x00, 0x00, 0x01, 0x00, 0x05, 0x74, 0x65, + 0x73, 0x74, 0x35, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x46, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x3a, 0xff, 0xff, 0xff, 0xff, 0x02, + 0x06, 0x76, 0x5e, 0x6f, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x6e, 0x5b, 0x6e, + 0xba, 0x2c, 0x00, 0x00, 0x01, 0x6e, 0x5b, 0x6e, 0xba, 0x2c, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, + 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x00, 0x00, 0x00, 0x01, 0x10, 0x00, 0x00, 0x00, + 0x02, 0x30, 0x02, 0x30, 0x00 + }, request, 7); + + BOOST_REQUIRE(request._transactional_id.is_null()); + BOOST_REQUIRE_EQUAL(*request._acks, -1); + BOOST_REQUIRE_EQUAL(*request._timeout_ms, 30000); + BOOST_REQUIRE_EQUAL(request._topics->size(), 1); + BOOST_REQUIRE_EQUAL(*request._topics[0]._name, "test5"); + BOOST_REQUIRE_EQUAL(request._topics[0]._partitions->size(), 1); + BOOST_REQUIRE_EQUAL(*request._topics[0]._partitions[0]._partition_index, 0); + const auto &records = request._topics[0]._partitions[0]._records; + BOOST_REQUIRE_EQUAL(records._record_batches.size(), 1); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._base_offset, 0); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._partition_leader_epoch, -1); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._magic, 2); + BOOST_REQUIRE(records._record_batches[0]._compression_type == k4s::kafka_record_compression_type::NO_COMPRESSION); + BOOST_REQUIRE(records._record_batches[0]._timestamp_type == k4s::kafka_record_timestamp_type::CREATE_TIME); + BOOST_REQUIRE(!records._record_batches[0]._is_transactional); + BOOST_REQUIRE(!records._record_batches[0]._is_control_batch); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._first_timestamp, 0x16e5b6eba2c); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._producer_id, -1); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._producer_epoch, -1); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._base_sequence, -1); + BOOST_REQUIRE_EQUAL(records._record_batches[0]._records.size(), 1); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._records[0]._timestamp_delta, 0); + BOOST_REQUIRE_EQUAL(*records._record_batches[0]._records[0]._offset_delta, 0); + BOOST_REQUIRE_EQUAL(records._record_batches[0]._records[0]._key, "0"); + BOOST_REQUIRE_EQUAL(records._record_batches[0]._records[0]._value, "0"); + BOOST_REQUIRE_EQUAL(records._record_batches[0]._records[0]._headers.size(), 0); +} + +BOOST_AUTO_TEST_CASE(kafka_produce_response_parsing_test) { + k4s::produce_response response; + test_deserialize_serialize({ + 0x00, 0x00, 0x00, 0x01, 0x00, 0x05, 0x74, 0x65, 0x73, 0x74, 0x35, 0x00, 0x00, 0x00, 0x01, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x46, 0xff, 0xff, 0xff, + 0xff, 0xff, 0xff, 0xff, 0xff, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00 + }, response, 7); + + BOOST_REQUIRE_EQUAL(response._responses->size(), 1); + BOOST_REQUIRE_EQUAL(*response._throttle_time_ms, 0); + + const auto &inner_response = response._responses[0]; + BOOST_REQUIRE_EQUAL(*inner_response._name, "test5"); + BOOST_REQUIRE_EQUAL(inner_response._partitions->size(), 1); + + const auto &partition = inner_response._partitions[0]; + BOOST_REQUIRE_EQUAL(*partition._partition_index, 0); + BOOST_REQUIRE( + partition._error_code + == k4s::error::kafka_error_code::NONE + ); + BOOST_REQUIRE_EQUAL(*partition._base_offset, 0x46); + BOOST_REQUIRE_EQUAL(*partition._log_append_time_ms, -1); + BOOST_REQUIRE_EQUAL(*partition._log_start_offset, 0); +} diff --git a/tests/unit/kafka_retry_helper_test.cc b/tests/unit/kafka_retry_helper_test.cc new file mode 100644 index 0000000..53b4f61 --- /dev/null +++ b/tests/unit/kafka_retry_helper_test.cc @@ -0,0 +1,87 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Copyright (C) 2019 ScyllaDB Ltd. + */ + +#include +#include +#include + +#include +#include + +using namespace seastar; +namespace k4s = kafka4seastar; + +SEASTAR_THREAD_TEST_CASE(kafka_retry_helper_test_early_stop) { + k4s::retry_helper helper(5, k4s::defaults::exp_retry_backoff(1, 1000)); + auto retry_count = 0; + auto data = 0; + helper.with_retry([&retry_count, data]() mutable { + retry_count++; + data++; + if (data >= 3) { + return k4s::do_retry::no; + } + return k4s::do_retry::yes; + }).wait(); + BOOST_REQUIRE_EQUAL(retry_count, 3); +} + +SEASTAR_THREAD_TEST_CASE(kafka_retry_helper_test_capped_retries) { + k4s::retry_helper helper(5, k4s::defaults::exp_retry_backoff(1, 1000)); + auto retry_count = 0; + helper.with_retry([&retry_count] { + retry_count++; + return k4s::do_retry::yes; + }).wait(); + BOOST_REQUIRE_EQUAL(retry_count, 5); +} + +SEASTAR_THREAD_TEST_CASE(kafka_retry_helper_test_future) { + k4s::retry_helper helper(5, k4s::defaults::exp_retry_backoff(1, 1000)); + auto retry_count = 0; + helper.with_retry([&retry_count] { + retry_count++; + return make_ready_future(k4s::do_retry::yes); + }).wait(); + BOOST_REQUIRE_EQUAL(retry_count, 5); +} + +SEASTAR_THREAD_TEST_CASE(kafka_retry_helper_test_modify_data) { + k4s::retry_helper helper(5, k4s::defaults::exp_retry_backoff(1, 1000)); + auto retry_count = 0; + std::vector data{1, 2, 3}; + std::vector retry_data; + + helper.with_retry([data = std::move(data), &retry_count, &retry_data]() mutable { + if (data.empty()) { + return k4s::do_retry::no; + } + retry_data.push_back(data.back()); + data.pop_back(); + retry_count++; + return k4s::do_retry::yes; + }).wait(); + + BOOST_REQUIRE_EQUAL(retry_count, 3); + std::vector expected_data{3, 2, 1}; + BOOST_TEST(retry_data == expected_data, boost::test_tools::per_element()); +} diff --git a/utility/kafkadev_local/.gitignore b/utility/kafkadev_local/.gitignore new file mode 100644 index 0000000..b68f0e5 --- /dev/null +++ b/utility/kafkadev_local/.gitignore @@ -0,0 +1,3 @@ +*.tfstate +*.tfstate.backup +.terraform/ diff --git a/utility/kafkadev_local/README.md b/utility/kafkadev_local/README.md new file mode 100644 index 0000000..f8b23cc --- /dev/null +++ b/utility/kafkadev_local/README.md @@ -0,0 +1,94 @@ +# Kafkadev (local) + +Dev/test local deployment of Kafka (and Zookeeper) using Docker. + +## Dependencies +- Docker +- Terraform v0.12 + +### Installing Docker +
+# Install and start Docker
+
+$ sudo dnf install docker
+$ sudo systemctl start docker
+
+# Add current user to docker group
+
+$ sudo groupadd docker
+$ sudo usermod -aG docker $USER
+$ newgrp docker 
+
+ +### Installing Terraform +
$ wget https://releases.hashicorp.com/terraform/0.12.12/terraform_0.12.12_linux_amd64.zip
+$ unzip terraform_0.12.12_linux_amd64.zip
+$ sudo cp terraform /usr/bin/
+
+ +## Usage +### Starting cluster +The following example creates a Kafka cluster with 5 brokers and a configured Zookeeper instance in the `172.13.0.0/16` network. + +The IP addresses of started Kafka brokers are printed as `kafka_addr` and the names of Docker containers are exported as `kafka_name`. + +Run the commands in the `kafkadev_local` directory: + +
+$ terraform init
+$ terraform apply
+
+var.kafka_count
+  The number of started Kafka brokers.
+
+  Enter a value: 5
+
+var.network_cidr
+  The IPv4 network prefix for started containers, written in CIDR format, e.g. 172.13.0.0/16.
+
+  Enter a value: 172.13.0.0/16
+
+[...]
+
+Do you want to perform these actions?
+  Terraform will perform the actions described above.
+  Only 'yes' will be accepted to approve.
+
+  Enter a value: yes
+
+[...]
+
+Apply complete! Resources: 10 added, 0 changed, 0 destroyed.
+
+Outputs:
+
+kafka_addr = [
+  "172.13.0.1",
+  "172.13.0.2",
+  "172.13.0.3",
+  "172.13.0.4",
+  "172.13.0.5",
+]
+kafka_name = [
+  "kafkadev_6b34_kafka1",
+  "kafkadev_6b34_kafka2",
+  "kafkadev_6b34_kafka3",
+  "kafkadev_6b34_kafka4",
+  "kafkadev_6b34_kafka5",
+]
+zookeeper_addr = 172.13.0.6
+zookeeper_name = kafkadev_6b34_zookeeper
+
+### Stopping cluster +Use the following command to stop and remove the Kafka cluster: + +
+$ terraform destroy
+[...]
+
+Do you really want to destroy all resources?
+  Terraform will destroy all your managed infrastructure, as shown above.
+  There is no undo. Only 'yes' will be accepted to confirm.
+
+  Enter a value: yes
+
diff --git a/utility/kafkadev_local/config.tf b/utility/kafkadev_local/config.tf new file mode 100644 index 0000000..7b12add --- /dev/null +++ b/utility/kafkadev_local/config.tf @@ -0,0 +1,17 @@ +provider "docker" { + host = "unix:///var/run/docker.sock" +} + +variable "network_cidr" { + type = string + description = "The IPv4 network prefix for started containers, written in CIDR format, e.g. 172.13.0.0/16." +} + +variable "kafka_count" { + type = number + description = "The number of started Kafka brokers." +} + +resource "random_id" "deployment_id" { + byte_length = 2 +} diff --git a/utility/kafkadev_local/kafka.tf b/utility/kafkadev_local/kafka.tf new file mode 100644 index 0000000..ea46165 --- /dev/null +++ b/utility/kafkadev_local/kafka.tf @@ -0,0 +1,39 @@ +data "docker_registry_image" "kafka" { + name = "wurstmeister/kafka:latest" +} + +resource "docker_image" "kafka" { + name = "docker.io/${data.docker_registry_image.kafka.name}" + pull_triggers = [data.docker_registry_image.kafka.sha256_digest] + keep_locally = true +} + +resource "docker_container" "kafka" { + count = var.kafka_count + + name = "kafkadev_${random_id.deployment_id.hex}_kafka${count.index + 1}" + image = docker_image.kafka.latest + + env = [ + "KAFKA_ADVERTISED_PORT=9092", + "KAFKA_ADVERTISED_HOST_NAME=${cidrhost(var.network_cidr, count.index + 1)}", + "KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://${cidrhost(var.network_cidr, count.index + 1)}:9092", + "KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT", + "KAFKA_ZOOKEEPER_CONNECT=${docker_container.zookeeper.network_data[0].ip_address}:2181" + ] + + networks_advanced { + name = docker_network.network.name + ipv4_address = cidrhost(var.network_cidr, count.index + 1) + } +} + +output "kafka_addr" { + value = docker_container.kafka[*].network_data[0].ip_address + description = "The IP addresses of started Kafka brokers." +} + +output "kafka_name" { + value = docker_container.kafka[*].name + description = "The names of Kafka Docker containers." +} diff --git a/utility/kafkadev_local/network.tf b/utility/kafkadev_local/network.tf new file mode 100644 index 0000000..5c2397e --- /dev/null +++ b/utility/kafkadev_local/network.tf @@ -0,0 +1,9 @@ +resource "docker_network" "network" { + name = "kafkadev_${random_id.deployment_id.hex}_network" + driver = "bridge" + + ipam_config { + subnet = var.network_cidr + gateway = cidrhost(var.network_cidr, var.kafka_count + 2) + } +} diff --git a/utility/kafkadev_local/zookeeper.tf b/utility/kafkadev_local/zookeeper.tf new file mode 100644 index 0000000..3f2cc18 --- /dev/null +++ b/utility/kafkadev_local/zookeeper.tf @@ -0,0 +1,29 @@ +data "docker_registry_image" "zookeeper" { + name = "wurstmeister/zookeeper:latest" +} + +resource "docker_image" "zookeeper" { + name = "docker.io/${data.docker_registry_image.zookeeper.name}" + pull_triggers = [data.docker_registry_image.zookeeper.sha256_digest] + keep_locally = true +} + +resource "docker_container" "zookeeper" { + name = "kafkadev_${random_id.deployment_id.hex}_zookeeper" + image = docker_image.zookeeper.latest + + networks_advanced { + name = docker_network.network.name + ipv4_address = cidrhost(var.network_cidr, var.kafka_count + 1) + } +} + +output "zookeeper_addr" { + value = docker_container.zookeeper.network_data[0].ip_address + description = "The IP address of Zookeeper." +} + +output "zookeeper_name" { + value = docker_container.zookeeper.name + description = "The name of Zookeeper Docker containers." +}