├── tla └── OsirisMsgPassing.toolbox │ ├── .settings │ └── org.lamport.tla.toolbox.prefs │ ├── .project │ ├── OsirisSafety │ ├── MC.cfg │ └── MC.tla │ └── OsirisMsgPassing___OsirisSafety.launch ├── .gitignore ├── LICENSE ├── src ├── osiris.app.src ├── osiris_peer_shim.hrl ├── osiris_replica_reader_sup.erl ├── osiris_app.erl ├── osiris_member.erl ├── osiris_counters.erl ├── osiris_sup.erl ├── osiris_log_shared.erl ├── osiris_server_sup.erl ├── osiris_ets.erl ├── osiris.hrl ├── osiris_retention.erl ├── osiris_bench.erl ├── osiris_bloom.erl ├── osiris_tracking.erl ├── osiris.erl ├── osiris_util.erl ├── osiris_replica_reader.erl ├── osiris_writer.erl └── osiris_replica.erl ├── rebar.config ├── rebar.lock ├── .github └── workflows │ ├── erlang.yml │ └── release.yml ├── test ├── osiris_util_SUITE_data │ ├── inter_node_tls_client_server_ok.config │ ├── inter_node_tls_server_client_ok.config │ └── inter_node_tls_broken.config ├── osiris_tracer.erl ├── osiris_bloom_SUITE.erl ├── osiris_tracking_SUITE.erl └── osiris_util_SUITE.erl ├── README.md ├── CONTRIBUTING.md ├── Makefile ├── LICENSE-APACHE2 └── LICENSE-MPL-RabbitMQ /tla/OsirisMsgPassing.toolbox/.settings/org.lamport.tla.toolbox.prefs: -------------------------------------------------------------------------------- 1 | ProjectRootFile=PARENT-1-PROJECT_LOC/OsirisMsgPassing.tla 2 | eclipse.preferences.version=1 3 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | erl_crash.dump 2 | .sw? 3 | .*.sw? 4 | *.beam 5 | /.erlang.mk/ 6 | /cover/ 7 | /deps/ 8 | /ebin/ 9 | /logs/ 10 | /plugins/ 11 | /xrefr 12 | elvis 13 | callgrind* 14 | ct.coverdata 15 | test/ct.cover.spec 16 | _build 17 | 18 | osiris.d 19 | *.plt 20 | *.d 21 | 22 | *.jar 23 | 24 | /.vscode/ 25 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | This package, Osiris, is dual-licensed under 2 | the Apache License v2 and the Mozilla Public License v2.0. 3 | 4 | For the Apache License, please see the file LICENSE-APACHE2. 5 | 6 | For the Mozilla Public License, please see the file LICENSE-MPL-RabbitMQ. 7 | 8 | For attribution of copyright and other details of provenance, please 9 | refer to the source code. 10 | 11 | If you have any questions regarding licensing, please contact us at 12 | info@rabbitmq.com. 13 | -------------------------------------------------------------------------------- /src/osiris.app.src: -------------------------------------------------------------------------------- 1 | {application,osiris, 2 | [{description,"Foundation of the log-based streaming subsystem for RabbitMQ"}, 3 | {vsn,"1.10.2"}, 4 | {licenses,["Apache-2.0","MPL-2.0"]}, 5 | {links,[{"github","https://github.com/rabbitmq/osiris"}]}, 6 | {modules,[]}, 7 | {registered,[ra_sup]}, 8 | {applications,[kernel,stdlib,sasl,crypto,gen_batch_server, 9 | seshat]}, 10 | {mod,{osiris_app,[]}}, 11 | {env,[{data_dir, "/tmp/osiris"}]}]}. 12 | -------------------------------------------------------------------------------- /src/osiris_peer_shim.hrl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -if(?OTP_RELEASE >= 25). 9 | -define(PEER_MODULE, peer). 10 | -else. 11 | -define(PEER_MODULE, slave). 12 | -endif. -------------------------------------------------------------------------------- /rebar.config: -------------------------------------------------------------------------------- 1 | {project_plugins, [rebar3_format]}. 2 | 3 | {deps, [ 4 | {gen_batch_server, "0.8.9"}, 5 | {seshat, "1.0.1"} 6 | 7 | ]}. 8 | 9 | {format, [ 10 | {files, ["src/*.erl", "test/*.erl"]}, 11 | {formatter, default_formatter}, 12 | {options, #{ 13 | paper => 80, 14 | ribbon => 70, 15 | inline_attributes => {when_under, 1}, 16 | inline_items => {when_under, 4} 17 | }} 18 | ]}. 19 | 20 | {dialyzer, [ 21 | {plt_extra_apps, [ssl]} 22 | ]}. 23 | -------------------------------------------------------------------------------- /rebar.lock: -------------------------------------------------------------------------------- 1 | {"1.2.0", 2 | [{<<"gen_batch_server">>,{pkg,<<"gen_batch_server">>,<<"0.8.9">>},0}, 3 | {<<"seshat">>,{pkg,<<"seshat">>,<<"1.0.1">>},0}]}. 4 | [ 5 | {pkg_hash,[ 6 | {<<"gen_batch_server">>, <<"1C6BC0F530BF8C17E8B4ACC20C2CC369FFA5BEE2B46DE01E21410745F24B1BC9">>}, 7 | {<<"seshat">>, <<"FA7A8E89218D19394F7DDC47BA6725471103D654CD0BD0A98E5FDD922A943EAC">>}]}, 8 | {pkg_hash_ext,[ 9 | {<<"gen_batch_server">>, <<"C8581FE4A4B6BCCF91E53CE6A8C7E6C27C8C591BAB5408B160166463F5579C22">>}, 10 | {<<"seshat">>, <<"38324FE8C5782C69D73B334DD00B20E16C0D99EEF3E7B4005C519FE9BC0E34FC">>}]} 11 | ]. 12 | -------------------------------------------------------------------------------- /src/osiris_replica_reader_sup.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_replica_reader_sup). 9 | 10 | -behaviour(supervisor). 11 | 12 | -export([start_link/0]). 13 | -export([init/1]). 14 | 15 | start_link() -> 16 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 17 | 18 | init([]) -> 19 | Procs = [], 20 | {ok, {{one_for_one, 1, 5}, Procs}}. 21 | -------------------------------------------------------------------------------- /src/osiris_app.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_app). 9 | 10 | -behaviour(application). 11 | 12 | -export([start/2]). 13 | -export([stop/1]). 14 | 15 | start(_Type, _Args) -> 16 | Logger = application:get_env(osiris, logger_module, logger), 17 | osiris:configure_logger(Logger), 18 | osiris_sup:start_link(). 19 | 20 | stop(_State) -> 21 | ok. 22 | -------------------------------------------------------------------------------- /.github/workflows/erlang.yml: -------------------------------------------------------------------------------- 1 | name: Erlang CI 2 | 3 | on: 4 | push: 5 | branches: 6 | - main 7 | pull_request: 8 | branches: 9 | - main 10 | 11 | jobs: 12 | 13 | build: 14 | runs-on: ubuntu-latest 15 | strategy: 16 | fail-fast: false 17 | matrix: 18 | erlang_version: 19 | - "26" 20 | - "27" 21 | - "28" 22 | steps: 23 | - name: Clone the repository 24 | uses: actions/checkout@v2 25 | - name: Set up Erlang/OTP 26 | uses: erlef/setup-beam@v1 27 | with: 28 | otp-version: ${{ matrix.erlang_version }} 29 | - name: Build 30 | run: make 31 | - name: Run tests 32 | run: make check 33 | -------------------------------------------------------------------------------- /test/osiris_util_SUITE_data/inter_node_tls_client_server_ok.config: -------------------------------------------------------------------------------- 1 | [ 2 | {client, [ 3 | {cacertfile, "/etc/rabbitmq/ca_certificate.pem"}, 4 | {certfile, "/etc/rabbitmq/client_certificate.pem"}, 5 | {keyfile, "/etc/rabbitmq/client_key.pem"}, 6 | {secure_renegotiate, true}, 7 | {verify, verify_peer}, 8 | {fail_if_no_peer_cert, true} 9 | ]}, 10 | {server, [ 11 | {cacertfile, "/etc/rabbitmq/ca_certificate.pem"}, 12 | {certfile, "/etc/rabbitmq/server_certificate.pem"}, 13 | {keyfile, "/etc/rabbitmq/server_key.pem"}, 14 | {secure_renegotiate, true}, 15 | {verify, verify_peer}, 16 | {fail_if_no_peer_cert, true} 17 | ]} 18 | ]. -------------------------------------------------------------------------------- /test/osiris_util_SUITE_data/inter_node_tls_server_client_ok.config: -------------------------------------------------------------------------------- 1 | [ 2 | {server, [ 3 | {cacertfile, "/etc/rabbitmq/ca_certificate.pem"}, 4 | {certfile, "/etc/rabbitmq/server_certificate.pem"}, 5 | {keyfile, "/etc/rabbitmq/server_key.pem"}, 6 | {secure_renegotiate, true}, 7 | {verify, verify_peer}, 8 | {fail_if_no_peer_cert, true} 9 | ]}, 10 | {client, [ 11 | {cacertfile, "/etc/rabbitmq/ca_certificate.pem"}, 12 | {certfile, "/etc/rabbitmq/client_certificate.pem"}, 13 | {keyfile, "/etc/rabbitmq/client_key.pem"}, 14 | {secure_renegotiate, true}, 15 | {verify, verify_peer}, 16 | {fail_if_no_peer_cert, true} 17 | ]} 18 | ]. -------------------------------------------------------------------------------- /tla/OsirisMsgPassing.toolbox/.project: -------------------------------------------------------------------------------- 1 | 2 | 3 | OsirisMsgPassing 4 | 5 | 6 | 7 | 8 | 9 | toolbox.builder.TLAParserBuilder 10 | 11 | 12 | 13 | 14 | 15 | toolbox.natures.TLANature 16 | 17 | 18 | 19 | OsirisMsgPassing.tla 20 | 1 21 | PARENT-1-PROJECT_LOC/OsirisMsgPassing.tla 22 | 23 | 24 | 25 | -------------------------------------------------------------------------------- /src/osiris_member.erl: -------------------------------------------------------------------------------- 1 | -module(osiris_member). 2 | 3 | 4 | -include("osiris.hrl"). 5 | -export([start/3, 6 | stop/2, 7 | delete/2]). 8 | 9 | %% Callbacks 10 | 11 | -callback start(node(), osiris:config()) -> 12 | supervisor:startchild_ret(). 13 | 14 | %% API 15 | 16 | -spec start(module(), node(), osiris:config()) -> 17 | supervisor:startchild_ret(). 18 | start(Mod, Node, Config) -> 19 | Mod:start(Node, Config). 20 | 21 | -spec stop(node(), osiris:config()) -> ok. 22 | stop(Node, Config) -> 23 | ?SUP:stop_child(Node, Config). 24 | 25 | -spec delete(node(), osiris:config()) -> 26 | ok | {error, term()}. 27 | delete(Node, Config) -> 28 | ?SUP:delete_child(Node, Config). 29 | -------------------------------------------------------------------------------- /test/osiris_util_SUITE_data/inter_node_tls_broken.config: -------------------------------------------------------------------------------- 1 | [ 2 | {server, [ 3 | {cacertfile, "/etc/rabbitmq/ca_certificate.pem"}, 4 | {certfile, "/etc/rabbitmq/server_certificate.pem"}, 5 | {keyfile, "/etc/rabbitmq/server_key.pem"}, 6 | {secure_renegotiate, true}, 7 | {verify, verify_peer}, 8 | {fail_if_no_peer_cert, true} 9 | ]} 10 | 11 | BROKEN 12 | 13 | {client, [ 14 | {cacertfile, "/etc/rabbitmq/ca_certificate.pem"}, 15 | {certfile, "/etc/rabbitmq/client_certificate.pem"}, 16 | {keyfile, "/etc/rabbitmq/client_key.pem"}, 17 | {secure_renegotiate, true}, 18 | {verify, verify_peer}, 19 | {fail_if_no_peer_cert, true} 20 | ]} 21 | ]. -------------------------------------------------------------------------------- /tla/OsirisMsgPassing.toolbox/OsirisSafety/MC.cfg: -------------------------------------------------------------------------------- 1 | \* MV CONSTANT declarations 2 | CONSTANTS 3 | r1 = r1 4 | r2 = r2 5 | r3 = r3 6 | \* MV CONSTANT declarations 7 | CONSTANTS 8 | a = a 9 | b = b 10 | c = c 11 | \* MV CONSTANT definitions 12 | CONSTANT 13 | R <- const_1589378301396272000 14 | \* MV CONSTANT definitions 15 | CONSTANT 16 | V <- const_1589378301396273000 17 | \* SYMMETRY definition 18 | SYMMETRY symm_1589378301396274000 19 | \* CONSTRAINT definition 20 | CONSTRAINT 21 | constr_1589378301396275000 22 | \* INIT definition 23 | INIT 24 | Init 25 | \* NEXT definition 26 | NEXT 27 | Next 28 | \* INVARIANT definition 29 | INVARIANT 30 | TypeOK 31 | NoDivergence 32 | FollowerEqualOrLowerEpoch 33 | NoLossOfConfirmedWrite 34 | LerMatchesLog 35 | TestInv -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Osiris 2 | 3 | Osiris is a foundation of the log-based streaming subsystem for RabbitMQ. 4 | 5 | ## Project Maturity 6 | 7 | Osiris is a reasonably mature and stable project: it first shipped as part of RabbitMQ in 2021. 8 | 9 | ## Build Status 10 | 11 | [![Erlang CI](https://github.com/rabbitmq/osiris/actions/workflows/erlang.yml/badge.svg)](https://github.com/rabbitmq/osiris/actions/workflows/erlang.yml) 12 | 13 | ## Documentation 14 | 15 | None at the moment. Consult the test suite for API usage examples 16 | and [TLA+ spec](./tla) to learn about the protocol. 17 | 18 | ## Copyright and License 19 | 20 | (c) 2020-2024 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 21 | 22 | Double licensed under the ASL2 and MPL2.0. 23 | See [LICENSE](./LICENSE) for details. 24 | -------------------------------------------------------------------------------- /CONTRIBUTING.md: -------------------------------------------------------------------------------- 1 | # Contributing to Osiris 2 | 3 | ## Overview 4 | 5 | Osiris is a very young project under heavy development, so consider discussing your idea with 6 | the maintainers on the RabbitMQ mailing list, rabbitmq-users. 7 | 8 | The process is fairly standard and straightforward: 9 | 10 | * Fork the repository 11 | * Create a branch for your changes 12 | * Add tests, modify code, refactor, repeat 13 | * Push your branch 14 | * Submit a pull request with a reasonably detailed justification of your changes 15 | * Be patient 16 | 17 | 18 | ## Building 19 | 20 | Osiris uses [erlang.mk](https://erlang.mk/) for build system. Build it with 21 | 22 | ``` shell 23 | make 24 | ``` 25 | 26 | Clean compilation artifacts with 27 | 28 | ``` shell 29 | make clean 30 | ``` 31 | 32 | ## Running Tests 33 | 34 | ``` 35 | make tests 36 | ``` 37 | 38 | and then open `logs/index.html` to see test run results. 39 | -------------------------------------------------------------------------------- /tla/OsirisMsgPassing.toolbox/OsirisSafety/MC.tla: -------------------------------------------------------------------------------- 1 | ---- MODULE MC ---- 2 | EXTENDS OsirisMsgPassing, TLC 3 | 4 | \* MV CONSTANT declarations@modelParameterConstants 5 | CONSTANTS 6 | r1, r2, r3 7 | ---- 8 | 9 | \* MV CONSTANT declarations@modelParameterConstants 10 | CONSTANTS 11 | a, b, c 12 | ---- 13 | 14 | \* MV CONSTANT definitions R 15 | const_1589378301396272000 == 16 | {r1, r2, r3} 17 | ---- 18 | 19 | \* MV CONSTANT definitions V 20 | const_1589378301396273000 == 21 | {a, b, c} 22 | ---- 23 | 24 | \* SYMMETRY definition 25 | symm_1589378301396274000 == 26 | Permutations(const_1589378301396272000) \union Permutations(const_1589378301396273000) 27 | ---- 28 | 29 | \* CONSTRAINT definition @modelParameterContraint:0 30 | constr_1589378301396275000 == 31 | /\ coord_epoch < 4 32 | /\ start_stop_ctr < 4 33 | ---- 34 | ============================================================================= 35 | \* Modification History 36 | \* Created Thu Jan 30 02:13:29 PST 2020 by jack 37 | -------------------------------------------------------------------------------- /Makefile: -------------------------------------------------------------------------------- 1 | PROJECT = osiris 2 | # PROJECT_DESCRIPTION = Foundation of the log-based streaming subsystem for RabbitMQ 3 | # PROJECT_VERSION = v1.10.2 4 | 5 | # define PROJECT_ENV 6 | # [ 7 | # {data_dir, "/tmp/osiris"}, 8 | # {port_range, {6000, 6500}}, 9 | # {max_segment_size_chunks, 256000}, 10 | # {replication_transport, tcp}, 11 | # {replica_forced_gc_default_interval, 4999} 12 | # ] 13 | # endef 14 | # 15 | # This project uses an app.src file 16 | 17 | LOCAL_DEPS = sasl crypto 18 | dep_gen_batch_server = hex 0.8.9 19 | dep_seshat = hex 1.0.1 20 | DEPS = gen_batch_server seshat 21 | 22 | # TEST_DEPS=eunit_formatters looking_glass 23 | dep_tls_gen = git https://github.com/rabbitmq/tls-gen.git main 24 | TEST_DEPS=eunit_formatters tls_gen 25 | 26 | dep_looking_glass = git https://github.com/rabbitmq/looking-glass.git master 27 | # PLT_APPS += eunit syntax_tools erts kernel stdlib common_test inets ssh ssl meck looking_glass gen_batch_server inet_tcp_proxy 28 | 29 | DIALYZER_OPTS += --src -r test -Wunmatched_returns -Werror_handling 30 | PLT_APPS += seshat ssl eunit common_test 31 | EUNIT_OPTS = no_tty, {report, {eunit_progress, [colored, profile]}} 32 | include $(if $(ERLANG_MK_FILENAME),$(ERLANG_MK_FILENAME),erlang.mk) 33 | -------------------------------------------------------------------------------- /.github/workflows/release.yml: -------------------------------------------------------------------------------- 1 | name: Release 2 | on: 3 | push: 4 | tags: 5 | - v1.* 6 | jobs: 7 | release: 8 | runs-on: ubuntu-latest 9 | steps: 10 | - name: CHECKOUT 11 | uses: actions/checkout@v3 12 | with: 13 | path: osiris 14 | - name: ASSERT VERSIONS 15 | id: versions 16 | working-directory: osiris 17 | run: | 18 | VERSION_ERLANG_MK="$(sed -ne 's/PROJECT_VERSION = \(.*\)/\1/p' Makefile)" 19 | 20 | if [[ "${{ github.ref_name }}" != "v$VERSION_ERLANG_MK" ]]; then 21 | echo "Version in Makefile ($VERSION_ERLANG_MK) does not match tag (${{ github.ref_name }})" 22 | exit 1 23 | fi 24 | - name: FETCH THE SOURCE ARCHIVE 25 | run: | 26 | curl \ 27 | -L \ 28 | -o osiris-${{ steps.versions.outputs.version }}.tar.gz \ 29 | https://github.com/${{ github.repository }}/archive/${{ github.ref }}.tar.gz 30 | - name: CREATE RELEASE 31 | id: create-release 32 | uses: ncipollo/release-action@v1.12.0 33 | with: 34 | token: ${{ secrets.REPO_SCOPED_TOKEN }} 35 | allowUpdates: true 36 | artifactErrorsFailBuild: true 37 | updateOnlyUnreleased: true 38 | generateReleaseNotes: true 39 | artifacts: >- 40 | osiris-${{ steps.versions.outputs.version }}.tar.gz 41 | -------------------------------------------------------------------------------- /src/osiris_counters.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_counters). 9 | 10 | -export([init/0, 11 | new/2, 12 | fetch/1, 13 | overview/0, 14 | overview/1, 15 | delete/1 16 | ]). 17 | 18 | -type name() :: term(). 19 | 20 | -spec init() -> ok. 21 | init() -> 22 | _ = seshat:new_group(osiris), 23 | ok. 24 | 25 | -spec new(name(), [{Name :: atom(), Position :: non_neg_integer(), 26 | Type :: atom(), Description :: term()}]) -> 27 | counters:counters_ref(). 28 | new(Name, Fields) -> 29 | seshat:new(osiris, Name, Fields). 30 | 31 | -spec fetch(name()) -> undefined | counters:counters_ref(). 32 | fetch(Name) -> 33 | seshat:fetch(osiris, Name). 34 | 35 | -spec delete(term()) -> ok. 36 | delete(Name) -> 37 | seshat:delete(osiris, Name). 38 | 39 | -spec overview() -> #{name() => #{atom() => non_neg_integer()}}. 40 | overview() -> 41 | seshat:counters(osiris). 42 | 43 | -spec overview(name()) -> #{atom() => non_neg_integer()} | undefined. 44 | overview(Name) -> 45 | seshat:counters(osiris, Name). 46 | -------------------------------------------------------------------------------- /src/osiris_sup.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_sup). 9 | 10 | -behaviour(supervisor). 11 | 12 | -export([start_link/0]). 13 | -export([init/1]). 14 | 15 | start_link() -> 16 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 17 | 18 | init([]) -> 19 | osiris_counters:init(), 20 | SupFlags = 21 | #{strategy => one_for_all, 22 | intensity => 5, 23 | period => 5}, 24 | %% todo put under own sup 25 | Ets = 26 | #{id => osiris_ets, 27 | type => worker, 28 | start => {osiris_ets, start_link, []}}, 29 | Retention = 30 | #{id => osiris_retention, 31 | type => worker, 32 | start => {osiris_retention, start_link, []}}, 33 | ServerSup = 34 | #{id => osiris_server_sup, 35 | type => supervisor, 36 | start => {osiris_server_sup, start_link, []}}, 37 | ReplicaReader = 38 | #{id => osiris_replica_reader_sup, 39 | type => supervisor, 40 | start => {osiris_replica_reader_sup, start_link, []}}, 41 | {ok, {SupFlags, [Ets, Retention, ServerSup, ReplicaReader]}}. 42 | -------------------------------------------------------------------------------- /test/osiris_tracer.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_tracer). 9 | 10 | -record(?MODULE, 11 | { 12 | calls = [] :: list() 13 | }). 14 | 15 | -export([start/1, 16 | loop/1, 17 | calls/1, 18 | calls/3, 19 | call_count/1, 20 | call_count/3, 21 | stop/1]). 22 | 23 | start(MFAS) -> 24 | P = spawn(?MODULE, loop, [#?MODULE{}]), 25 | erlang:trace(self(), true, [call, {tracer, P}]), 26 | [erlang:trace_pattern(MFA, true, [global]) || MFA <- MFAS], 27 | P. 28 | 29 | loop(#?MODULE{calls = Calls} = S) -> 30 | receive 31 | {trace, _Pid, call, {Module, Function, Arguments}} -> 32 | loop(#?MODULE{calls = [{Module, Function, Arguments} | Calls]}); 33 | {P, calls} -> 34 | P ! lists:reverse(Calls), 35 | loop(S); 36 | stop -> 37 | ok; 38 | _ -> 39 | loop(S) 40 | end. 41 | 42 | calls(P) -> 43 | P ! {self(), calls}, 44 | receive 45 | Calls -> Calls 46 | end. 47 | 48 | calls(P, M, F) -> 49 | Calls = calls(P), 50 | lists:filter(fun({M1, F1, _A}) when M1 == M andalso F1 == F -> 51 | true; 52 | (_) -> false 53 | end, Calls). 54 | 55 | call_count(P) -> 56 | length(calls(P)). 57 | 58 | call_count(P, M, F) -> 59 | length(calls(P, M, F)). 60 | 61 | stop(P) -> 62 | erlang:trace(self(), false, [call]), 63 | P ! stop. 64 | -------------------------------------------------------------------------------- /src/osiris_log_shared.erl: -------------------------------------------------------------------------------- 1 | -module(osiris_log_shared). 2 | 3 | -define(COMMITTED_IDX, 1). 4 | -define(FIRST_IDX, 2). 5 | -define(LAST_IDX, 3). 6 | 7 | -export([ 8 | new/0, 9 | committed_chunk_id/1, 10 | first_chunk_id/1, 11 | last_chunk_id/1, 12 | set_committed_chunk_id/2, 13 | set_first_chunk_id/2, 14 | set_last_chunk_id/2 15 | ]). 16 | 17 | -type chunk_id() :: -1 | non_neg_integer(). 18 | 19 | -spec new() -> atomics:atomics_ref(). 20 | new() -> 21 | %% Oh why, oh why did we think the first chunk id in 22 | %% a stream should have offset 0? 23 | Ref = atomics:new(3, [{signed, true}]), 24 | atomics:put(Ref, ?COMMITTED_IDX, -1), 25 | atomics:put(Ref, ?FIRST_IDX, -1), 26 | atomics:put(Ref, ?LAST_IDX, -1), 27 | Ref. 28 | 29 | -spec committed_chunk_id(atomics:atomics_ref()) -> chunk_id(). 30 | committed_chunk_id(Ref) -> 31 | atomics:get(Ref, ?COMMITTED_IDX). 32 | 33 | -spec first_chunk_id(atomics:atomics_ref()) -> chunk_id(). 34 | first_chunk_id(Ref) -> 35 | atomics:get(Ref, ?FIRST_IDX). 36 | 37 | -spec last_chunk_id(atomics:atomics_ref()) -> chunk_id(). 38 | last_chunk_id(Ref) -> 39 | atomics:get(Ref, ?LAST_IDX). 40 | 41 | -spec set_committed_chunk_id(atomics:atomics_ref(), chunk_id()) -> ok. 42 | set_committed_chunk_id(Ref, Value) when is_integer(Value) -> 43 | atomics:put(Ref, ?COMMITTED_IDX, Value). 44 | 45 | -spec set_first_chunk_id(atomics:atomics_ref(), chunk_id()) -> ok. 46 | set_first_chunk_id(Ref, Value) when is_integer(Value) -> 47 | atomics:put(Ref, ?FIRST_IDX, Value). 48 | 49 | -spec set_last_chunk_id(atomics:atomics_ref(), chunk_id()) -> ok. 50 | set_last_chunk_id(Ref, Value) when is_integer(Value) -> 51 | atomics:put(Ref, ?LAST_IDX, Value). 52 | 53 | 54 | 55 | -ifdef(TEST). 56 | 57 | -include_lib("eunit/include/eunit.hrl"). 58 | basics_test() -> 59 | R = new(), 60 | ?assertEqual(-1, committed_chunk_id(R)), 61 | ?assertEqual(-1, first_chunk_id(R)), 62 | ?assertEqual(-1, last_chunk_id(R)), 63 | ok = set_committed_chunk_id(R, 2), 64 | ok = set_first_chunk_id(R, 1), 65 | ok = set_last_chunk_id(R, 3), 66 | ?assertEqual(2, committed_chunk_id(R)), 67 | ?assertEqual(1, first_chunk_id(R)), 68 | ?assertEqual(3, last_chunk_id(R)), 69 | 70 | ok. 71 | 72 | -endif. 73 | -------------------------------------------------------------------------------- /src/osiris_server_sup.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_server_sup). 9 | 10 | -behaviour(supervisor). 11 | 12 | -include("osiris.hrl"). 13 | -export([start_link/0]). 14 | -export([init/1, 15 | stop_child/2, 16 | delete_child/2]). 17 | 18 | start_link() -> 19 | supervisor:start_link({local, ?MODULE}, ?MODULE, []). 20 | 21 | init([]) -> 22 | Procs = [], 23 | {ok, {{one_for_one, 1, 5}, Procs}}. 24 | 25 | stop_child(Node, #{name := Name}) -> 26 | stop_child(Node, Name); 27 | stop_child(Node, Name) when ?IS_STRING(Name) -> 28 | try 29 | %% as replicas are temporary we don't have to explicitly 30 | %% delete them 31 | case supervisor:terminate_child({?MODULE, Node}, Name) of 32 | ok -> 33 | ok; 34 | {error, not_found} -> 35 | OthName = flip_name(Name), 36 | case supervisor:terminate_child({?MODULE, Node}, OthName) of 37 | ok -> 38 | ok; 39 | {error, not_found} -> 40 | ok; 41 | Err -> 42 | Err 43 | end; 44 | Err -> 45 | Err 46 | end 47 | catch 48 | _:{noproc, _} -> 49 | %% Whole supervisor or app is already down - i.e. stop_app 50 | ok 51 | end. 52 | 53 | delete_child(Node, #{name := Name} = Config) -> 54 | try 55 | case supervisor:get_childspec({?MODULE, Node}, Name) of 56 | {ok, _} -> 57 | _ = stop_child(Node, Name), 58 | rpc:call(Node, osiris_log, delete_directory, [Config]); 59 | {error, not_found} -> 60 | OthName = flip_name(Name), 61 | case supervisor:get_childspec({?MODULE, Node}, OthName) of 62 | {ok, _} -> 63 | _ = stop_child(Node, OthName), 64 | rpc:call(Node, osiris_log, delete_directory, [Config]); 65 | {error, not_found} -> 66 | rpc:call(Node, osiris_log, delete_directory, [Config]) 67 | end 68 | end 69 | catch 70 | _:{noproc, _} -> 71 | %% Whole supervisor or app is already down - i.e. stop_app 72 | rpc:call(Node, osiris_log, delete_directory, [Config]) 73 | end. 74 | 75 | flip_name(N) when is_binary(N) -> 76 | binary_to_list(N); 77 | flip_name(N) when is_list(N) -> 78 | list_to_binary(N). 79 | -------------------------------------------------------------------------------- /src/osiris_ets.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_ets). 9 | 10 | -behaviour(gen_server). 11 | 12 | %% API functions 13 | -export([start_link/0]). 14 | %% gen_server callbacks 15 | -export([init/1, 16 | handle_call/3, 17 | handle_cast/2, 18 | handle_info/2, 19 | terminate/2, 20 | code_change/3]). 21 | 22 | -record(state, {}). 23 | 24 | %%%=================================================================== 25 | %%% API functions 26 | %%%=================================================================== 27 | 28 | %% @spec start_link() -> {ok, Pid} | ignore | {error, Error} 29 | start_link() -> 30 | gen_server:start_link({local, ?MODULE}, ?MODULE, [], []). 31 | 32 | %%%=================================================================== 33 | %%% gen_server callbacks 34 | %%%=================================================================== 35 | 36 | % @spec init(Args) -> {ok, State} | 37 | %% {ok, State, Timeout} | 38 | %% ignore | 39 | %% {stop, Reason} 40 | init(_) -> 41 | _ = ets:new(osiris_reader_context_cache, [set, named_table, public]), 42 | {ok, #state{}}. 43 | 44 | %% @spec handle_call(Request, From, State) -> 45 | %% {reply, Reply, State} | 46 | %% {reply, Reply, State, Timeout} | 47 | %% {noreply, State} | 48 | %% {noreply, State, Timeout} | 49 | %% {stop, Reason, Reply, State} | 50 | %% {stop, Reason, State} 51 | handle_call(_Request, _From, State) -> 52 | Reply = ok, 53 | {reply, Reply, State}. 54 | 55 | %% @spec handle_cast(Msg, State) -> {noreply, State} | 56 | %% {noreply, State, Timeout} | 57 | %% {stop, Reason, State} 58 | handle_cast(_Msg, State) -> 59 | {noreply, State}. 60 | 61 | %% @spec handle_info(Info, State) -> {noreply, State} | 62 | %% {noreply, State, Timeout} | 63 | %% {stop, Reason, State} 64 | handle_info(_Info, State) -> 65 | {noreply, State}. 66 | 67 | %% @spec terminate(Reason, State) -> void() 68 | terminate(_Reason, _State) -> 69 | ok. 70 | 71 | %% @spec code_change(OldVsn, State, Extra) -> {ok, NewState} 72 | code_change(_OldVsn, State, _Extra) -> 73 | {ok, State}. 74 | 75 | %%%=================================================================== 76 | %%% Internal functions 77 | %%%=================================================================== 78 | -------------------------------------------------------------------------------- /src/osiris.hrl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | %% logging shim 9 | -define(DEBUG(Fmt, Args), ?DISPATCH_LOG(debug, Fmt, Args)). 10 | -define(DEBUG_IF(Fmt, Args, Bool), 11 | if Bool -> 12 | ?DISPATCH_LOG(debug, Fmt, Args); 13 | true -> ok 14 | end). 15 | -define(INFO(Fmt, Args), ?DISPATCH_LOG(info, Fmt, Args)). 16 | -define(NOTICE(Fmt, Args), ?DISPATCH_LOG(notice, Fmt, Args)). 17 | -define(WARN(Fmt, Args), ?DISPATCH_LOG(warning, Fmt, Args)). 18 | -define(WARNING(Fmt, Args), ?DISPATCH_LOG(warning, Fmt, Args)). 19 | -define(ERR(Fmt, Args), ?DISPATCH_LOG(error, Fmt, Args)). 20 | -define(ERROR(Fmt, Args), ?DISPATCH_LOG(error, Fmt, Args)). 21 | 22 | -define(DISPATCH_LOG(Level, Fmt, Args), 23 | %% same as OTP logger does when using the macro 24 | catch (persistent_term:get('$osiris_logger')):log(Level, Fmt, Args, 25 | #{mfa => {?MODULE, 26 | ?FUNCTION_NAME, 27 | ?FUNCTION_ARITY}, 28 | file => ?FILE, 29 | line => ?LINE, 30 | domain => [osiris]}), 31 | ok). 32 | 33 | -define(IS_STRING(S), is_list(S) orelse is_binary(S)). 34 | 35 | -define(C_NUM_LOG_FIELDS, 5). 36 | 37 | -define(MAGIC, 5). 38 | %% chunk format version 39 | -define(VERSION, 0). 40 | -define(IDX_VERSION, 1). 41 | -define(LOG_VERSION, 1). 42 | -define(IDX_HEADER, <<"OSII", ?IDX_VERSION:32/unsigned>>). 43 | -define(LOG_HEADER, <<"OSIL", ?LOG_VERSION:32/unsigned>>). 44 | -define(HEADER_SIZE_B, 48). 45 | -define(IDX_HEADER_SIZE, 8). 46 | -define(LOG_HEADER_SIZE, 8). 47 | -define(FILE_OPTS_WRITE, [raw, binary, write, read]). 48 | -define(INDEX_RECORD_SIZE_B, 29). 49 | 50 | 51 | %% chunk types 52 | -define(CHNK_USER, 0). 53 | -define(CHNK_TRK_DELTA, 1). 54 | -define(CHNK_TRK_SNAPSHOT, 2). 55 | 56 | -define(SUP, osiris_server_sup). 57 | 58 | -define(DEFAULT_FILTER_SIZE, 16). 59 | 60 | -define(INFO_(Name, Str, Args), 61 | ?INFO("~ts [~s:~s/~b] " Str, 62 | [Name, ?MODULE, ?FUNCTION_NAME, ?FUNCTION_ARITY | Args])). 63 | 64 | -define(WARN_(Name, Str, Args), 65 | ?WARN("~ts [~s:~s/~b] " Str, 66 | [Name, ?MODULE, ?FUNCTION_NAME, ?FUNCTION_ARITY | Args])). 67 | 68 | -define(ERROR_(Name, Str, Args), 69 | ?ERROR("~ts [~s:~s/~b] " Str, 70 | [Name, ?MODULE, ?FUNCTION_NAME, ?FUNCTION_ARITY | Args])). 71 | 72 | -define(DEBUG_(Name, Str, Args), 73 | ?DEBUG("~ts [~s:~s/~b] " Str, 74 | [Name, ?MODULE, ?FUNCTION_NAME, ?FUNCTION_ARITY | Args])). 75 | -------------------------------------------------------------------------------- /tla/OsirisMsgPassing.toolbox/OsirisMsgPassing___OsirisSafety.launch: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | 41 | 42 | 43 | 44 | 45 | 46 | 47 | 48 | 49 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | -------------------------------------------------------------------------------- /test/osiris_bloom_SUITE.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_bloom_SUITE). 9 | 10 | -compile(nowarn_export_all). 11 | -compile(export_all). 12 | 13 | -export([]). 14 | 15 | % -include_lib("common_test/include/ct.hrl"). 16 | -include_lib("eunit/include/eunit.hrl"). 17 | 18 | %%%=================================================================== 19 | %%% Common Test callbacks 20 | %%%=================================================================== 21 | 22 | all() -> 23 | [{group, tests}]. 24 | 25 | all_tests() -> [ 26 | basics, 27 | multi_matcher, 28 | match_unfiltered, 29 | eval_hash 30 | ]. 31 | 32 | groups() -> 33 | [{tests, [], all_tests()}]. 34 | 35 | init_per_suite(Config) -> 36 | Config. 37 | 38 | end_per_suite(_Config) -> 39 | ok. 40 | 41 | init_per_group(_Group, Config) -> 42 | Config. 43 | 44 | end_per_group(_Group, _Config) -> 45 | ok. 46 | 47 | init_per_testcase(_TestCase, Config) -> 48 | Config. 49 | 50 | end_per_testcase(_TestCase, _Config) -> 51 | ok. 52 | 53 | %%%=================================================================== 54 | %%% Test cases 55 | %%%=================================================================== 56 | 57 | basics(_Config) -> 58 | S0 = osiris_bloom:init(16), 59 | S1 = osiris_bloom:insert(<<"banana">>, S0), 60 | S2 = osiris_bloom:insert(<<"apple">>, S1), 61 | B = osiris_bloom:to_binary(S2), 62 | 63 | ?assert(osiris_bloom:is_match(B, matcher(<<"banana">>))), 64 | ?assert(osiris_bloom:is_match(B, matcher(<<"apple">>))), 65 | ?assertNot(osiris_bloom:is_match(B, matcher(<<"pineapple">>))), 66 | ok. 67 | 68 | multi_matcher(_Config) -> 69 | B1 = make_filter_bin([<<"banana">>], 16), 70 | B2 = make_filter_bin([<<"apple">>], 16), 71 | B3 = make_filter_bin([<<"pineapple">>], 16), 72 | 73 | Spec = #{filters =>[<<"banana">>, <<"apple">>]}, 74 | M = osiris_bloom:init_matcher(Spec), 75 | ?assert(osiris_bloom:is_match(B1, M)), 76 | ?assert(osiris_bloom:is_match(B2, M)), 77 | ?assertNot(osiris_bloom:is_match(B3, M)), 78 | ok. 79 | 80 | match_unfiltered(_Config) -> 81 | 82 | <<>> = make_filter_bin([<<>>], 16), 83 | Spec = #{filters => [<<"banana">>, <<"apple">>], 84 | match_unfiltered => true}, 85 | M = osiris_bloom:init_matcher(Spec), 86 | %% first check the "empty" filter (i.e. none of the messages in the chunk 87 | %% had a filter) 88 | ?assert(osiris_bloom:is_match(<<>>, M)), 89 | %% then check if the match unfiltered is false or missing it does not match 90 | M2 = osiris_bloom:init_matcher(Spec#{match_unfiltered => false}), 91 | ?assertNot(osiris_bloom:is_match(<<>>, M2)), 92 | 93 | %% the empty filter indicates unfiltered 94 | B1 = make_filter_bin([<<>>, <<"banana">>], 16), 95 | %% the first bit should be set to indicate stuff 96 | ?assertMatch(<<1:1, _/bitstring>>, B1), 97 | ?assert(osiris_bloom:is_match(B1, M)), 98 | 99 | B32 = make_filter_bin([<<"pineapple">>], 17), 100 | {retry_with, M32} = osiris_bloom:is_match(B32, M), 101 | ?assertNot(osiris_bloom:is_match(B32, M32)), 102 | 103 | %% an empty filter combined with a non match should match 104 | B3 = make_filter_bin([<<>>, <<"pineapple">>], 16), 105 | ?assert(osiris_bloom:is_match(B3, M)), 106 | ok. 107 | 108 | eval_hash(_Config) -> 109 | N = lists:seq(1, 1000), 110 | Size = 16, 111 | 112 | C = counters:new(Size * 8, []), 113 | 114 | [begin 115 | {H1, H2} = osiris_bloom:make_hash( 116 | crypto:strong_rand_bytes(16), Size * 8), 117 | counters:add(C, H1+1, 1), 118 | counters:add(C, H2+1, 1) 119 | end || _ <- N], 120 | 121 | 122 | CList = [counters:get(C, I) || I <- lists:seq(1, Size * 8)], 123 | ct:pal("counters ~p", [CList]), 124 | 125 | ok. 126 | 127 | 128 | %% helpers 129 | 130 | matcher(Value) -> 131 | Spec = #{filters => [Value]}, 132 | osiris_bloom:init_matcher(Spec). 133 | 134 | make_filter_bin(Values, SizeB) -> 135 | S0 = osiris_bloom:init(SizeB), 136 | <<>> = osiris_bloom:to_binary(S0), 137 | S1 = lists:foldl(fun osiris_bloom:insert/2, 138 | S0, Values), 139 | B = osiris_bloom:to_binary(S1), 140 | ?assert(byte_size(B) == SizeB orelse byte_size(B) == 0), 141 | B. 142 | 143 | -------------------------------------------------------------------------------- /src/osiris_retention.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_retention). 9 | 10 | -behaviour(gen_server). 11 | 12 | -include("osiris.hrl"). 13 | %% API functions 14 | -export([start_link/0, 15 | eval/4]). 16 | %% gen_server callbacks 17 | -export([init/1, 18 | handle_call/3, 19 | handle_cast/2, 20 | handle_info/2, 21 | terminate/2, 22 | code_change/3]). 23 | 24 | -define(DEFAULT_SCHEDULED_EVAL_TIME, 1000 * 60 * 60). %% 1HR 25 | 26 | -record(state, {scheduled = #{} :: #{osiris:name() => timer:tref()}}). 27 | 28 | %%%=================================================================== 29 | %%% API functions 30 | %%%=================================================================== 31 | 32 | %% @spec start_link() -> {ok, Pid} | ignore | {error, Error} 33 | start_link() -> 34 | gen_server:start_link({local, ?MODULE}, ?MODULE, [], []). 35 | 36 | -spec eval(osiris:name(), file:name_all(), [osiris:retention_spec()], 37 | fun((osiris_log:range()) -> ok)) -> 38 | ok. 39 | eval(_Name, _Dir, [], _Fun) -> 40 | ok; 41 | eval(Name, Dir, Specs, Fun) -> 42 | gen_server:cast(?MODULE, {eval, self(), Name, Dir, Specs, Fun}). 43 | 44 | %%%=================================================================== 45 | %%% gen_server callbacks 46 | %%%=================================================================== 47 | 48 | % @spec init(Args) -> {ok, State} | 49 | %% {ok, State, Timeout} | 50 | %% ignore | 51 | %% {stop, Reason} 52 | init([]) -> 53 | {ok, #state{}}. 54 | 55 | %% @spec handle_call(Request, From, State) -> 56 | %% {reply, Reply, State} | 57 | %% {reply, Reply, State, Timeout} | 58 | %% {noreply, State} | 59 | %% {noreply, State, Timeout} | 60 | %% {stop, Reason, Reply, State} | 61 | %% {stop, Reason, State} 62 | handle_call(_Request, _From, State) -> 63 | Reply = ok, 64 | {reply, Reply, State}. 65 | 66 | %% @spec handle_cast(Msg, State) -> {noreply, State} | 67 | %% {noreply, State, Timeout} | 68 | %% {stop, Reason, State} 69 | handle_cast({eval, Pid, Name, Dir, Specs, Fun} = Eval, State) -> 70 | %% only do retention evaluation for stream processes that are 71 | %% alive as the callback Fun passed in would update a shared atomic 72 | %% value and this atomic is new per process incarnation 73 | case is_process_alive(Pid) of 74 | true -> 75 | try osiris_log:evaluate_retention(Dir, Specs) of 76 | Result -> 77 | _ = Fun(Result), 78 | {noreply, schedule(Eval, Result, State)} 79 | catch _:Err -> 80 | ?DEBUG_(Name, "retention evaluation failed with ~w", [Err]), 81 | {noreply, State} 82 | end; 83 | false -> 84 | {noreply, State} 85 | end. 86 | 87 | %% @spec handle_info(Info, State) -> {noreply, State} | 88 | %% {noreply, State, Timeout} | 89 | %% {stop, Reason, State} 90 | handle_info(_Info, State) -> 91 | {noreply, State}. 92 | 93 | %% @spec terminate(Reason, State) -> void() 94 | terminate(_Reason, _State) -> 95 | ok. 96 | 97 | %% @spec code_change(OldVsn, State, Extra) -> {ok, NewState} 98 | code_change(_OldVsn, State, _Extra) -> 99 | {ok, State}. 100 | 101 | %%%=================================================================== 102 | %%% Internal functions 103 | %%%=================================================================== 104 | schedule({eval, _Pid, Name, _Dir, Specs, _Fun} = Eval, 105 | {_, _, NumSegmentRemaining}, 106 | #state{scheduled = Scheduled0} = State) -> 107 | %% we need to check the scheduled map even if the current specs do not 108 | %% include max_age as the retention config could have changed 109 | Scheduled = case maps:take(Name, Scheduled0) of 110 | {OldRef, Scheduled1} -> 111 | _ = erlang:cancel_timer(OldRef), 112 | Scheduled1; 113 | error -> 114 | Scheduled0 115 | end, 116 | case lists:any(fun ({T, _}) -> T == max_age end, Specs) andalso 117 | NumSegmentRemaining > 1 of 118 | true -> 119 | EvalInterval = application:get_env(osiris, retention_eval_interval, 120 | ?DEFAULT_SCHEDULED_EVAL_TIME), 121 | Ref = erlang:send_after(EvalInterval, self(), {'$gen_cast', Eval}), 122 | State#state{scheduled = Scheduled#{Name => Ref}}; 123 | false -> 124 | State#state{scheduled = Scheduled} 125 | end. 126 | 127 | -------------------------------------------------------------------------------- /src/osiris_bench.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_bench). 9 | 10 | -include("osiris.hrl"). 11 | -include("osiris_peer_shim.hrl"). 12 | 13 | -export([run/1, 14 | stop/1, 15 | do_metrics/1, 16 | do_publish/1, 17 | test/1]). 18 | 19 | -define(METRICS_INT_S, 10). 20 | 21 | %% holds static or rarely changing fields 22 | -record(cfg, {}). 23 | -record(?MODULE, {cfg :: #cfg{}}). 24 | 25 | -opaque state() :: #?MODULE{}. 26 | 27 | -export_type([state/0]). 28 | 29 | % -type spec() :: #{name := string(), 30 | % in_flight := non_neg_integer() 31 | % }. 32 | 33 | test(Name) -> 34 | Spec = #{name => Name, in_flight => 5000}, 35 | run(Spec). 36 | 37 | run(#{name := Name} = Spec) -> 38 | {ok, Cwd} = file:get_cwd(), 39 | Dir0 = maps:get(directory, Spec, Cwd), 40 | Dir = filename:join([Dir0, ?MODULE, Name]), 41 | %% create cluster (if needed) 42 | [LeaderNode | Replicas] = 43 | Nodes = [start_secondary(N, Dir) || N <- [s1, s2, s3]], 44 | 45 | %% declare osiris cluster 46 | Conf0 = 47 | #{name => Name, 48 | epoch => 1, 49 | leader_node => LeaderNode, 50 | retention => [{max_bytes, 100 * 1000 * 1000}], 51 | replica_nodes => Replicas}, 52 | {ok, #{leader_pid := Leader}} = osiris:start_cluster(Conf0), 53 | {ok, #{leader_pid := Leader2}} = 54 | osiris:start_cluster(Conf0#{name => Name ++ Name}), 55 | %% start metrics gatherer on leader node 56 | start_metrics_gatherer(node(Leader)), 57 | %% 58 | %% start publisher 59 | InFlight = maps:get(in_flight, Spec, 1000), 60 | start_publisher(node(Leader), 61 | #{leader => Leader, in_flight => InFlight}), 62 | start_publisher(node(Leader2), 63 | #{leader => Leader2, in_flight => InFlight}), 64 | Nodes. 65 | 66 | stop(Nodes) when is_list(Nodes) -> 67 | [stop_peer(N) || N <- Nodes]. 68 | 69 | -if(?OTP_RELEASE >= 25). 70 | stop_peer(RefOrName) -> 71 | %% peer:stop/1 not idempotent 72 | try 73 | ?PEER_MODULE:stop(RefOrName) 74 | catch exit:_:_Stacktrace -> 75 | ok 76 | end. 77 | -else. 78 | stop_peer(RefOrName) -> 79 | ?PEER_MODULE:stop(RefOrName). 80 | -endif. 81 | 82 | start_publisher(Node, Conf) -> 83 | erlang:spawn(Node, ?MODULE, do_publish, [Conf]). 84 | 85 | do_publish(#{in_flight := InFlight} = Conf) -> 86 | do_publish0(Conf, InFlight). 87 | 88 | do_publish0(Conf, 0) -> 89 | receive 90 | {osiris_written, _, Corrs} -> 91 | do_publish0(Conf, length(Corrs)) 92 | after 1000000 -> 93 | exit(publish_timeout) 94 | end; 95 | do_publish0(#{leader := Leader} = Conf, InFlight) -> 96 | Ref = make_ref(), 97 | ok = osiris:write(Leader, undefined, Ref, <<"datadata">>), 98 | do_publish0(Conf, InFlight - 1). 99 | 100 | start_metrics_gatherer(Node) -> 101 | erlang:spawn(Node, ?MODULE, do_metrics, [#{}]). 102 | 103 | do_metrics(O0) -> 104 | O = osiris_counters:overview(), 105 | O1 = maps:with( 106 | maps:keys(O0), O), 107 | _ = maps:map( 108 | fun(K, CC) -> 109 | M = element(1, K), 110 | N = element(2, K), 111 | %% get last counters 112 | CL = maps:get(K, O0), 113 | Rates = 114 | maps:fold(fun(F, V, Acc) -> 115 | LV = maps:get(F, CL), 116 | [{F, (V - LV) / ?METRICS_INT_S} | Acc] 117 | end, 118 | [], CC), 119 | io:format("~s: ~s/~s - Rates ~w~n~n", [node(), M, N, Rates]) 120 | end, 121 | O1), 122 | timer:sleep(?METRICS_INT_S * 1000), 123 | do_metrics(O). 124 | 125 | start_secondary(NodeName, RunDir) -> 126 | Dir0 = filename:join(RunDir, NodeName), 127 | Host = get_current_host(), 128 | Dir = "'\"" ++ Dir0 ++ "\"'", 129 | Args = ["-pa" | search_paths()] ++ ["-osiris data_dir", Dir], 130 | ?INFO("osiris_bench: starting child node ~p with ~s~n", [NodeName, Args]), 131 | {ok, N} = start_peer_node(Host, NodeName, Args), 132 | ?INFO("osiris_bench: started child node ~w ~w~n", [N, Host]), 133 | Res = rpc:call(N, application, ensure_all_started, [osiris]), 134 | ok = rpc:call(N, logger, set_primary_config, [level, all]), 135 | ?INFO("osiris_bench: application start result ~p", [Res]), 136 | N. 137 | 138 | 139 | -if(?OTP_RELEASE >= 25). 140 | start_peer_node(Host, NodeName, Args) when is_atom(NodeName) -> 141 | Opts = #{ 142 | name => string:trim(atom_to_list(NodeName)), 143 | host => Host, 144 | args => Args 145 | }, 146 | {ok, _Pid, Node} = ?PEER_MODULE:start_link(Opts), 147 | {ok, Node}. 148 | -else. 149 | start_peer_node(Host, NodeName, Args) when is_atom(NodeName) -> 150 | ArgString = string:join(Args, " "), 151 | ?PEER_MODULE:start_link(Host, NodeName, ArgString). 152 | -endif. 153 | 154 | get_current_host() -> 155 | {ok, H} = inet:gethostname(), 156 | H. 157 | 158 | search_paths() -> 159 | Ld = code:lib_dir(), 160 | lists:filter(fun(P) -> string:prefix(P, Ld) =:= nomatch end, 161 | code:get_path()). 162 | 163 | -ifdef(TEST). 164 | 165 | -include_lib("eunit/include/eunit.hrl"). 166 | 167 | -endif. 168 | -------------------------------------------------------------------------------- /src/osiris_bloom.erl: -------------------------------------------------------------------------------- 1 | -module(osiris_bloom). 2 | 3 | -export([ 4 | init/1, 5 | insert/2, 6 | to_binary/1, 7 | 8 | is_in_set/2, 9 | 10 | init_matcher/1, 11 | is_match/2, 12 | filter_size/1, 13 | 14 | bit_set/2, 15 | make_hash/2 16 | ]). 17 | 18 | -define(NIL, []). 19 | -define(DEFAULT_FILTER_SIZE, 16). 20 | 21 | 22 | -record(?MODULE, {size_b = ?DEFAULT_FILTER_SIZE, 23 | filter_values = #{} :: #{binary() => ?NIL} 24 | }). 25 | 26 | -type hash() :: {1..2040, 1..2040} | {0, 0}. 27 | -type filter_size() :: 16..255. 28 | 29 | -record(matcher, 30 | {current_bit_size = ?DEFAULT_FILTER_SIZE * 8, 31 | values :: [binary()], 32 | hashes :: [hash()] 33 | }). 34 | 35 | -opaque state() :: #?MODULE{}. 36 | -opaque mstate() :: #matcher{}. 37 | 38 | -type filter_spec() :: #{filters := [binary()], 39 | match_unfiltered => boolean()}. 40 | 41 | -export_type([ 42 | state/0, 43 | mstate/0, 44 | filter_size/0, 45 | filter_spec/0 46 | ]). 47 | 48 | -spec init_matcher(filter_spec()) -> mstate(). 49 | init_matcher(Spec) -> 50 | init_matcher(Spec, ?DEFAULT_FILTER_SIZE). 51 | 52 | init_matcher(#{filters := [_|_] = Values} = Spec, SizeB) -> 53 | MatchUnfiltered = maps:get(match_unfiltered, Spec, false), 54 | BitSize = SizeB * 8, 55 | %% pre-calculate matchine hashes 56 | Hashes0 = [make_hash(Value, BitSize) || Value <- Values], 57 | Hashes = case MatchUnfiltered of 58 | true -> 59 | %% the 0th bit is reserved to indicate the presence of entries 60 | %% that do not include a filter 61 | [{0, 0} | Hashes0]; 62 | false -> 63 | Hashes0 64 | end, 65 | 66 | #matcher{current_bit_size = BitSize, 67 | values = Values, 68 | hashes = Hashes}. 69 | 70 | -spec is_match(binary(), mstate()) -> 71 | {retry_with, mstate()} | boolean(). 72 | is_match(Filter, #matcher{hashes = [FirstHash | _], 73 | values = Values, 74 | current_bit_size = BitSize}) 75 | when byte_size(Filter) > 0 andalso 76 | bit_size(Filter) =/= BitSize -> 77 | MatchUnfiltered = FirstHash == {0, 0}, 78 | NewMatcher = init_matcher(#{filters => Values, 79 | match_unfiltered => MatchUnfiltered}, 80 | byte_size(Filter)), 81 | {retry_with, NewMatcher}; 82 | is_match(Filter, #matcher{hashes = Hashes}) 83 | when is_binary(Filter) andalso 84 | byte_size(Filter) > 0 -> 85 | lists:any(fun (Hash) -> 86 | is_in_set(Hash, Filter) 87 | end, Hashes); 88 | is_match(<<>>, #matcher{hashes = [Hash1 | _]}) -> 89 | Hash1 == {0, 0}; 90 | is_match(_Filter, undefined) -> 91 | %% if no reader filter is set 92 | true. 93 | 94 | -spec filter_size(mstate()) -> non_neg_integer(). 95 | filter_size(#matcher{current_bit_size = BitSz}) -> 96 | BitSz div 8. 97 | 98 | 99 | -spec init(filter_size()) -> state(). 100 | init(SizeB) when is_integer(SizeB) andalso 101 | SizeB >= 16 andalso 102 | SizeB =< 255 -> 103 | #?MODULE{size_b = SizeB}; 104 | init(SizeB) -> 105 | exit({invalid_filter_size, SizeB}). 106 | 107 | -spec insert(binary(), state()) -> state(). 108 | insert(Value, #?MODULE{filter_values = Values} = State) 109 | when is_binary(Value) andalso 110 | not is_map_key(Value, Values) -> 111 | State#?MODULE{filter_values = Values#{Value => ?NIL}}; 112 | insert(_Value, State) -> 113 | State. 114 | 115 | -spec to_binary(state()) -> binary(). 116 | to_binary(#?MODULE{filter_values = Values}) 117 | when map_size(Values) == 0 -> 118 | <<>>; 119 | to_binary(#?MODULE{filter_values = Values}) 120 | when map_size(Values) == 1 andalso 121 | is_map_key(<<>>, Values) -> 122 | <<>>; 123 | to_binary(#?MODULE{size_b = SizeB, 124 | filter_values = Values}) -> 125 | BitSize = SizeB * 8, 126 | B0 = <<0:BitSize>>, 127 | B = maps:fold(fun(K, _, Acc) -> 128 | {H1, H2} = make_hash(K, BitSize), 129 | set_bits(H1, H2, Acc) 130 | end, B0, Values), 131 | B. 132 | 133 | is_in_set({H1, H2}, Bin) when is_binary(Bin) -> 134 | bit_set(H1, Bin) andalso bit_set(H2, Bin); 135 | is_in_set(Value, Bin) when is_binary(Value) andalso 136 | is_binary(Bin) -> 137 | {H1, H2} = make_hash(Value, bit_size(Bin)), 138 | bit_set(H1, Bin) andalso bit_set(H2, Bin). 139 | 140 | % INTERNAL 141 | 142 | %% returns two has values in the range 1..BitSize-1 143 | %% bit 0 is reserved to indicate the presence of an unfiltered entry 144 | %% in a chunk 145 | make_hash(<<>>, _BitSize) -> 146 | {0, 0}; 147 | make_hash(Value, BitSize) when is_binary(Value) -> 148 | H1 = erlang:phash2(Value, BitSize-1) + 1, 149 | H2 = erlang:phash2([Value], BitSize-1) + 1, 150 | {H1, H2}. 151 | 152 | set_bits(I, I, Bin) -> 153 | set_bit(I, Bin); 154 | set_bits(BeforeI, AfterI, Bin) when BeforeI < AfterI -> 155 | %% ordered case 156 | BetweenN = AfterI - BeforeI - 1, %% in between 157 | case Bin of 158 | <<_:BeforeI/bitstring, 1:1, 159 | _:BetweenN/bitstring, 1:1, _/bitstring>> -> 160 | %% already set 161 | Bin; 162 | <> -> 164 | <> 165 | end; 166 | set_bits(I1, I2, Bin) -> 167 | set_bits(I2, I1, Bin). 168 | 169 | set_bit(N, Bin) -> 170 | case Bin of 171 | <<_:N/bitstring, 1:1, _/bitstring>> -> 172 | Bin; 173 | <> -> 174 | <> 175 | end. 176 | 177 | 178 | bit_set(N, Bin) -> 179 | case Bin of 180 | <<_:N, 1:1, _/bitstring>> -> 181 | true; 182 | _ -> 183 | false 184 | end. 185 | -------------------------------------------------------------------------------- /test/osiris_tracking_SUITE.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_tracking_SUITE). 9 | 10 | -compile(nowarn_export_all). 11 | -compile(export_all). 12 | 13 | -export([]). 14 | 15 | % -include_lib("common_test/include/ct.hrl"). 16 | -include_lib("eunit/include/eunit.hrl"). 17 | 18 | -define(TRK_TYPE_SEQUENCE, 0). 19 | -define(TRK_TYPE_OFFSET, 1). 20 | -define(TRK_TYPE_TIMESTAMP, 2). 21 | %%%=================================================================== 22 | %%% Common Test callbacks 23 | %%%=================================================================== 24 | 25 | all() -> 26 | [{group, tests}]. 27 | 28 | all_tests() -> [basics, 29 | max_writers, 30 | recover]. 31 | 32 | groups() -> 33 | [{tests, [], all_tests()}]. 34 | 35 | init_per_suite(Config) -> 36 | Config. 37 | 38 | end_per_suite(_Config) -> 39 | ok. 40 | 41 | init_per_group(_Group, Config) -> 42 | Config. 43 | 44 | end_per_group(_Group, _Config) -> 45 | ok. 46 | 47 | init_per_testcase(_TestCase, Config) -> 48 | Config. 49 | 50 | end_per_testcase(_TestCase, _Config) -> 51 | ok. 52 | 53 | %%%=================================================================== 54 | %%% Test cases 55 | %%%=================================================================== 56 | 57 | basics(_Config) -> 58 | T0 = osiris_tracking:init(undefined, #{}), 59 | ChId1 = ?LINE, 60 | T1 = osiris_tracking:add(<<"w1">>, sequence, 55, ChId1, T0), 61 | ?assert(osiris_tracking:needs_flush(T1)), 62 | ?assertEqual({ok, {ChId1, 55}}, osiris_tracking:query(<<"w1">>, sequence, T1)), 63 | ?assertEqual({error, not_found}, osiris_tracking:query(<<"w2">>, sequence, T1)), 64 | {Trailer1, T2} = osiris_tracking:flush(T1), 65 | ?assert(false == osiris_tracking:needs_flush(T2)), 66 | ?assertMatch(<>, 69 | iolist_to_binary(Trailer1)), 70 | 71 | ChId2 = ?LINE, 72 | T3 = osiris_tracking:add(<<"t1">>, offset, 99, ChId2, T2), 73 | ?assertEqual({ok, 99}, osiris_tracking:query(<<"t1">>, offset, T3)), 74 | {Trailer2, T4} = osiris_tracking:flush(T3), 75 | ?assertMatch(<>, 78 | iolist_to_binary(Trailer2)), 79 | 80 | ChId3 = ?LINE, 81 | Now = erlang:system_time(millisecond), 82 | T5 = osiris_tracking:add(<<"t2">>, timestamp, Now, ChId3, T4), 83 | ?assertEqual({ok, Now}, osiris_tracking:query(<<"t2">>, timestamp, T5)), 84 | {Trailer3, T6} = osiris_tracking:flush(T5), 85 | ?assertMatch(<>, 88 | iolist_to_binary(Trailer3)), 89 | 90 | %% ensure negative timestamps work (although they shouldn't be used in practice) 91 | ChId4 = ?LINE, 92 | NegativeTs = -9, 93 | T7 = osiris_tracking:add(<<"t3">>, timestamp, NegativeTs, ChId4, T6), 94 | ?assertEqual({ok, NegativeTs}, osiris_tracking:query(<<"t3">>, timestamp, T7)), 95 | {Trailer4, T8} = osiris_tracking:flush(T7), 96 | ?assertMatch(<>, 99 | iolist_to_binary(Trailer4)), 100 | 101 | {Snap1, _T9} = osiris_tracking:snapshot(99, NegativeTs, T8), 102 | ?assertMatch(<>, iolist_to_binary(Snap1)), 119 | %% tracking offsets lower than first offset in stream should be discarded 120 | %% tracking timestamps lower than first timestamp in stream should be discarded 121 | {Snap2, _T10} = osiris_tracking:snapshot(100, Now+1, T8), 122 | ?assertMatch(<>, iolist_to_binary(Snap2)), 127 | ok. 128 | 129 | max_writers(_Config) -> 130 | Trk0 = osiris_tracking:init(undefined, #{max_sequences => 4}), 131 | Trk1 = lists:foldl( 132 | fun(I, T0) -> 133 | osiris_tracking:add(integer_to_binary(I), sequence, I, I, T0) 134 | end, Trk0, lists:seq(1, 8)), 135 | [?assertEqual({ok, {I, I}}, osiris_tracking:query(integer_to_binary(I), sequence, Trk1)) 136 | || I <- lists:seq(1, 8)], 137 | 138 | {_, Trk} = osiris_tracking:snapshot(1, 1, Trk1), 139 | [?assertEqual({error, not_found}, osiris_tracking:query(integer_to_binary(I), sequence, Trk)) 140 | || I <- lists:seq(1, 4)], 141 | 142 | [?assertEqual({ok, {I, I}}, osiris_tracking:query(integer_to_binary(I), sequence, Trk)) 143 | || I <- lists:seq(5, 8)], 144 | ok. 145 | 146 | recover(_Config) -> 147 | ChId1 = ?LINE, 148 | Now = erlang:system_time(millisecond), 149 | SnapBin = <>, 162 | 163 | T0 = osiris_tracking:init(SnapBin, #{}), 164 | ?assertEqual({ok, {ChId1, 55}}, osiris_tracking:query(<<"w1">>, sequence, T0)), 165 | ?assertEqual({ok, 99}, osiris_tracking:query(<<"t1">>, offset, T0)), 166 | ?assertEqual({ok, Now}, osiris_tracking:query(<<"t2">>, timestamp, T0)), 167 | 168 | Trailer = <>, 180 | 181 | ChId2 = ?LINE, 182 | T1 = osiris_tracking:append_trailer(ChId2, Trailer, T0), 183 | ?assertEqual({ok, {ChId1, 55}}, osiris_tracking:query(<<"w1">>, sequence, T1)), 184 | ?assertEqual({ok, {ChId2, 77}}, osiris_tracking:query(<<"w2">>, sequence, T1)), 185 | ?assertEqual({ok, 99}, osiris_tracking:query(<<"t1">>, offset, T1)), 186 | ?assertEqual({ok, Now}, osiris_tracking:query(<<"t2">>, timestamp, T1)), 187 | ?assertEqual({ok, 103}, osiris_tracking:query(<<"t3">>, offset, T1)), 188 | ?assertEqual({ok, 11}, osiris_tracking:query(<<"t4">>, timestamp, T1)), 189 | ok. 190 | -------------------------------------------------------------------------------- /test/osiris_util_SUITE.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_util_SUITE). 9 | 10 | -compile(nowarn_export_all). 11 | -compile(export_all). 12 | 13 | -export([]). 14 | 15 | -include_lib("common_test/include/ct.hrl"). 16 | -include_lib("eunit/include/eunit.hrl"). 17 | 18 | %%%=================================================================== 19 | %%% Common Test callbacks 20 | %%%=================================================================== 21 | 22 | all() -> 23 | [{group, tests}]. 24 | 25 | all_tests() -> 26 | [to_base64uri_test, 27 | inet_tls_enabled, 28 | replication_over_tls_configuration_with_optfile, 29 | replication_over_tls_configuration_with_opt]. 30 | 31 | groups() -> 32 | [{tests, [], all_tests()}]. 33 | 34 | init_per_suite(Config) -> 35 | Config. 36 | 37 | end_per_suite(_Config) -> 38 | ok. 39 | 40 | init_per_group(_Group, Config) -> 41 | Config. 42 | 43 | end_per_group(_Group, _Config) -> 44 | ok. 45 | 46 | init_per_testcase(_TestCase, Config) -> 47 | Config. 48 | 49 | end_per_testcase(_TestCase, _Config) -> 50 | ok. 51 | 52 | %%%=================================================================== 53 | %%% Test cases 54 | %%%=================================================================== 55 | 56 | to_base64uri_test(_Config) -> 57 | ?assertEqual("Myqueue", osiris_util:to_base64uri("Myqueue")), 58 | ?assertEqual("my__queue", osiris_util:to_base64uri("my%*queue")), 59 | ?assertEqual("my99___queue", 60 | osiris_util:to_base64uri("my99_[]queue")). 61 | 62 | inet_tls_enabled(_) -> 63 | InitArgs = init:get_arguments(), 64 | ?assert(osiris_util:inet_tls_enabled(InitArgs 65 | ++ [{proto_dist, ["inet_tls"]}])), 66 | ?assertNot(osiris_util:inet_tls_enabled(InitArgs)), 67 | ok. 68 | 69 | replication_over_tls_configuration_with_optfile(Config) -> 70 | ExpectedOkConfig = 71 | [{replication_transport, ssl}, 72 | {replication_server_ssl_options, 73 | [{cacertfile, "/etc/rabbitmq/ca_certificate.pem"}, 74 | {certfile, "/etc/rabbitmq/server_certificate.pem"}, 75 | {keyfile, "/etc/rabbitmq/server_key.pem"}, 76 | {secure_renegotiate, true}, 77 | {verify, verify_peer}, 78 | {fail_if_no_peer_cert, true}]}, 79 | {replication_client_ssl_options, 80 | [{cacertfile, "/etc/rabbitmq/ca_certificate.pem"}, 81 | {certfile, "/etc/rabbitmq/client_certificate.pem"}, 82 | {keyfile, "/etc/rabbitmq/client_key.pem"}, 83 | {secure_renegotiate, true}, 84 | {verify, verify_peer}, 85 | {fail_if_no_peer_cert, true}]}], 86 | [begin 87 | InitArgs = 88 | [{proto_dist, ["inet_tls"]}, 89 | {ssl_dist_optfile, [?config(data_dir, Config) ++ File]}], 90 | ?assertEqual(ExpectedOkConfig, 91 | replication_over_tls_configuration(InitArgs)) 92 | end 93 | || File 94 | <- ["inter_node_tls_server_client_ok.config", 95 | "inter_node_tls_client_server_ok.config"]], 96 | 97 | FileBroken = 98 | ?config(data_dir, Config) ++ "inter_node_tls_broken.config", 99 | InitArgsBroken = 100 | [{proto_dist, ["inet_tls"]}, {ssl_dist_optfile, [FileBroken]}], 101 | ?assertEqual([], replication_over_tls_configuration(InitArgsBroken)), 102 | 103 | FileNotFound = 104 | ?config(data_dir, Config) ++ "inter_node_tls_not_found.config", 105 | InitArgsNotFound = 106 | [{proto_dist, ["inet_tls"]}, {ssl_dist_optfile, [FileNotFound]}], 107 | ?assertEqual([], 108 | replication_over_tls_configuration(InitArgsNotFound)), 109 | 110 | ok. 111 | 112 | replication_over_tls_configuration_with_opt(_) -> 113 | InitArgs = 114 | [{proto_dist, ["inet_tls"]}, 115 | {ssl_dist_opt, 116 | ["server_cacertfile", "/etc/rabbitmq/ca_certificate.pem"]}, 117 | {ssl_dist_opt, 118 | ["server_certfile", "/etc/rabbitmq/server_certificate.pem"]}, 119 | {ssl_dist_opt, ["server_keyfile", "/etc/rabbitmq/server_key.pem"]}, 120 | {ssl_dist_opt, ["server_verify", "verify_peer"]}, 121 | {ssl_dist_opt, ["server_fail_if_no_peer_cert", "true"]}, 122 | {ssl_dist_opt, 123 | ["client_cacertfile", "/etc/rabbitmq/ca_certificate.pem"]}, 124 | {ssl_dist_opt, 125 | ["client_certfile", "/etc/rabbitmq/client_certificate.pem"]}, 126 | {ssl_dist_opt, ["client_keyfile", "/etc/rabbitmq/client_key.pem"]}, 127 | {ssl_dist_opt, ["client_verify", "verify_peer"]}, 128 | {ssl_dist_opt, 129 | ["server_secure_renegotiate", 130 | "true", 131 | "client_secure_renegotiate", 132 | "true"]}], 133 | 134 | ?assertEqual([{replication_transport, ssl}, 135 | {replication_server_ssl_options, 136 | [{cacertfile, "/etc/rabbitmq/ca_certificate.pem"}, 137 | {certfile, "/etc/rabbitmq/server_certificate.pem"}, 138 | {keyfile, "/etc/rabbitmq/server_key.pem"}, 139 | {verify, verify_peer}, 140 | {fail_if_no_peer_cert, true}, 141 | {secure_renegotiate, true}]}, 142 | {replication_client_ssl_options, 143 | [{cacertfile, "/etc/rabbitmq/ca_certificate.pem"}, 144 | {certfile, "/etc/rabbitmq/client_certificate.pem"}, 145 | {keyfile, "/etc/rabbitmq/client_key.pem"}, 146 | {verify, verify_peer}, 147 | {secure_renegotiate, true}]}], 148 | replication_over_tls_configuration(InitArgs)), 149 | 150 | ExtraInitArgs = 151 | [{proto_dist, ["inet_tls"]}, 152 | {ssl_dist_opt, 153 | ["server_verify_fun", 154 | "{some_module,some_function,some_initial_state}"]}, 155 | {ssl_dist_opt, ["server_crl_check", "true"]}, 156 | {ssl_dist_opt, 157 | ["server_crl_cache", "{ssl_crl_cache, {internal, []}}"]}, 158 | {ssl_dist_opt, ["server_reuse_sessions", "save"]}, 159 | {ssl_dist_opt, ["server_depth", "1"]}, 160 | {ssl_dist_opt, ["server_hibernate_after", "10"]}, 161 | {ssl_dist_opt, 162 | ["server_ciphers", "TLS_AES_256_GCM_SHA384:TLS_AES_128_GCM_SHA256"]}, 163 | {ssl_dist_opt, ["server_dhfile", "/some/file"]}, 164 | {ssl_dist_opt, ["server_password", "bunnies"]}], 165 | 166 | ?assertEqual([{replication_transport, ssl}, 167 | {replication_server_ssl_options, 168 | [{verify_fun, 169 | {some_module, some_function, some_initial_state}}, 170 | {crl_check, true}, 171 | {crl_cache, {ssl_crl_cache, {internal, []}}}, 172 | {reuse_sessions, save}, 173 | {depth, 1}, 174 | {hibernate_after, 10}, 175 | {ciphers, "TLS_AES_256_GCM_SHA384:TLS_AES_128_GCM_SHA256"}, 176 | {dhfile, "/some/file"}, 177 | {password, "bunnies"}]}, 178 | {replication_client_ssl_options, []}], 179 | replication_over_tls_configuration(ExtraInitArgs)), 180 | 181 | ok. 182 | 183 | replication_over_tls_configuration(Args) -> 184 | osiris_util:replication_over_tls_configuration(Args, 185 | fun file:consult/1, 186 | fun tls_replication_log/3). 187 | 188 | tls_replication_log(_Level, Fmt, Args) -> 189 | ct:log(Fmt, Args). 190 | -------------------------------------------------------------------------------- /src/osiris_tracking.erl: -------------------------------------------------------------------------------- 1 | -module(osiris_tracking). 2 | 3 | -export([ 4 | init/2, 5 | add/5, 6 | flush/1, 7 | snapshot/3, 8 | query/3, 9 | append_trailer/3, 10 | needs_flush/1, 11 | is_empty/1, 12 | overview/1, 13 | max_sequences/1 14 | ]). 15 | 16 | 17 | -define(TRK_TYPE_SEQUENCE, 0). 18 | -define(TRK_TYPE_OFFSET, 1). 19 | -define(TRK_TYPE_TIMESTAMP, 2). 20 | -define(MAX_SEQUENCES, 255). 21 | %% holds static or rarely changing fields 22 | -record(cfg, {max_sequences = ?MAX_SEQUENCES :: non_neg_integer()}). 23 | 24 | -type config() :: #{max_sequences => non_neg_integer()}. 25 | 26 | -type tracking_id() :: binary(). 27 | -type tracking_type() :: sequence | offset | timestamp. 28 | -type tracking() :: non_neg_integer() | osiris:offset() | osiris:timestamp(). 29 | 30 | -record(?MODULE, {cfg = #cfg{} :: #cfg{}, 31 | pending = init_pending() :: #{sequences | offsets | timestamps => 32 | #{tracking_id() => tracking()}}, 33 | sequences = #{} :: #{tracking_id() => {osiris:offset(), non_neg_integer()}}, 34 | offsets = #{} :: #{tracking_id() => osiris:offset()}, 35 | timestamps = #{} :: #{tracking_id() => osiris:timestamp()} 36 | }). 37 | 38 | -opaque state() :: #?MODULE{}. 39 | 40 | -export_type([ 41 | state/0, 42 | config/0, 43 | tracking_type/0, 44 | tracking_id/0 45 | ]). 46 | 47 | init_pending() -> 48 | #{sequences => #{}, 49 | offsets => #{}, 50 | timestamps => #{}}. 51 | 52 | -spec init(undefined | binary(), config() | state()) -> state(). 53 | init(From, Config) when is_map(Config) -> 54 | init(From, #?MODULE{cfg = 55 | #cfg{max_sequences = 56 | maps:get(max_sequences, Config, ?MAX_SEQUENCES)}}); 57 | init(undefined, #?MODULE{cfg = Cfg}) -> 58 | #?MODULE{cfg = Cfg}; 59 | init(Bin, #?MODULE{cfg = Cfg}) when is_binary(Bin) -> 60 | parse_snapshot(Bin, #?MODULE{cfg = Cfg}). 61 | 62 | -spec add(tracking_id(), tracking_type(), tracking(), osiris:offset() | undefined, 63 | state()) -> state(). 64 | add(TrkId, TrkType, TrkData, ChunkId, 65 | #?MODULE{pending = Pend0} = State) when is_integer(TrkData) andalso 66 | byte_size(TrkId) =< 256 -> 67 | Type = plural(TrkType), 68 | Trackings0 = maps:get(Type, Pend0), 69 | Trackings1 = Trackings0#{TrkId => TrkData}, 70 | Pend = Pend0#{Type := Trackings1}, 71 | update_tracking(TrkId, TrkType, TrkData, 72 | ChunkId, State#?MODULE{pending = Pend}). 73 | 74 | -spec flush(state()) -> {iodata(), state()}. 75 | flush(#?MODULE{pending = Pending} = State) -> 76 | TData = maps:fold(fun(TrkType, TrackingMap, Acc) -> 77 | T = case TrkType of 78 | sequences -> 79 | ?TRK_TYPE_SEQUENCE; 80 | offsets -> 81 | ?TRK_TYPE_OFFSET; 82 | timestamps -> 83 | ?TRK_TYPE_TIMESTAMP 84 | end, 85 | maps:fold(fun(TrkId, TrkData, Acc0) -> 86 | [<> | Acc0] 90 | end, Acc, TrackingMap) 91 | end, [], Pending), 92 | {TData, State#?MODULE{pending = init_pending()}}. 93 | 94 | -spec snapshot(osiris:offset(), osiris:timestamp(), state()) -> 95 | {iodata(), state()}. 96 | snapshot(FirstOffset, FirstTimestamp, 97 | #?MODULE{cfg = #cfg{max_sequences = MaxSeqs}, 98 | sequences = Seqs0, 99 | offsets = Offsets0, 100 | timestamps = Timestamps0} = State) -> 101 | %% discard any tracking info with offsets lower than the first offset 102 | %% in the stream 103 | Offsets = maps:filter(fun(_, Off) -> Off >= FirstOffset end, Offsets0), 104 | %% discard any tracking info with timestamps lower than the first 105 | %% timestamp in the stream 106 | Timestamps = maps:filter(fun(_, Ts) -> Ts >= FirstTimestamp end, Timestamps0), 107 | Seqs = trim_sequences(MaxSeqs, Seqs0), 108 | 109 | Data0 = maps:fold(fun(TrkId, {ChId, Seq} , Acc) -> 110 | [<> 115 | | Acc] 116 | end, [], Seqs), 117 | Data1 = maps:fold(fun(TrkId, Offs, Acc) -> 118 | [<> 122 | | Acc] 123 | end, Data0, Offsets), 124 | Data2 = maps:fold(fun(TrkId, Ts, Acc) -> 125 | [<> 129 | | Acc] 130 | end, Data1, Timestamps), 131 | {Data2, State#?MODULE{pending = init_pending(), 132 | sequences = Seqs, 133 | offsets = Offsets, 134 | timestamps = Timestamps}}. 135 | 136 | -spec query(tracking_id(), TrkType :: tracking_type(), state()) -> 137 | {ok, term()} | {error, not_found}. 138 | query(TrkId, sequence, #?MODULE{sequences = Seqs}) 139 | when is_binary(TrkId) -> 140 | case Seqs of 141 | #{TrkId := Tracking} -> 142 | {ok, Tracking}; 143 | _ -> 144 | {error, not_found} 145 | end; 146 | query(TrkId, offset, #?MODULE{offsets = Offs}) 147 | when is_binary(TrkId) -> 148 | case Offs of 149 | #{TrkId := Tracking} -> 150 | {ok, Tracking}; 151 | _ -> 152 | {error, not_found} 153 | end; 154 | query(TrkId, timestamp, #?MODULE{timestamps = Timestamps}) 155 | when is_binary(TrkId) -> 156 | case Timestamps of 157 | #{TrkId := Tracking} -> 158 | {ok, Tracking}; 159 | _ -> 160 | {error, not_found} 161 | end. 162 | 163 | -spec append_trailer(osiris:offset(), binary(), state()) -> 164 | state(). 165 | append_trailer(ChId, Bin, State) -> 166 | parse_trailer(Bin, ChId, State). 167 | 168 | -spec needs_flush(state()) -> boolean(). 169 | needs_flush(#?MODULE{pending = #{sequences := Sequences, 170 | offsets := Offsets, 171 | timestamps := Timestamps}}) -> 172 | map_size(Sequences) > 0 orelse 173 | map_size(Offsets) > 0 orelse 174 | map_size(Timestamps) > 0. 175 | 176 | -spec is_empty(state()) -> boolean(). 177 | is_empty(#?MODULE{sequences = Seqs, offsets = Offs, timestamps = Timestamps}) -> 178 | map_size(Seqs) + map_size(Offs) + map_size(Timestamps) == 0. 179 | 180 | -spec overview(state()) -> map(). %% TODO refine 181 | overview(#?MODULE{sequences = Seqs, offsets = Offs, timestamps = Timestamps}) -> 182 | #{offsets => Offs, 183 | sequences => Seqs, 184 | timestamps => Timestamps}. 185 | 186 | -spec max_sequences(state()) -> non_neg_integer(). 187 | max_sequences(#?MODULE{cfg = #cfg{max_sequences = MaxSequences}}) -> 188 | MaxSequences. 189 | 190 | %% INTERNAL 191 | plural(sequence) -> sequences; 192 | plural(offset) -> offsets; 193 | plural(timestamp) -> timestamps. 194 | 195 | update_tracking(TrkId, sequence, Tracking, ChId, 196 | #?MODULE{sequences = Seqs0} = State) when is_integer(ChId) -> 197 | State#?MODULE{sequences = Seqs0#{TrkId => {ChId, Tracking}}}; 198 | update_tracking(TrkId, offset, Tracking, _ChId, 199 | #?MODULE{offsets = Offs} = State) -> 200 | State#?MODULE{offsets = Offs#{TrkId => Tracking}}; 201 | update_tracking(TrkId, timestamp, Tracking, _ChId, 202 | #?MODULE{timestamps = Timestamps} = State) -> 203 | State#?MODULE{timestamps = Timestamps#{TrkId => Tracking}}. 204 | 205 | parse_snapshot(<<>>, State) -> 206 | State; 207 | parse_snapshot(<>, 212 | #?MODULE{sequences = Seqs} = State) -> 213 | parse_snapshot(Rem, State#?MODULE{sequences = Seqs#{TrkId => {ChId, Seq}}}); 214 | parse_snapshot(<>, 218 | #?MODULE{offsets = Offsets} = State) -> 219 | parse_snapshot(Rem, State#?MODULE{offsets = Offsets#{TrkId => Offs}}); 220 | parse_snapshot(<>, 224 | #?MODULE{timestamps = Timestamps} = State) -> 225 | parse_snapshot(Rem, State#?MODULE{timestamps = Timestamps#{TrkId => Ts}}). 226 | 227 | parse_trailer(<<>>, _ChId, State) -> 228 | State; 229 | parse_trailer(<>, 233 | ChId, #?MODULE{sequences = Seqs} = State) -> 234 | parse_trailer(Rem, ChId, State#?MODULE{sequences = Seqs#{TrkId => {ChId, Seq}}}); 235 | parse_trailer(<>, 239 | ChId, #?MODULE{offsets = Offsets} = State) -> 240 | parse_trailer(Rem, ChId, State#?MODULE{offsets = Offsets#{TrkId => Offs}}); 241 | parse_trailer(<>, 245 | ChId, #?MODULE{timestamps = Timestamps} = State) -> 246 | parse_trailer(Rem, ChId, State#?MODULE{timestamps = Timestamps#{TrkId => Ts}}). 247 | 248 | trim_sequences(Max, Sequences) when map_size(Sequences) =< Max -> 249 | Sequences; 250 | trim_sequences(Max, Sequences) -> 251 | Sorted = lists:sort(fun ({_, {C0, _}}, {_, {C1, _}}) -> 252 | C0 < C1 253 | end, maps:to_list(Sequences)), 254 | maps:from_list(lists:nthtail(map_size(Sequences) - Max, Sorted)). 255 | -------------------------------------------------------------------------------- /LICENSE-APACHE2: -------------------------------------------------------------------------------- 1 | 2 | Apache License 3 | Version 2.0, January 2004 4 | https://www.apache.org/licenses/ 5 | 6 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 7 | 8 | 1. Definitions. 9 | 10 | "License" shall mean the terms and conditions for use, reproduction, 11 | and distribution as defined by Sections 1 through 9 of this document. 12 | 13 | "Licensor" shall mean the copyright owner or entity authorized by 14 | the copyright owner that is granting the License. 15 | 16 | "Legal Entity" shall mean the union of the acting entity and all 17 | other entities that control, are controlled by, or are under common 18 | control with that entity. For the purposes of this definition, 19 | "control" means (i) the power, direct or indirect, to cause the 20 | direction or management of such entity, whether by contract or 21 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 22 | outstanding shares, or (iii) beneficial ownership of such entity. 23 | 24 | "You" (or "Your") shall mean an individual or Legal Entity 25 | exercising permissions granted by this License. 26 | 27 | "Source" form shall mean the preferred form for making modifications, 28 | including but not limited to software source code, documentation 29 | source, and configuration files. 30 | 31 | "Object" form shall mean any form resulting from mechanical 32 | transformation or translation of a Source form, including but 33 | not limited to compiled object code, generated documentation, 34 | and conversions to other media types. 35 | 36 | "Work" shall mean the work of authorship, whether in Source or 37 | Object form, made available under the License, as indicated by a 38 | copyright notice that is included in or attached to the work 39 | (an example is provided in the Appendix below). 40 | 41 | "Derivative Works" shall mean any work, whether in Source or Object 42 | form, that is based on (or derived from) the Work and for which the 43 | editorial revisions, annotations, elaborations, or other modifications 44 | represent, as a whole, an original work of authorship. For the purposes 45 | of this License, Derivative Works shall not include works that remain 46 | separable from, or merely link (or bind by name) to the interfaces of, 47 | the Work and Derivative Works thereof. 48 | 49 | "Contribution" shall mean any work of authorship, including 50 | the original version of the Work and any modifications or additions 51 | to that Work or Derivative Works thereof, that is intentionally 52 | submitted to Licensor for inclusion in the Work by the copyright owner 53 | or by an individual or Legal Entity authorized to submit on behalf of 54 | the copyright owner. For the purposes of this definition, "submitted" 55 | means any form of electronic, verbal, or written communication sent 56 | to the Licensor or its representatives, including but not limited to 57 | communication on electronic mailing lists, source code control systems, 58 | and issue tracking systems that are managed by, or on behalf of, the 59 | Licensor for the purpose of discussing and improving the Work, but 60 | excluding communication that is conspicuously marked or otherwise 61 | designated in writing by the copyright owner as "Not a Contribution." 62 | 63 | "Contributor" shall mean Licensor and any individual or Legal Entity 64 | on behalf of whom a Contribution has been received by Licensor and 65 | subsequently incorporated within the Work. 66 | 67 | 2. Grant of Copyright License. Subject to the terms and conditions of 68 | this License, each Contributor hereby grants to You a perpetual, 69 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 70 | copyright license to reproduce, prepare Derivative Works of, 71 | publicly display, publicly perform, sublicense, and distribute the 72 | Work and such Derivative Works in Source or Object form. 73 | 74 | 3. Grant of Patent License. Subject to the terms and conditions of 75 | this License, each Contributor hereby grants to You a perpetual, 76 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 77 | (except as stated in this section) patent license to make, have made, 78 | use, offer to sell, sell, import, and otherwise transfer the Work, 79 | where such license applies only to those patent claims licensable 80 | by such Contributor that are necessarily infringed by their 81 | Contribution(s) alone or by combination of their Contribution(s) 82 | with the Work to which such Contribution(s) was submitted. If You 83 | institute patent litigation against any entity (including a 84 | cross-claim or counterclaim in a lawsuit) alleging that the Work 85 | or a Contribution incorporated within the Work constitutes direct 86 | or contributory patent infringement, then any patent licenses 87 | granted to You under this License for that Work shall terminate 88 | as of the date such litigation is filed. 89 | 90 | 4. Redistribution. You may reproduce and distribute copies of the 91 | Work or Derivative Works thereof in any medium, with or without 92 | modifications, and in Source or Object form, provided that You 93 | meet the following conditions: 94 | 95 | (a) You must give any other recipients of the Work or 96 | Derivative Works a copy of this License; and 97 | 98 | (b) You must cause any modified files to carry prominent notices 99 | stating that You changed the files; and 100 | 101 | (c) You must retain, in the Source form of any Derivative Works 102 | that You distribute, all copyright, patent, trademark, and 103 | attribution notices from the Source form of the Work, 104 | excluding those notices that do not pertain to any part of 105 | the Derivative Works; and 106 | 107 | (d) If the Work includes a "NOTICE" text file as part of its 108 | distribution, then any Derivative Works that You distribute must 109 | include a readable copy of the attribution notices contained 110 | within such NOTICE file, excluding those notices that do not 111 | pertain to any part of the Derivative Works, in at least one 112 | of the following places: within a NOTICE text file distributed 113 | as part of the Derivative Works; within the Source form or 114 | documentation, if provided along with the Derivative Works; or, 115 | within a display generated by the Derivative Works, if and 116 | wherever such third-party notices normally appear. The contents 117 | of the NOTICE file are for informational purposes only and 118 | do not modify the License. You may add Your own attribution 119 | notices within Derivative Works that You distribute, alongside 120 | or as an addendum to the NOTICE text from the Work, provided 121 | that such additional attribution notices cannot be construed 122 | as modifying the License. 123 | 124 | You may add Your own copyright statement to Your modifications and 125 | may provide additional or different license terms and conditions 126 | for use, reproduction, or distribution of Your modifications, or 127 | for any such Derivative Works as a whole, provided Your use, 128 | reproduction, and distribution of the Work otherwise complies with 129 | the conditions stated in this License. 130 | 131 | 5. Submission of Contributions. Unless You explicitly state otherwise, 132 | any Contribution intentionally submitted for inclusion in the Work 133 | by You to the Licensor shall be under the terms and conditions of 134 | this License, without any additional terms or conditions. 135 | Notwithstanding the above, nothing herein shall supersede or modify 136 | the terms of any separate license agreement you may have executed 137 | with Licensor regarding such Contributions. 138 | 139 | 6. Trademarks. This License does not grant permission to use the trade 140 | names, trademarks, service marks, or product names of the Licensor, 141 | except as required for reasonable and customary use in describing the 142 | origin of the Work and reproducing the content of the NOTICE file. 143 | 144 | 7. Disclaimer of Warranty. Unless required by applicable law or 145 | agreed to in writing, Licensor provides the Work (and each 146 | Contributor provides its Contributions) on an "AS IS" BASIS, 147 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 148 | implied, including, without limitation, any warranties or conditions 149 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 150 | PARTICULAR PURPOSE. You are solely responsible for determining the 151 | appropriateness of using or redistributing the Work and assume any 152 | risks associated with Your exercise of permissions under this License. 153 | 154 | 8. Limitation of Liability. In no event and under no legal theory, 155 | whether in tort (including negligence), contract, or otherwise, 156 | unless required by applicable law (such as deliberate and grossly 157 | negligent acts) or agreed to in writing, shall any Contributor be 158 | liable to You for damages, including any direct, indirect, special, 159 | incidental, or consequential damages of any character arising as a 160 | result of this License or out of the use or inability to use the 161 | Work (including but not limited to damages for loss of goodwill, 162 | work stoppage, computer failure or malfunction, or any and all 163 | other commercial damages or losses), even if such Contributor 164 | has been advised of the possibility of such damages. 165 | 166 | 9. Accepting Warranty or Additional Liability. While redistributing 167 | the Work or Derivative Works thereof, You may choose to offer, 168 | and charge a fee for, acceptance of support, warranty, indemnity, 169 | or other liability obligations and/or rights consistent with this 170 | License. However, in accepting such obligations, You may act only 171 | on Your own behalf and on Your sole responsibility, not on behalf 172 | of any other Contributor, and only if You agree to indemnify, 173 | defend, and hold each Contributor harmless for any liability 174 | incurred by, or claims asserted against, such Contributor by reason 175 | of your accepting any such warranty or additional liability. 176 | 177 | END OF TERMS AND CONDITIONS 178 | 179 | APPENDIX: How to apply the Apache License to your work. 180 | 181 | To apply the Apache License to your work, attach the following 182 | boilerplate notice, with the fields enclosed by brackets "[]" 183 | replaced with your own identifying information. (Don't include 184 | the brackets!) The text should be enclosed in the appropriate 185 | comment syntax for the file format. We also recommend that a 186 | file or class name and description of purpose be included on the 187 | same "printed page" as the copyright notice for easier 188 | identification within third-party archives. 189 | 190 | Copyright 2020-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 191 | 192 | Licensed under the Apache License, Version 2.0 (the "License"); 193 | you may not use this file except in compliance with the License. 194 | You may obtain a copy of the License at 195 | 196 | https://www.apache.org/licenses/LICENSE-2.0 197 | 198 | Unless required by applicable law or agreed to in writing, software 199 | distributed under the License is distributed on an "AS IS" BASIS, 200 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 201 | See the License for the specific language governing permissions and 202 | limitations under the License. 203 | -------------------------------------------------------------------------------- /src/osiris.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris). 9 | 10 | -include("osiris.hrl"). 11 | 12 | -export([write/2, 13 | write/4, 14 | write_tracking/3, 15 | read_tracking/1, 16 | read_tracking/2, 17 | read_tracking/3, 18 | fetch_writer_seq/2, 19 | init_reader/3, 20 | init_reader/4, 21 | register_offset_listener/2, 22 | register_offset_listener/3, 23 | update_retention/2, 24 | start_cluster/1, 25 | stop_cluster/1, 26 | 27 | start_writer/1, 28 | start_replica/2, 29 | stop_member/2, 30 | delete_member/2, 31 | 32 | delete_cluster/1, 33 | configure_logger/1, 34 | get_stats/1]). 35 | 36 | 37 | -type name() :: string() | binary(). 38 | -type config() :: 39 | #{name := name(), 40 | reference => term(), 41 | event_formatter => {module(), atom(), list()}, 42 | retention => [osiris:retention_spec()], 43 | atom() => term()}. 44 | 45 | -type mfarg() :: {module(), atom(), list()}. 46 | -type offset() :: non_neg_integer(). 47 | -type tracking_id() :: osiris_tracking:tracking_id(). 48 | -type tracking_type() :: osiris_tracking:tracking_type(). 49 | -type epoch() :: non_neg_integer(). 50 | -type milliseconds() :: non_neg_integer(). 51 | -type timestamp() :: integer(). % milliseconds since epoch 52 | -type tail_info() :: {NextOffset :: offset(), 53 | Last :: empty | {epoch(), offset(), osiris:timestamp()}}. 54 | -type compression_type() :: 0 | % no compression 55 | 1 | % gzip 56 | 2 | % snappy 57 | 3 | % lz4 58 | 4 | % zstd 59 | 5 | % reserved 60 | 6 | % reserved 61 | 7. % user defined 62 | -type offset_spec() :: 63 | first | 64 | last | 65 | next | 66 | {abs, offset()} | 67 | offset() | 68 | {timestamp, timestamp()}. 69 | -type retention_spec() :: 70 | {max_bytes, non_neg_integer()} | {max_age, milliseconds()}. 71 | -type writer_id() :: binary(). 72 | -type batch() :: {batch, NumRecords :: non_neg_integer(), 73 | compression_type(), 74 | UncompressedDataSize :: non_neg_integer(), 75 | iodata()}. 76 | -type filter_value() :: binary(). 77 | -type data() :: iodata() | 78 | batch() | 79 | {filter_value(), iodata() | batch()}. 80 | 81 | %% returned when reading 82 | -type entry() :: binary() | batch(). 83 | -type reader_options() :: #{transport => tcp | ssl, 84 | chunk_selector => all | user_data, 85 | filter_spec => osiris_bloom:filter_spec(), 86 | read_ahead => boolean() | non_neg_integer() 87 | }. 88 | 89 | -export_type([name/0, 90 | config/0, 91 | offset/0, 92 | epoch/0, 93 | tail_info/0, 94 | tracking_id/0, 95 | offset_spec/0, 96 | retention_spec/0, 97 | timestamp/0, 98 | writer_id/0, 99 | data/0, 100 | entry/0]). 101 | 102 | -spec start_cluster(config()) -> 103 | {ok, config()} | 104 | {error, term()} | 105 | {error, term(), config()}. 106 | start_cluster(Config00 = #{name := Name}) -> 107 | ?DEBUG("osiris: starting new cluster ~ts", [Name]), 108 | true = osiris_util:validate_base64uri(Name), 109 | %% ensure reference is set 110 | Config0 = maps:merge(#{reference => Name}, Config00), 111 | case start_writer(Config0) of 112 | {ok, Pid} -> 113 | Config = Config0#{leader_pid => Pid}, 114 | case start_replicas(Config) of 115 | {ok, ReplicaPids} -> 116 | {ok, Config#{replica_pids => ReplicaPids}} 117 | end; 118 | Error -> 119 | Error 120 | end. 121 | 122 | stop_cluster(Config) -> 123 | WriterNode = maps:get(leader_node, Config), 124 | ok = stop_member(WriterNode, Config), 125 | [ok = stop_member(N, Config) 126 | || N <- maps:get(replica_nodes, Config)], 127 | ok. 128 | 129 | -spec delete_cluster(config()) -> ok. 130 | delete_cluster(Config) -> 131 | [ok = delete_member(N, Config) 132 | || N <- maps:get(replica_nodes, Config)], 133 | WriterNode = maps:get(leader_node, Config), 134 | ok = delete_member(WriterNode, Config). 135 | 136 | -spec start_writer(osiris:config()) -> 137 | supervisor:startchild_ret(). 138 | start_writer(Config) -> 139 | Mod = get_writer_module(Config), 140 | Node = maps:get(leader_node, Config), 141 | osiris_member:start(Mod, Node, Config). 142 | 143 | -spec start_replica(node(), osiris:config()) -> 144 | supervisor:startchild_ret(). 145 | start_replica(Node, Config) -> 146 | Mod = maps:get(replica_mod, Config, osiris_replica), 147 | osiris_member:start(Mod, Node, Config). 148 | 149 | -spec stop_member(node(), osiris:config()) -> ok. 150 | stop_member(Node, Config) -> 151 | osiris_member:stop(Node, Config). 152 | 153 | -spec delete_member(node(), osiris:config()) -> 154 | ok | {error, not_found}. 155 | delete_member(Node, Config) -> 156 | osiris_member:delete(Node, Config). 157 | 158 | -spec write(Pid :: pid(), Data :: data()) -> ok. 159 | write(Pid, Data) -> 160 | osiris_writer:write(Pid, Data). 161 | 162 | -spec write(Pid :: pid(), 163 | WriterId :: binary() | undefined, 164 | CorrOrSeq :: non_neg_integer() | term(), 165 | Data :: data()) -> 166 | ok. 167 | write(Pid, WriterId, Corr, Data) -> 168 | osiris_writer:write(Pid, self(), WriterId, Corr, Data). 169 | 170 | -spec write_tracking(pid(), binary(), {tracking_type(), offset() | timestamp()} | offset()) -> ok. 171 | write_tracking(Pid, TrackingId, {_TrkType, _TrkData} = Tracking) -> 172 | osiris_writer:write_tracking(Pid, TrackingId, Tracking); 173 | %% for backwards compatiblity 174 | write_tracking(Pid, TrackingId, Offset) when is_integer(Offset) -> 175 | osiris_writer:write_tracking(Pid, TrackingId, {offset, Offset}). 176 | 177 | -spec read_tracking(pid()) -> map(). 178 | read_tracking(Pid) -> 179 | osiris_writer:read_tracking(Pid). 180 | 181 | %% for backwards compatiblity 182 | -spec read_tracking(pid(), binary()) -> {offset, offset()} | undefined. 183 | read_tracking(Pid, TrackingId) -> 184 | osiris_writer:read_tracking(Pid, offset, TrackingId). 185 | 186 | -spec read_tracking(pid(), tracking_type(), binary()) -> 187 | {tracking_type(), offset() | timestamp()} | undefined. 188 | read_tracking(Pid, TrackingType, TrackingId) -> 189 | osiris_writer:read_tracking(Pid, TrackingType, TrackingId). 190 | 191 | -spec fetch_writer_seq(pid(), binary()) -> 192 | non_neg_integer() | undefined. 193 | fetch_writer_seq(Pid, WriterId) 194 | when is_pid(Pid) andalso is_binary(WriterId) -> 195 | osiris_writer:query_writers(Pid, 196 | fun(W) -> 197 | case maps:get(WriterId, W, undefined) of 198 | undefined -> undefined; 199 | {_, Seq} -> Seq 200 | end 201 | end). 202 | 203 | %% @doc Initialise a new offset reader 204 | %% @param Pid the pid of a writer or replica process 205 | %% @param OffsetSpec specifies where in the log to attach the reader 206 | %% `first': Attach at first available offset. 207 | %% `last': Attach at the last available chunk offset or the next available offset 208 | %% if the log is empty. 209 | %% `next': Attach to the next chunk offset to be written. 210 | %% `{abs, offset()}': Attach at the provided offset. If this offset does not exist 211 | %% in the log it will error with `{error, {offset_out_of_range, Range}}' 212 | %% `offset()': Like `{abs, offset()}' but instead of erroring it will fall back 213 | %% to `first' (if lower than first offset in log) or `nextl' if higher than 214 | %% last offset in log. 215 | %% @returns `{ok, state()} | {error, Error}' when error can be 216 | %% `{offset_out_of_range, empty | {From :: offset(), To :: offset()}}' 217 | %% @end 218 | -spec init_reader(pid(), offset_spec(), osiris_log:counter_spec()) -> 219 | {ok, osiris_log:state()} | 220 | {error, 221 | {offset_out_of_range, empty | {offset(), offset()}}} | 222 | {error, {invalid_last_offset_epoch, offset(), offset()}}. 223 | init_reader(Pid, OffsetSpec, CounterSpec) -> 224 | init_reader(Pid, OffsetSpec, CounterSpec, #{transport => tcp, 225 | chunk_selector => user_data}). 226 | 227 | -spec init_reader(pid(), offset_spec(), osiris_log:counter_spec(), 228 | reader_options()) -> 229 | {ok, osiris_log:state()} | 230 | {error, {offset_out_of_range, empty | {offset(), offset()}}} | 231 | {error, {invalid_last_offset_epoch, offset(), offset()}}. 232 | init_reader(Pid, OffsetSpec, {_, _} = CounterSpec, Options) 233 | when is_pid(Pid) andalso node(Pid) =:= node() -> 234 | ?DEBUG("osiris: initialising reader. Spec: ~w", [OffsetSpec]), 235 | Ctx0 = osiris_util:get_reader_context(Pid), 236 | Ctx = Ctx0#{counter_spec => CounterSpec, 237 | options => Options}, 238 | osiris_log:init_offset_reader(OffsetSpec, Ctx). 239 | 240 | -spec register_offset_listener(pid(), offset()) -> ok. 241 | register_offset_listener(Pid, Offset) -> 242 | register_offset_listener(Pid, Offset, undefined). 243 | 244 | %% @doc 245 | %% Registers a one-off offset listener that will send an `{osiris_offset, offset()}' 246 | %% message when the osiris cluster committed offset moves beyond the provided offset 247 | %% @end 248 | -spec register_offset_listener(pid(), offset(), 249 | mfarg() | undefined) -> 250 | ok. 251 | register_offset_listener(Pid, Offset, EvtFormatter) -> 252 | Msg = {'$gen_cast', 253 | {register_offset_listener, self(), EvtFormatter, Offset}}, 254 | try 255 | erlang:send(Pid, Msg) 256 | catch 257 | error:_ -> 258 | ok 259 | end, 260 | ok. 261 | 262 | -spec update_retention(pid(), [osiris:retention_spec()]) -> 263 | ok | {error, term()}. 264 | update_retention(Pid, Retention) 265 | when is_pid(Pid) andalso is_list(Retention) -> 266 | Msg = {update_retention, Retention}, 267 | try 268 | case gen:call(Pid, '$gen_call', Msg) of 269 | {ok, ok} -> 270 | ok 271 | end 272 | catch 273 | _:Reason -> 274 | {error, Reason} 275 | end. 276 | 277 | start_replicas(Config) -> 278 | start_replicas(Config, maps:get(replica_nodes, Config), []). 279 | 280 | start_replicas(_Config, [], ReplicaPids) -> 281 | {ok, ReplicaPids}; 282 | start_replicas(Config, [Node | Nodes], ReplicaPids) -> 283 | try 284 | case start_replica(Node, Config) of 285 | {ok, Pid} -> 286 | start_replicas(Config, Nodes, [Pid | ReplicaPids]); 287 | {ok, Pid, _} -> 288 | start_replicas(Config, Nodes, [Pid | ReplicaPids]); 289 | {error, {already_started, Pid}} -> 290 | start_replicas(Config, Nodes, [Pid | ReplicaPids]); 291 | {error, Reason} -> 292 | Name = maps:get(name, Config, undefined), 293 | error_logger:info_msg("osiris:start_replicas for ~ts failed to start replica " 294 | "on ~w, reason: ~w", 295 | [Name, Node, Reason]), 296 | %% coordinator might try to start this replica in the future 297 | start_replicas(Config, Nodes, ReplicaPids) 298 | end 299 | catch 300 | _:_ -> 301 | %% coordinator might try to start this replica in the future 302 | start_replicas(Config, Nodes, ReplicaPids) 303 | end. 304 | 305 | -spec configure_logger(module()) -> ok. 306 | configure_logger(Module) -> 307 | persistent_term:put('$osiris_logger', Module). 308 | 309 | -spec get_stats(pid()) -> #{committed_chunk_id => integer(), 310 | first_chunk_id => integer()}. 311 | get_stats(Pid) 312 | when node(Pid) =:= node() -> 313 | #{shared := Shared} = osiris_util:get_reader_context(Pid), 314 | #{committed_chunk_id => osiris_log_shared:committed_chunk_id(Shared), 315 | first_chunk_id => osiris_log_shared:first_chunk_id(Shared), 316 | last_chunk_id => osiris_log_shared:last_chunk_id(Shared)}; 317 | get_stats(Pid) when is_pid(Pid) -> 318 | erpc:call(node(Pid), ?MODULE, ?FUNCTION_NAME, [Pid]). 319 | 320 | get_writer_module(Config) -> 321 | maps:get(writer_mod, Config, osiris_writer). 322 | -------------------------------------------------------------------------------- /src/osiris_util.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_util). 9 | 10 | -include("osiris.hrl"). 11 | -include_lib("kernel/include/file.hrl"). 12 | 13 | -export([validate_base64uri/1, 14 | to_base64uri/1, 15 | id/1, 16 | lists_find/2, 17 | hostname_from_node/0, 18 | get_replication_configuration_from_tls_dist/0, 19 | get_replication_configuration_from_tls_dist/1, 20 | get_replication_configuration_from_tls_dist/2, 21 | get_replica_listener_inet_address_family/0, 22 | partition_parallel/3, 23 | normalise_name/1, 24 | get_reader_context/1, 25 | cache_reader_context/6, 26 | is_dir/1, 27 | is_file/1 28 | ]). 29 | 30 | %% For testing 31 | -export([inet_tls_enabled/1, 32 | replication_over_tls_configuration/3]). 33 | 34 | -define(BASE64_URI_CHARS, 35 | "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz01" 36 | "23456789_-="). 37 | 38 | -spec validate_base64uri(string() | binary()) -> boolean(). 39 | validate_base64uri(Str) when ?IS_STRING(Str) -> 40 | catch begin 41 | [begin 42 | case lists:member(C, ?BASE64_URI_CHARS) of 43 | true -> 44 | ok; 45 | false -> 46 | throw(false) 47 | end 48 | end 49 | || C <- string:to_graphemes(Str)], 50 | string:is_empty(Str) == false 51 | end. 52 | 53 | -spec to_base64uri(string() | binary()) -> string(). 54 | to_base64uri(Str) when ?IS_STRING(Str) -> 55 | lists:foldr(fun(G, Acc) -> 56 | case lists:member(G, ?BASE64_URI_CHARS) of 57 | true -> [G | Acc]; 58 | false -> [$_ | Acc] 59 | end 60 | end, 61 | [], string:to_graphemes(Str)). 62 | 63 | -spec id(term()) -> term(). 64 | id(X) -> 65 | X. 66 | 67 | -spec lists_find(fun((term()) -> boolean()), list()) -> 68 | {ok, term()} | not_found. 69 | lists_find(_Pred, []) -> 70 | not_found; 71 | lists_find(Pred, [Item | Rem]) -> 72 | case Pred(Item) of 73 | true -> 74 | {ok, Item}; 75 | false -> 76 | lists_find(Pred, Rem) 77 | end. 78 | 79 | hostname_from_node() -> 80 | case re:split(atom_to_list(node()), "@", [{return, list}, {parts, 2}]) 81 | of 82 | [_, Hostname] -> 83 | Hostname; 84 | [_] -> 85 | {ok, H} = inet:gethostname(), 86 | H 87 | end. 88 | 89 | get_replication_configuration_from_tls_dist() -> 90 | get_replication_configuration_from_tls_dist(fun (debug, Fmt, Args) -> 91 | ?DEBUG(Fmt, Args); 92 | (warn, Fmt, Args) -> 93 | ?WARN(Fmt, Args); 94 | (warning, Fmt, Args) -> 95 | ?WARN(Fmt, Args); 96 | (_, Fmt, Args) -> 97 | ?INFO(Fmt, Args) 98 | end). 99 | 100 | get_replication_configuration_from_tls_dist(LogFun) -> 101 | get_replication_configuration_from_tls_dist(fun file:consult/1, 102 | LogFun). 103 | 104 | get_replication_configuration_from_tls_dist(FileConsultFun, LogFun) -> 105 | InitArguments = init:get_arguments(), 106 | case inet_tls_enabled(InitArguments) of 107 | true -> 108 | LogFun(debug, 109 | "Inter-node TLS enabled, " 110 | ++ "configuring stream replication over TLS", 111 | []), 112 | replication_over_tls_configuration(InitArguments, FileConsultFun, LogFun); 113 | false -> 114 | LogFun(debug, "Inter-node TLS not enabled", []), 115 | [] 116 | end. 117 | 118 | replication_over_tls_configuration(InitArgs, FileConsultFun, LogFun) -> 119 | case proplists:lookup(ssl_dist_optfile, InitArgs) of 120 | none -> 121 | LogFun(debug, 122 | "Using ssl_dist_opt to configure " 123 | ++ "stream replication over TLS", 124 | []), 125 | SslDistOpt = proplists:lookup_all(ssl_dist_opt, InitArgs), 126 | [{replication_transport, ssl}, 127 | {replication_server_ssl_options, 128 | build_replication_over_tls_options("server_", SslDistOpt, [])}, 129 | {replication_client_ssl_options, 130 | build_replication_over_tls_options("client_", SslDistOpt, [])}]; 131 | {ssl_dist_optfile, [OptFile]} -> 132 | LogFun(debug, 133 | "Using ssl_dist_optfile to configure " 134 | ++ "stream replication over TLS", 135 | []), 136 | case FileConsultFun(OptFile) of 137 | {ok, [TlsDist]} -> 138 | SslServerOptions = proplists:get_value(server, TlsDist, []), 139 | SslClientOptions = proplists:get_value(client, TlsDist, []), 140 | [{replication_transport, ssl}, 141 | {replication_server_ssl_options, SslServerOptions}, 142 | {replication_client_ssl_options, SslClientOptions}]; 143 | {error, Error} -> 144 | LogFun(warn, 145 | "Error while reading TLS " 146 | ++ "distributon option file ~ts: ~0p", 147 | [OptFile, Error]), 148 | LogFun(warn, 149 | "Stream replication over TLS will NOT be enabled", 150 | []), 151 | []; 152 | R -> 153 | LogFun(warn, 154 | "Unexpected result while reading TLS distributon " 155 | "option file ~ts: ~0p", 156 | [OptFile, R]), 157 | LogFun(warn, 158 | "Stream replication over TLS will NOT be enabled", 159 | []), 160 | [] 161 | end 162 | end. 163 | 164 | build_replication_over_tls_options(_Prefix, [], Acc) -> 165 | Acc; 166 | build_replication_over_tls_options("server_" = Prefix, 167 | [{ssl_dist_opt, ["server_" ++ Key, Value]} 168 | | Tail], 169 | Acc) -> 170 | Option = list_to_atom(Key), 171 | build_replication_over_tls_options(Prefix, Tail, 172 | Acc 173 | ++ [extract_replication_over_tls_option(Option, 174 | Value)]); 175 | build_replication_over_tls_options("client_" = Prefix, 176 | [{ssl_dist_opt, ["client_" ++ Key, Value]} 177 | | Tail], 178 | Acc) -> 179 | Option = list_to_atom(Key), 180 | build_replication_over_tls_options(Prefix, Tail, 181 | Acc 182 | ++ [extract_replication_over_tls_option(Option, 183 | Value)]); 184 | build_replication_over_tls_options(Prefix, 185 | [{ssl_dist_opt, [Key1, Value1, Key2, Value2]} 186 | | Tail], 187 | Acc) -> 188 | %% For -ssl_dist_opt server_secure_renegotiate true client_secure_renegotiate true 189 | build_replication_over_tls_options(Prefix, 190 | [{ssl_dist_opt, [Key1, Value1]}, 191 | {ssl_dist_opt, [Key2, Value2]}] 192 | ++ Tail, 193 | Acc); 194 | build_replication_over_tls_options(Prefix, 195 | [{ssl_dist_opt, [_Key, _Value]} | Tail], 196 | Acc) -> 197 | build_replication_over_tls_options(Prefix, Tail, Acc). 198 | 199 | extract_replication_over_tls_option(certfile, V) -> 200 | {certfile, V}; 201 | extract_replication_over_tls_option(keyfile, V) -> 202 | {keyfile, V}; 203 | extract_replication_over_tls_option(password, V) -> 204 | {password, V}; 205 | extract_replication_over_tls_option(cacertfile, V) -> 206 | {cacertfile, V}; 207 | extract_replication_over_tls_option(verify, V) -> 208 | {verify, list_to_atom(V)}; 209 | extract_replication_over_tls_option(verify_fun, V) -> 210 | %% Write as {Module, Function, InitialUserState} 211 | {verify_fun, eval_term(V)}; 212 | extract_replication_over_tls_option(crl_check, V) -> 213 | {crl_check, list_to_atom(V)}; 214 | extract_replication_over_tls_option(crl_cache, V) -> 215 | %% Write as Erlang term 216 | {crl_cache, eval_term(V)}; 217 | extract_replication_over_tls_option(reuse_sessions, V) -> 218 | %% boolean() | save 219 | {reuse_sessions, eval_term(V)}; 220 | extract_replication_over_tls_option(secure_renegotiate, V) -> 221 | {secure_renegotiate, list_to_atom(V)}; 222 | extract_replication_over_tls_option(depth, V) -> 223 | {depth, list_to_integer(V)}; 224 | extract_replication_over_tls_option(hibernate_after, "undefined") -> 225 | {hibernate_after, undefined}; 226 | extract_replication_over_tls_option(hibernate_after, V) -> 227 | {hibernate_after, list_to_integer(V)}; 228 | extract_replication_over_tls_option(ciphers, V) -> 229 | %% Use old string format 230 | %% e.g. TLS_AES_256_GCM_SHA384:TLS_AES_128_GCM_SHA256 231 | {ciphers, V}; 232 | extract_replication_over_tls_option(fail_if_no_peer_cert, V) -> 233 | {fail_if_no_peer_cert, list_to_atom(V)}; 234 | extract_replication_over_tls_option(dhfile, V) -> 235 | {dhfile, V}. 236 | 237 | eval_term(V) -> 238 | {ok, Tokens, _EndLine} = erl_scan:string(V ++ "."), 239 | {ok, AbsForm} = erl_parse:parse_exprs(Tokens), 240 | {value, Term, _Bs} = erl_eval:exprs(AbsForm, erl_eval:new_bindings()), 241 | Term. 242 | 243 | -spec get_replica_listener_inet_address_family() -> inet | inet6. 244 | get_replica_listener_inet_address_family() -> 245 | case application:get_env(osiris, replica_ip_address_family, inet) of 246 | inet -> inet; 247 | inet6 -> inet6; 248 | _ -> inet 249 | end. 250 | 251 | inet_tls_enabled([]) -> 252 | false; 253 | inet_tls_enabled([{proto_dist, ["inet_tls"]} | _]) -> 254 | true; 255 | inet_tls_enabled([_Opt | Tail]) -> 256 | inet_tls_enabled(Tail). 257 | 258 | partition_parallel(F, Es, Timeout) -> 259 | Parent = self(), 260 | Running = [{spawn_monitor(fun() -> Parent ! {self(), F(E)} end), E} 261 | || E <- Es], 262 | collect(Running, {[], []}, Timeout). 263 | 264 | collect([], Acc, _Timeout) -> 265 | Acc; 266 | collect([{{Pid, MRef}, E} | Next], {Left, Right}, Timeout) -> 267 | receive 268 | {Pid, true} -> 269 | erlang:demonitor(MRef, [flush]), 270 | collect(Next, {[E | Left], Right}, Timeout); 271 | {Pid, false} -> 272 | erlang:demonitor(MRef, [flush]), 273 | collect(Next, {Left, [E | Right]}, Timeout); 274 | {'DOWN', MRef, process, Pid, _Reason} -> 275 | collect(Next, {Left, [E | Right]}, Timeout) 276 | after Timeout -> 277 | exit(partition_parallel_timeout) 278 | end. 279 | 280 | normalise_name(Name) when is_binary(Name) -> 281 | Name; 282 | normalise_name(Name) when is_list(Name) -> 283 | list_to_binary(Name). 284 | 285 | get_reader_context(Pid) 286 | when is_pid(Pid) andalso node(Pid) == node() -> 287 | case ets:lookup(osiris_reader_context_cache, Pid) of 288 | [] -> 289 | {ok, Ctx0} = gen:call(Pid, '$gen_call', get_reader_context, infinity), 290 | Ctx0; 291 | [{_Pid, Dir, Name, Shared, Ref, ReadersCountersFun}] -> 292 | #{dir => Dir, 293 | name => Name, 294 | shared => Shared, 295 | reference => Ref, 296 | readers_counter_fun => ReadersCountersFun} 297 | end. 298 | 299 | cache_reader_context(Pid, Dir, Name, Shared, Ref, ReadersCounterFun) 300 | when is_pid(Pid) andalso 301 | ?IS_STRING(Dir) andalso 302 | is_function(ReadersCounterFun) -> 303 | true = ets:insert(osiris_reader_context_cache, 304 | {Pid, Dir, Name, Shared, Ref, ReadersCounterFun}), 305 | ok. 306 | 307 | is_dir(Dir) -> 308 | case prim_file:read_file_info(Dir) of 309 | {ok, #file_info{type=directory}} -> 310 | true; 311 | _ -> 312 | false 313 | end. 314 | 315 | is_file(File) -> 316 | case prim_file:read_file_info(File) of 317 | {ok, #file_info{type = directory}} -> 318 | true; 319 | {ok, #file_info{type = regular}} -> 320 | true; 321 | _ -> 322 | false 323 | end. 324 | -------------------------------------------------------------------------------- /LICENSE-MPL-RabbitMQ: -------------------------------------------------------------------------------- 1 | Mozilla Public License Version 2.0 2 | ================================== 3 | 4 | 1. Definitions 5 | -------------- 6 | 7 | 1.1. "Contributor" 8 | means each individual or legal entity that creates, contributes to 9 | the creation of, or owns Covered Software. 10 | 11 | 1.2. "Contributor Version" 12 | means the combination of the Contributions of others (if any) used 13 | by a Contributor and that particular Contributor's Contribution. 14 | 15 | 1.3. "Contribution" 16 | means Covered Software of a particular Contributor. 17 | 18 | 1.4. "Covered Software" 19 | means Source Code Form to which the initial Contributor has attached 20 | the notice in Exhibit A, the Executable Form of such Source Code 21 | Form, and Modifications of such Source Code Form, in each case 22 | including portions thereof. 23 | 24 | 1.5. "Incompatible With Secondary Licenses" 25 | means 26 | 27 | (a) that the initial Contributor has attached the notice described 28 | in Exhibit B to the Covered Software; or 29 | 30 | (b) that the Covered Software was made available under the terms of 31 | version 1.1 or earlier of the License, but not also under the 32 | terms of a Secondary License. 33 | 34 | 1.6. "Executable Form" 35 | means any form of the work other than Source Code Form. 36 | 37 | 1.7. "Larger Work" 38 | means a work that combines Covered Software with other material, in 39 | a separate file or files, that is not Covered Software. 40 | 41 | 1.8. "License" 42 | means this document. 43 | 44 | 1.9. "Licensable" 45 | means having the right to grant, to the maximum extent possible, 46 | whether at the time of the initial grant or subsequently, any and 47 | all of the rights conveyed by this License. 48 | 49 | 1.10. "Modifications" 50 | means any of the following: 51 | 52 | (a) any file in Source Code Form that results from an addition to, 53 | deletion from, or modification of the contents of Covered 54 | Software; or 55 | 56 | (b) any new file in Source Code Form that contains any Covered 57 | Software. 58 | 59 | 1.11. "Patent Claims" of a Contributor 60 | means any patent claim(s), including without limitation, method, 61 | process, and apparatus claims, in any patent Licensable by such 62 | Contributor that would be infringed, but for the grant of the 63 | License, by the making, using, selling, offering for sale, having 64 | made, import, or transfer of either its Contributions or its 65 | Contributor Version. 66 | 67 | 1.12. "Secondary License" 68 | means either the GNU General Public License, Version 2.0, the GNU 69 | Lesser General Public License, Version 2.1, the GNU Affero General 70 | Public License, Version 3.0, or any later versions of those 71 | licenses. 72 | 73 | 1.13. "Source Code Form" 74 | means the form of the work preferred for making modifications. 75 | 76 | 1.14. "You" (or "Your") 77 | means an individual or a legal entity exercising rights under this 78 | License. For legal entities, "You" includes any entity that 79 | controls, is controlled by, or is under common control with You. For 80 | purposes of this definition, "control" means (a) the power, direct 81 | or indirect, to cause the direction or management of such entity, 82 | whether by contract or otherwise, or (b) ownership of more than 83 | fifty percent (50%) of the outstanding shares or beneficial 84 | ownership of such entity. 85 | 86 | 2. License Grants and Conditions 87 | -------------------------------- 88 | 89 | 2.1. Grants 90 | 91 | Each Contributor hereby grants You a world-wide, royalty-free, 92 | non-exclusive license: 93 | 94 | (a) under intellectual property rights (other than patent or trademark) 95 | Licensable by such Contributor to use, reproduce, make available, 96 | modify, display, perform, distribute, and otherwise exploit its 97 | Contributions, either on an unmodified basis, with Modifications, or 98 | as part of a Larger Work; and 99 | 100 | (b) under Patent Claims of such Contributor to make, use, sell, offer 101 | for sale, have made, import, and otherwise transfer either its 102 | Contributions or its Contributor Version. 103 | 104 | 2.2. Effective Date 105 | 106 | The licenses granted in Section 2.1 with respect to any Contribution 107 | become effective for each Contribution on the date the Contributor first 108 | distributes such Contribution. 109 | 110 | 2.3. Limitations on Grant Scope 111 | 112 | The licenses granted in this Section 2 are the only rights granted under 113 | this License. No additional rights or licenses will be implied from the 114 | distribution or licensing of Covered Software under this License. 115 | Notwithstanding Section 2.1(b) above, no patent license is granted by a 116 | Contributor: 117 | 118 | (a) for any code that a Contributor has removed from Covered Software; 119 | or 120 | 121 | (b) for infringements caused by: (i) Your and any other third party's 122 | modifications of Covered Software, or (ii) the combination of its 123 | Contributions with other software (except as part of its Contributor 124 | Version); or 125 | 126 | (c) under Patent Claims infringed by Covered Software in the absence of 127 | its Contributions. 128 | 129 | This License does not grant any rights in the trademarks, service marks, 130 | or logos of any Contributor (except as may be necessary to comply with 131 | the notice requirements in Section 3.4). 132 | 133 | 2.4. Subsequent Licenses 134 | 135 | No Contributor makes additional grants as a result of Your choice to 136 | distribute the Covered Software under a subsequent version of this 137 | License (see Section 10.2) or under the terms of a Secondary License (if 138 | permitted under the terms of Section 3.3). 139 | 140 | 2.5. Representation 141 | 142 | Each Contributor represents that the Contributor believes its 143 | Contributions are its original creation(s) or it has sufficient rights 144 | to grant the rights to its Contributions conveyed by this License. 145 | 146 | 2.6. Fair Use 147 | 148 | This License is not intended to limit any rights You have under 149 | applicable copyright doctrines of fair use, fair dealing, or other 150 | equivalents. 151 | 152 | 2.7. Conditions 153 | 154 | Sections 3.1, 3.2, 3.3, and 3.4 are conditions of the licenses granted 155 | in Section 2.1. 156 | 157 | 3. Responsibilities 158 | ------------------- 159 | 160 | 3.1. Distribution of Source Form 161 | 162 | All distribution of Covered Software in Source Code Form, including any 163 | Modifications that You create or to which You contribute, must be under 164 | the terms of this License. You must inform recipients that the Source 165 | Code Form of the Covered Software is governed by the terms of this 166 | License, and how they can obtain a copy of this License. You may not 167 | attempt to alter or restrict the recipients' rights in the Source Code 168 | Form. 169 | 170 | 3.2. Distribution of Executable Form 171 | 172 | If You distribute Covered Software in Executable Form then: 173 | 174 | (a) such Covered Software must also be made available in Source Code 175 | Form, as described in Section 3.1, and You must inform recipients of 176 | the Executable Form how they can obtain a copy of such Source Code 177 | Form by reasonable means in a timely manner, at a charge no more 178 | than the cost of distribution to the recipient; and 179 | 180 | (b) You may distribute such Executable Form under the terms of this 181 | License, or sublicense it under different terms, provided that the 182 | license for the Executable Form does not attempt to limit or alter 183 | the recipients' rights in the Source Code Form under this License. 184 | 185 | 3.3. Distribution of a Larger Work 186 | 187 | You may create and distribute a Larger Work under terms of Your choice, 188 | provided that You also comply with the requirements of this License for 189 | the Covered Software. If the Larger Work is a combination of Covered 190 | Software with a work governed by one or more Secondary Licenses, and the 191 | Covered Software is not Incompatible With Secondary Licenses, this 192 | License permits You to additionally distribute such Covered Software 193 | under the terms of such Secondary License(s), so that the recipient of 194 | the Larger Work may, at their option, further distribute the Covered 195 | Software under the terms of either this License or such Secondary 196 | License(s). 197 | 198 | 3.4. Notices 199 | 200 | You may not remove or alter the substance of any license notices 201 | (including copyright notices, patent notices, disclaimers of warranty, 202 | or limitations of liability) contained within the Source Code Form of 203 | the Covered Software, except that You may alter any license notices to 204 | the extent required to remedy known factual inaccuracies. 205 | 206 | 3.5. Application of Additional Terms 207 | 208 | You may choose to offer, and to charge a fee for, warranty, support, 209 | indemnity or liability obligations to one or more recipients of Covered 210 | Software. However, You may do so only on Your own behalf, and not on 211 | behalf of any Contributor. You must make it absolutely clear that any 212 | such warranty, support, indemnity, or liability obligation is offered by 213 | You alone, and You hereby agree to indemnify every Contributor for any 214 | liability incurred by such Contributor as a result of warranty, support, 215 | indemnity or liability terms You offer. You may include additional 216 | disclaimers of warranty and limitations of liability specific to any 217 | jurisdiction. 218 | 219 | 4. Inability to Comply Due to Statute or Regulation 220 | --------------------------------------------------- 221 | 222 | If it is impossible for You to comply with any of the terms of this 223 | License with respect to some or all of the Covered Software due to 224 | statute, judicial order, or regulation then You must: (a) comply with 225 | the terms of this License to the maximum extent possible; and (b) 226 | describe the limitations and the code they affect. Such description must 227 | be placed in a text file included with all distributions of the Covered 228 | Software under this License. Except to the extent prohibited by statute 229 | or regulation, such description must be sufficiently detailed for a 230 | recipient of ordinary skill to be able to understand it. 231 | 232 | 5. Termination 233 | -------------- 234 | 235 | 5.1. The rights granted under this License will terminate automatically 236 | if You fail to comply with any of its terms. However, if You become 237 | compliant, then the rights granted under this License from a particular 238 | Contributor are reinstated (a) provisionally, unless and until such 239 | Contributor explicitly and finally terminates Your grants, and (b) on an 240 | ongoing basis, if such Contributor fails to notify You of the 241 | non-compliance by some reasonable means prior to 60 days after You have 242 | come back into compliance. Moreover, Your grants from a particular 243 | Contributor are reinstated on an ongoing basis if such Contributor 244 | notifies You of the non-compliance by some reasonable means, this is the 245 | first time You have received notice of non-compliance with this License 246 | from such Contributor, and You become compliant prior to 30 days after 247 | Your receipt of the notice. 248 | 249 | 5.2. If You initiate litigation against any entity by asserting a patent 250 | infringement claim (excluding declaratory judgment actions, 251 | counter-claims, and cross-claims) alleging that a Contributor Version 252 | directly or indirectly infringes any patent, then the rights granted to 253 | You by any and all Contributors for the Covered Software under Section 254 | 2.1 of this License shall terminate. 255 | 256 | 5.3. In the event of termination under Sections 5.1 or 5.2 above, all 257 | end user license agreements (excluding distributors and resellers) which 258 | have been validly granted by You or Your distributors under this License 259 | prior to termination shall survive termination. 260 | 261 | ************************************************************************ 262 | * * 263 | * 6. Disclaimer of Warranty * 264 | * ------------------------- * 265 | * * 266 | * Covered Software is provided under this License on an "as is" * 267 | * basis, without warranty of any kind, either expressed, implied, or * 268 | * statutory, including, without limitation, warranties that the * 269 | * Covered Software is free of defects, merchantable, fit for a * 270 | * particular purpose or non-infringing. The entire risk as to the * 271 | * quality and performance of the Covered Software is with You. * 272 | * Should any Covered Software prove defective in any respect, You * 273 | * (not any Contributor) assume the cost of any necessary servicing, * 274 | * repair, or correction. This disclaimer of warranty constitutes an * 275 | * essential part of this License. No use of any Covered Software is * 276 | * authorized under this License except under this disclaimer. * 277 | * * 278 | ************************************************************************ 279 | 280 | ************************************************************************ 281 | * * 282 | * 7. Limitation of Liability * 283 | * -------------------------- * 284 | * * 285 | * Under no circumstances and under no legal theory, whether tort * 286 | * (including negligence), contract, or otherwise, shall any * 287 | * Contributor, or anyone who distributes Covered Software as * 288 | * permitted above, be liable to You for any direct, indirect, * 289 | * special, incidental, or consequential damages of any character * 290 | * including, without limitation, damages for lost profits, loss of * 291 | * goodwill, work stoppage, computer failure or malfunction, or any * 292 | * and all other commercial damages or losses, even if such party * 293 | * shall have been informed of the possibility of such damages. This * 294 | * limitation of liability shall not apply to liability for death or * 295 | * personal injury resulting from such party's negligence to the * 296 | * extent applicable law prohibits such limitation. Some * 297 | * jurisdictions do not allow the exclusion or limitation of * 298 | * incidental or consequential damages, so this exclusion and * 299 | * limitation may not apply to You. * 300 | * * 301 | ************************************************************************ 302 | 303 | 8. Litigation 304 | ------------- 305 | 306 | Any litigation relating to this License may be brought only in the 307 | courts of a jurisdiction where the defendant maintains its principal 308 | place of business and such litigation shall be governed by laws of that 309 | jurisdiction, without reference to its conflict-of-law provisions. 310 | Nothing in this Section shall prevent a party's ability to bring 311 | cross-claims or counter-claims. 312 | 313 | 9. Miscellaneous 314 | ---------------- 315 | 316 | This License represents the complete agreement concerning the subject 317 | matter hereof. If any provision of this License is held to be 318 | unenforceable, such provision shall be reformed only to the extent 319 | necessary to make it enforceable. Any law or regulation which provides 320 | that the language of a contract shall be construed against the drafter 321 | shall not be used to construe this License against a Contributor. 322 | 323 | 10. Versions of the License 324 | --------------------------- 325 | 326 | 10.1. New Versions 327 | 328 | Mozilla Foundation is the license steward. Except as provided in Section 329 | 10.3, no one other than the license steward has the right to modify or 330 | publish new versions of this License. Each version will be given a 331 | distinguishing version number. 332 | 333 | 10.2. Effect of New Versions 334 | 335 | You may distribute the Covered Software under the terms of the version 336 | of the License under which You originally received the Covered Software, 337 | or under the terms of any subsequent version published by the license 338 | steward. 339 | 340 | 10.3. Modified Versions 341 | 342 | If you create software not governed by this License, and you want to 343 | create a new license for such software, you may create and use a 344 | modified version of this License if you rename the license and remove 345 | any references to the name of the license steward (except to note that 346 | such modified license differs from this License). 347 | 348 | 10.4. Distributing Source Code Form that is Incompatible With Secondary 349 | Licenses 350 | 351 | If You choose to distribute Source Code Form that is Incompatible With 352 | Secondary Licenses under the terms of this version of the License, the 353 | notice described in Exhibit B of this License must be attached. 354 | 355 | Exhibit A - Source Code Form License Notice 356 | ------------------------------------------- 357 | 358 | This Source Code Form is subject to the terms of the Mozilla Public 359 | License, v. 2.0. If a copy of the MPL was not distributed with this 360 | file, You can obtain one at http://mozilla.org/MPL/2.0/. 361 | 362 | If it is not possible or desirable to put the notice in a particular 363 | file, then You may include the notice in a location (such as a LICENSE 364 | file in a relevant directory) where a recipient would be likely to look 365 | for such a notice. 366 | 367 | You may add additional accurate notices of copyright ownership. 368 | 369 | Exhibit B - "Incompatible With Secondary Licenses" Notice 370 | --------------------------------------------------------- 371 | 372 | This Source Code Form is "Incompatible With Secondary Licenses", as 373 | defined by the Mozilla Public License, v. 2.0. 374 | -------------------------------------------------------------------------------- /src/osiris_replica_reader.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_replica_reader). 9 | 10 | -behaviour(gen_server). 11 | 12 | -include("osiris.hrl"). 13 | 14 | -define(DEF_SND_BUF, 146988 * 10). 15 | %% replica reader, spawned remotely by replica process, connects back to 16 | %% configured host/port, reads entries from master and uses file:sendfile to 17 | %% replicate read records 18 | 19 | %% API functions 20 | -export([start_link/1, 21 | start/2, 22 | stop/1]). 23 | %% gen_server callbacks 24 | -export([init/1, 25 | handle_call/3, 26 | handle_cast/2, 27 | handle_info/2, 28 | terminate/2, 29 | code_change/3]). 30 | -export([formatter/1]). 31 | 32 | -record(state, 33 | {log :: osiris_log:state(), 34 | name :: osiris:name(), 35 | transport :: osiris_log:transport(), 36 | socket :: gen_tcp:socket() | ssl:sslsocket(), 37 | replica_pid :: pid(), 38 | leader_pid :: pid(), 39 | leader_monitor_ref :: reference(), 40 | counter :: counters:counters_ref(), 41 | counter_id :: term(), 42 | committed_offset = -1 :: -1 | osiris:offset(), 43 | offset_listener :: undefined | osiris:offset()}). 44 | 45 | -define(C_OFFSET_LISTENERS, ?C_NUM_LOG_FIELDS + 1). 46 | -define(COUNTER_FIELDS, 47 | [ 48 | {offset_listeners, ?C_OFFSET_LISTENERS, counter, "Number of offset listeners"} 49 | ] 50 | ). 51 | 52 | %%%=================================================================== 53 | %%% API functions 54 | %%%=================================================================== 55 | 56 | %%-------------------------------------------------------------------- 57 | %% @doc 58 | %% Starts the server 59 | %% 60 | %% @spec start_link() -> {ok, Pid} | ignore | {error, Error} 61 | %% @end 62 | %%-------------------------------------------------------------------- 63 | start_link(Conf) -> 64 | gen_server:start_link(?MODULE, Conf, []). 65 | 66 | stop(Pid) -> 67 | gen_server:cast(Pid, stop). 68 | 69 | start(Node, ReplicaReaderConf) when is_map(ReplicaReaderConf) -> 70 | try 71 | supervisor:start_child({osiris_replica_reader_sup, Node}, 72 | #{id => make_ref(), 73 | start => 74 | {osiris_replica_reader, start_link, 75 | [ReplicaReaderConf]}, 76 | %% replica readers should never be 77 | %% restarted by their sups 78 | %% instead they need to be re-started 79 | %% by their replica 80 | restart => temporary, 81 | shutdown => 5000, 82 | type => worker, 83 | modules => [osiris_replica_reader]}) 84 | catch 85 | exit:{{nodedown, _} = Res, _Stack} -> 86 | {error, Res}; 87 | exit:{noproc = Res, _Stack} -> 88 | {error, Res} 89 | end. 90 | 91 | %%%=================================================================== 92 | %%% gen_server callbacks 93 | %%%=================================================================== 94 | 95 | %%-------------------------------------------------------------------- 96 | %% @private 97 | %% @doc 98 | %% Initializes the server 99 | %% 100 | %% @spec init(Args) -> {ok, State} | 101 | %% {ok, State, Timeout} | 102 | %% ignore | 103 | %% {stop, Reason} 104 | %% @end 105 | %%-------------------------------------------------------------------- 106 | 107 | init(#{hosts := Hosts, 108 | port := Port, 109 | transport := Transport, 110 | name := Name, 111 | replica_pid := ReplicaPid, 112 | leader_pid := LeaderPid, 113 | start_offset := {StartOffset, _} = TailInfo, 114 | reference := ExtRef, 115 | connection_token := Token}) -> 116 | process_flag(trap_exit, true), 117 | 118 | ?DEBUG("~ts: trying to connect to replica at ~0p", [Name, Hosts]), 119 | 120 | case maybe_connect(Name, Transport, Hosts, Port, connect_options()) of 121 | {ok, Sock, Host} -> 122 | ?INFO_(Name, "replica reader successfully connected to host ~0p port ~b", 123 | [Host, Port]), 124 | CntId = {?MODULE, ExtRef, Host, Port}, 125 | CntSpec = {CntId, ?COUNTER_FIELDS}, 126 | Config = #{counter_spec => CntSpec, transport => Transport}, 127 | %% send token to replica to complete connection setup 128 | ok = send(Transport, Sock, Token), 129 | Ret = osiris_writer:init_data_reader(LeaderPid, TailInfo, Config), 130 | case Ret of 131 | {ok, Log} -> 132 | CntRef = osiris_log:counters_ref(Log), 133 | ?INFO_(Name, "starting osiris replica reader at offset ~b", 134 | [osiris_log:next_offset(Log)]), 135 | 136 | %% register data listener with osiris_proc 137 | ok = osiris_writer:register_data_listener(LeaderPid, StartOffset), 138 | MRef = monitor(process, LeaderPid), 139 | State = 140 | maybe_register_offset_listener( 141 | maybe_send_committed_offset(#state{log = Log, 142 | name = Name, 143 | transport = Transport, 144 | socket = Sock, 145 | replica_pid = ReplicaPid, 146 | leader_pid = LeaderPid, 147 | leader_monitor_ref = MRef, 148 | counter = CntRef, 149 | counter_id = CntId})), 150 | {ok, State}; 151 | {error, no_process} -> 152 | ?WARN_(Name, 153 | "osiris writer for ~0p is down, " 154 | "replica reader will not start", 155 | [ExtRef]), 156 | {stop, normal}; 157 | {error, enoent} -> 158 | ?WARN_(Name, 159 | "data reader for ~0p encountered an 'enonet' error whilst 160 | initialising, replica reader will not start", 161 | [ExtRef]), 162 | {stop, normal}; 163 | {error, {offset_out_of_range, Range}} -> 164 | ?WARN_(Name, 165 | "data reader requested an offset (~b) that was out 166 | of range: ~0p, replica reader will not start", 167 | [StartOffset, Range]), 168 | {stop, normal}; 169 | {error, {invalid_last_offset_epoch, Epoch, Offset}} -> 170 | ?WARN_(Name, 171 | "data reader found an invalid last offset epoch: 172 | epoch ~0p offset ~0p, replica reader will not start", 173 | [Epoch, Offset]), 174 | {stop, normal} 175 | end; 176 | {error, Reason} -> 177 | ?WARN_(Name, "could not connect replica reader to replica at ~0p port ~b, Reason: ~0p", 178 | [Hosts, Port, Reason]), 179 | {error, Reason} 180 | end. 181 | 182 | %%-------------------------------------------------------------------- 183 | %% @private 184 | %% @doc 185 | %% Handling call messages 186 | %% 187 | %% @spec handle_call(Request, From, State) -> 188 | %% {reply, Reply, State} | 189 | %% {reply, Reply, State, Timeout} | 190 | %% {noreply, State} | 191 | %% {noreply, State, Timeout} | 192 | %% {stop, Reason, Reply, State} | 193 | %% {stop, Reason, State} 194 | %% @end 195 | %%-------------------------------------------------------------------- 196 | handle_call(_Request, _From, State) -> 197 | Reply = ok, 198 | {reply, Reply, State}. 199 | 200 | %%-------------------------------------------------------------------- 201 | %% @private 202 | %% @doc 203 | %% Handling cast messages 204 | %% 205 | %% @spec handle_cast(Msg, State) -> {noreply, State} | 206 | %% {noreply, State, Timeout} | 207 | %% {stop, Reason, State} 208 | %% @end 209 | %%-------------------------------------------------------------------- 210 | handle_cast({more_data, _LastOffset}, 211 | #state{leader_pid = LeaderPid} = State0) -> 212 | #state{log = Log} = State = do_sendfile(State0), 213 | NextOffs = osiris_log:next_offset(Log), 214 | ok = osiris_writer:register_data_listener(LeaderPid, NextOffs), 215 | {noreply, maybe_register_offset_listener(State)}; 216 | handle_cast(stop, State) -> 217 | {stop, normal, State}. 218 | 219 | maybe_register_offset_listener(#state{leader_pid = LeaderPid, 220 | committed_offset = COffs, 221 | counter = Cnt, 222 | offset_listener = undefined} = 223 | State) -> 224 | ok = counters:add(Cnt, ?C_OFFSET_LISTENERS, 1), 225 | ok = 226 | osiris:register_offset_listener(LeaderPid, COffs + 1, 227 | {?MODULE, formatter, []}), 228 | State#state{offset_listener = COffs + 1}; 229 | maybe_register_offset_listener(State) -> 230 | State. 231 | 232 | %%-------------------------------------------------------------------- 233 | %% @private 234 | %% @doc 235 | %% Handling all non call/cast messages 236 | %% 237 | %% @spec handle_info(Info, State) -> {noreply, State} | 238 | %% {noreply, State, Timeout} | 239 | %% {stop, Reason, State} 240 | %% @end 241 | %%-------------------------------------------------------------------- 242 | handle_info({osiris_offset, _, _Offs}, State0) -> 243 | State1 = maybe_send_committed_offset(State0), 244 | State = 245 | maybe_register_offset_listener(State1#state{offset_listener = 246 | undefined}), 247 | {noreply, State}; 248 | handle_info({'DOWN', Ref, _, _, Info}, 249 | #state{name = Name, 250 | transport = Transport, 251 | socket = Sock, 252 | leader_monitor_ref = Ref} = 253 | State) -> 254 | %% leader is down, exit 255 | ?ERROR_(Name, "detected leader down with ~W - exiting...", 256 | [Info, 10]), 257 | %% this should be enough to make the replica shut down 258 | ok = close(Transport, Sock), 259 | %% If the reason is `noproc`, it means the leader is already gone at the 260 | %% time `init/1` was called. Therefore the set of processes is being shut 261 | %% down concurrently. We can exit with the `normal` reason in this case. 262 | Reason = case Info of 263 | noproc -> normal; 264 | _ -> Info 265 | end, 266 | {stop, Reason, State}; 267 | handle_info({tcp_closed, Socket}, 268 | #state{name = Name, socket = Socket} = State) -> 269 | ?DEBUG_(Name, "Socket closed. Exiting...", []), 270 | {stop, normal, State}; 271 | handle_info({ssl_closed, Socket}, 272 | #state{name = Name, socket = Socket} = State) -> 273 | ?DEBUG_(Name, "TLS socket closed. Exiting...", []), 274 | {stop, normal, State}; 275 | handle_info({tcp_error, Socket, Error}, 276 | #state{name = Name, socket = Socket} = State) -> 277 | ?DEBUG_(Name, "Socket error ~0p. " 278 | "Exiting...", [Error]), 279 | {stop, {tcp_error, Error}, State}; 280 | handle_info({ssl_error, Socket, Error}, 281 | #state{name = Name, socket = Socket} = State) -> 282 | ?DEBUG_(Name, "TLS socket error ~0p. " 283 | "Exiting...", [Error]), 284 | {stop, {ssl_error, Error}, State}; 285 | handle_info({'EXIT', Ref, Info}, #state{name = Name} = State) -> 286 | ?DEBUG_(Name, "EXIT received " 287 | "~w, Info: ~w", 288 | [Ref, Info]), 289 | {stop, normal, State}; 290 | handle_info(Info, #state{name = Name} = State) -> 291 | ?DEBUG_(Name, "'~ts' unhandled message ~W", 292 | [Info, 10]), 293 | {noreply, State}. 294 | 295 | %%-------------------------------------------------------------------- 296 | %% @private 297 | %% @doc 298 | %% This function is called by a gen_server when it is about to 299 | %% terminate. It should be the opposite of Module:init/1 and do any 300 | %% necessary cleaning up. When it returns, the gen_server terminates 301 | %% with Reason. The return value is ignored. 302 | %% 303 | %% @spec terminate(Reason, State) -> void() 304 | %% @end 305 | %%-------------------------------------------------------------------- 306 | terminate(_Reason, #state{log = Log}) -> 307 | ok = osiris_log:close(Log), 308 | ok. 309 | 310 | %%-------------------------------------------------------------------- 311 | %% @private 312 | %% @doc 313 | %% Convert process state when code is changed 314 | %% 315 | %% @spec code_change(OldVsn, State, Extra) -> {ok, NewState} 316 | %% @end 317 | %%-------------------------------------------------------------------- 318 | code_change(_OldVsn, State, _Extra) -> 319 | {ok, State}. 320 | 321 | %%%=================================================================== 322 | %%% Internal functions 323 | %%%=================================================================== 324 | 325 | do_sendfile(#state{socket = Sock, 326 | transport = Transport} = State) -> 327 | ok = setopts(Transport, Sock, [{nopush, true}]), 328 | do_sendfile0(State). 329 | 330 | do_sendfile0(#state{name = Name, 331 | socket = Sock, 332 | transport = Transport, 333 | log = Log0} = State0) -> 334 | State = maybe_send_committed_offset(State0), 335 | case osiris_log:send_file(Sock, Log0) of 336 | {ok, Log} -> 337 | do_sendfile0(State#state{log = Log}); 338 | {error, _Err} -> 339 | %% ignore return value here as we've already hit an error 340 | %% and it is likely we'll get another one when setting opts 341 | _ = setopts(Transport, Sock, [{nopush, false}]), 342 | ?DEBUG_(Name, "sendfile err ~w", [_Err]), 343 | State; 344 | {end_of_stream, Log} -> 345 | ok = setopts(Transport, Sock, [{nopush, false}]), 346 | State#state{log = Log} 347 | end. 348 | 349 | maybe_send_committed_offset(#state{log = Log, 350 | committed_offset = Last, 351 | replica_pid = RPid} = 352 | State) -> 353 | COffs = osiris_log:committed_offset(Log), 354 | case COffs of 355 | COffs when COffs > Last -> 356 | ok = 357 | erlang:send(RPid, {'$gen_cast', {committed_offset, COffs}}, 358 | [noconnect, nosuspend]), 359 | State#state{committed_offset = COffs}; 360 | _ -> 361 | State 362 | end. 363 | 364 | formatter(Evt) -> 365 | Evt. 366 | 367 | send(tcp, Socket, Data) -> 368 | gen_tcp:send(Socket, Data); 369 | send(ssl, Socket, Data) -> 370 | ssl:send(Socket, Data). 371 | 372 | close(tcp, Socket) -> 373 | gen_tcp:close(Socket); 374 | close(ssl, Socket) -> 375 | ssl:close(Socket). 376 | 377 | connect_options() -> 378 | SndBuf = application:get_env(osiris, replica_sndbuf, ?DEF_SND_BUF), 379 | KeepAlive = application:get_env(osiris, replica_keepalive, false), 380 | IPAddrFamily = osiris_util:get_replica_listener_inet_address_family(), 381 | [binary, 382 | IPAddrFamily, 383 | {packet, 0}, 384 | {nodelay, true}, 385 | {sndbuf, SndBuf}, 386 | {keepalive, KeepAlive}]. 387 | 388 | setopts(tcp, Sock, Opts) -> 389 | inet:setopts(Sock, Opts); 390 | setopts(ssl, Sock, Opts) -> 391 | ssl:setopts(Sock, Opts). 392 | 393 | 394 | maybe_connect(Name, T, Hosts, Port, Options) -> 395 | maybe_connect(Name, T, Hosts, Port, Options, []). 396 | 397 | maybe_connect(Name, _, [], Port, _Options, Acc) -> 398 | ?INFO_(Name, "could not connect replica reader to replica on port ~b, " 399 | "reason(s): ~0p", [Port, Acc]), 400 | {error, connection_refused}; 401 | maybe_connect(Name, tcp, [H | T], Port, Options, Acc) -> 402 | ?DEBUG_(Name, "trying to connect to ~0p on port ~b", [H, Port]), 403 | case gen_tcp:connect(H, Port, Options) of 404 | {ok, Sock} -> 405 | {ok, Sock, H}; 406 | {error, Reason} -> 407 | ?DEBUG_(Name, "connection refused, reason: ~w host:~0p - port: ~0p", 408 | [Reason, H, Port]), 409 | maybe_connect(Name, tcp, T, Port, Options, [{H, Reason} | Acc]) 410 | end; 411 | maybe_connect(Name, ssl, [H | T], Port, Options, Acc) -> 412 | ?DEBUG_(Name, "trying to establish TLS connection to ~0p using port ~b", [H, Port]), 413 | Opts = Options ++ 414 | application:get_env(osiris, replication_client_ssl_options, []) ++ 415 | maybe_add_sni_option(H), 416 | case ssl:connect(H, Port, Opts) of 417 | {ok, Sock} -> 418 | {ok, Sock, H}; 419 | {error, {tls_alert, {handshake_failure = R, _}}} -> 420 | ?DEBUG_(Name, "TLS connection refused (handshake failure), host:~0p - port: ~0p", 421 | [H, Port]), 422 | maybe_connect(Name, ssl, T, Port, Options, [{H, R} | Acc]); 423 | {error, E} -> 424 | ?DEBUG_(Name, "TLS connection refused, host:~0p - port: ~0p", [H, Port]), 425 | ?DEBUG_(Name, "error while trying to establish TLS connection ~0p", [E]), 426 | maybe_connect(Name, ssl, T, Port, Options, [{H, E} | Acc]) 427 | end. 428 | 429 | maybe_add_sni_option(H) when is_binary(H) -> 430 | [{server_name_indication, binary_to_list(H)}]; 431 | maybe_add_sni_option(H) when is_list(H) -> 432 | [{server_name_indication, H}]; 433 | maybe_add_sni_option(_) -> 434 | []. 435 | 436 | -------------------------------------------------------------------------------- /src/osiris_writer.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | %% @hidden 9 | -module(osiris_writer). 10 | 11 | -behaviour(gen_batch_server). 12 | -behaviour(osiris_member). 13 | 14 | -include("osiris.hrl"). 15 | 16 | -export([start_link/1, 17 | start/1, 18 | overview/1, 19 | init_data_reader/3, 20 | register_data_listener/2, 21 | ack/2, 22 | write/2, 23 | write/5, 24 | write_tracking/3, 25 | read_tracking/3, 26 | read_tracking/1, 27 | query_writers/2, 28 | query_replication_state/1, 29 | init/1, 30 | handle_continue/2, 31 | handle_batch/2, 32 | terminate/2, 33 | format_status/1, 34 | stop/1, 35 | delete/1]). 36 | 37 | %% osiris_member impl 38 | -export([start/2, 39 | stop/2, 40 | delete/2]). 41 | 42 | -define(C_COMMITTED_OFFSET, ?C_NUM_LOG_FIELDS + 1). 43 | -define(C_READERS, ?C_NUM_LOG_FIELDS + 2). 44 | -define(C_EPOCH, ?C_NUM_LOG_FIELDS + 3). 45 | -define(ADD_COUNTER_FIELDS, 46 | [ 47 | {committed_offset, ?C_COMMITTED_OFFSET, counter, "Last committed offset"}, 48 | {readers, ?C_READERS, counter, "Number of readers"}, 49 | {epoch, ?C_EPOCH, counter, "Current epoch"} 50 | ] 51 | ). 52 | 53 | %% primary osiris process 54 | %% batch writes incoming data 55 | %% notifies replicator and reader processes of the new max index 56 | %% manages incoming max index 57 | 58 | -record(cfg, 59 | {name :: osiris:name(), 60 | reference :: term(), 61 | replicas = [] :: [node()], 62 | directory :: file:filename_all(), 63 | counter :: counters:counters_ref(), 64 | event_formatter :: undefined | mfa()}). 65 | -record(?MODULE, 66 | {cfg :: #cfg{}, 67 | log :: osiris_log:state(), 68 | tracking :: osiris_tracking:state(), 69 | replica_state = #{} :: #{node() => {osiris:offset(), osiris:timestamp()}}, 70 | pending_corrs = queue:new() :: queue:queue(), 71 | duplicates = [] :: 72 | [{osiris:offset(), pid(), osiris:writer_id(), non_neg_integer()}], 73 | data_listeners = [] :: [{pid(), osiris:offset()}], 74 | offset_listeners = [] :: 75 | [{pid(), osiris:offset(), mfa() | undefined}], 76 | committed_offset = -1 :: osiris:offset()}). 77 | 78 | -opaque state() :: #?MODULE{}. 79 | 80 | -export_type([state/0]). 81 | 82 | -spec start(node(), Config :: osiris:config()) -> 83 | supervisor:startchild_ret(). 84 | start(Node, #{name := Name, leader_node := Node} = Config) -> 85 | supervisor:start_child({?SUP, Node}, 86 | #{id => osiris_util:normalise_name(Name), 87 | start => {?MODULE, start_link, [Config]}, 88 | restart => temporary, 89 | shutdown => 5000, 90 | type => worker}). 91 | 92 | -spec stop(node(), osiris:config()) -> 93 | ok | {error, not_found}. 94 | stop(Node, #{leader_node := Node} = Config) -> 95 | ?SUP:stop_child(Node, Config). 96 | 97 | -spec delete(node(), osiris:config()) -> 98 | ok | {error, term()}. 99 | delete(Node, #{leader_node := Node} = Config) -> 100 | ?SUP:delete_child(Node, Config). 101 | 102 | %% backwards compat 103 | start(#{leader_node := LeaderNode} = Config) -> 104 | start(LeaderNode, Config). 105 | %% backwards compat 106 | stop(#{leader_node := Node} = Config) -> 107 | stop(Node, Config). 108 | %% backwards compat 109 | delete(#{leader_node := Node} = Config) -> 110 | ?SUP:delete_child(Node, Config). 111 | 112 | -spec start_link(Config :: map()) -> 113 | {ok, pid()} | {error, {already_started, pid()}}. 114 | start_link(Config) -> 115 | Mod = ?MODULE, 116 | Opts = [{reversed_batch, true}], 117 | gen_batch_server:start_link(undefined, Mod, Config, Opts). 118 | 119 | overview(Pid) when node(Pid) == node() -> 120 | case erlang:is_process_alive(Pid) of 121 | true -> 122 | #{dir := Dir} = osiris_util:get_reader_context(Pid), 123 | {ok, osiris_log:overview(Dir)}; 124 | false -> 125 | {error, no_process} 126 | end. 127 | 128 | init_data_reader(Pid, TailInfo, Config) 129 | when node(Pid) == node() -> 130 | case erlang:is_process_alive(Pid) of 131 | true -> 132 | Ctx0 = osiris_util:get_reader_context(Pid), 133 | Ctx = maps:merge(Ctx0, Config), 134 | osiris_log:init_data_reader(TailInfo, Ctx); 135 | false -> 136 | {error, no_process} 137 | end. 138 | 139 | register_data_listener(Pid, Offset) -> 140 | ok = 141 | gen_batch_server:cast(Pid, {register_data_listener, self(), Offset}). 142 | 143 | -spec ack(identifier(), {osiris:offset(), osiris:timestamp()}) -> ok. 144 | ack(LeaderPid, {Offset, _} = OffsetTs) 145 | when is_integer(Offset) andalso Offset >= 0 -> 146 | gen_batch_server:cast(LeaderPid, {ack, node(), OffsetTs}). 147 | 148 | -spec write(Pid :: pid(), Data :: osiris:data()) -> ok. 149 | write(Pid, Data) 150 | when is_pid(Pid) -> 151 | gen_batch_server:cast(Pid, {write, Data}). 152 | 153 | -spec write(Pid :: pid(), 154 | Sender :: pid(), 155 | WriterId :: binary() | undefined, 156 | CorrOrSeq :: non_neg_integer() | term(), 157 | Data :: osiris:data()) -> ok. 158 | write(Pid, Sender, WriterId, Corr, Data) 159 | when is_pid(Pid) andalso is_pid(Sender) -> 160 | gen_batch_server:cast(Pid, {write, Sender, WriterId, Corr, Data}). 161 | 162 | -spec write_tracking(pid(), binary(), {offset | timestamp, osiris:offset() | osiris:timestamp()}) -> ok. 163 | write_tracking(Pid, TrackingId, {TrackingType, TrackingData}) 164 | when is_pid(Pid) 165 | andalso is_binary(TrackingId) 166 | andalso byte_size(TrackingId) =< 255 167 | andalso is_integer(TrackingData) -> 168 | gen_batch_server:cast(Pid, {write_tracking, TrackingId, TrackingType, TrackingData}). 169 | 170 | read_tracking(Pid, TrackingType, TrackingId) -> 171 | gen_batch_server:call(Pid, {read_tracking, TrackingType, TrackingId}). 172 | 173 | read_tracking(Pid) -> 174 | gen_batch_server:call(Pid, read_tracking). 175 | 176 | query_writers(Pid, QueryFun) -> 177 | gen_batch_server:call(Pid, {query_writers, QueryFun}). 178 | 179 | query_replication_state(Pid) when is_pid(Pid) -> 180 | gen_batch_server:call(Pid, query_replication_state). 181 | 182 | -spec init(osiris:config()) -> 183 | {ok, undefined, {continue, osiris:config()}}. 184 | init(#{name := Name0, 185 | reference := ExtRef} = Config0) -> 186 | %% augment config 187 | Name = osiris_util:normalise_name(Name0), 188 | Shared = osiris_log_shared:new(), 189 | Dir = osiris_log:directory(Config0), 190 | CntName = {?MODULE, ExtRef}, 191 | Config = Config0#{name => Name, 192 | dir => Dir, 193 | shared => Shared, 194 | counter_spec => {CntName, ?ADD_COUNTER_FIELDS}}, 195 | CntRef = osiris_log:make_counter(Config), 196 | {ok, undefined, {continue, Config#{counter => CntRef}}}. 197 | 198 | handle_continue(#{name := Name, 199 | dir := Dir, 200 | epoch := Epoch, 201 | reference := ExtRef, 202 | shared := Shared, 203 | counter := CntRef, 204 | replica_nodes := Replicas} = 205 | Config, undefined) 206 | when ?IS_STRING(Name) -> 207 | process_flag(trap_exit, true), 208 | process_flag(message_queue_data, off_heap), 209 | process_flag(fullsweep_after, 0), 210 | Log = osiris_log:init(Config), 211 | %% reader context can only be cached _after_ log init as we need to ensure 212 | %% there is at least 1 segment/index pair and also that the log has been 213 | %% truncated such that only valid index / segment data remains. 214 | osiris_util:cache_reader_context(self(), Dir, Name, Shared, ExtRef, 215 | fun(Inc) -> 216 | counters:add(CntRef, ?C_READERS, Inc) 217 | end), 218 | Trk = osiris_log:recover_tracking(Log), 219 | %% should this not be be chunk id rather than last offset? 220 | LastOffs = osiris_log:next_offset(Log) - 1, 221 | CommittedOffset = 222 | case osiris_log:tail_info(Log) of 223 | {_, {_, TailChId, _}} when Replicas == [] -> 224 | %% only when there are no replicas can we 225 | %% recover the committed offset from the last 226 | %% batch offset in the log 227 | TailChId; 228 | _ -> 229 | -1 230 | end, 231 | ok = osiris_log:set_committed_chunk_id(Log, CommittedOffset), 232 | counters:put(CntRef, ?C_COMMITTED_OFFSET, CommittedOffset), 233 | counters:put(CntRef, ?C_EPOCH, Epoch), 234 | EvtFmt = maps:get(event_formatter, Config, undefined), 235 | ?INFO("osiris_writer:init/1: name: ~ts last offset: ~b " 236 | "committed chunk id: ~b epoch: ~b", 237 | [Name, LastOffs, CommittedOffset, Epoch]), 238 | {ok, 239 | #?MODULE{cfg = 240 | #cfg{name = Name, 241 | %% reference used for notification 242 | %% if not provided use the name 243 | reference = ExtRef, 244 | event_formatter = EvtFmt, 245 | replicas = Replicas, 246 | directory = Dir, 247 | counter = CntRef}, 248 | committed_offset = CommittedOffset, 249 | replica_state = maps:from_list([{R, {-1, 0}} || R <- Replicas]), 250 | log = Log, 251 | tracking = Trk}}. 252 | 253 | handle_batch(Commands, 254 | #?MODULE{cfg = #cfg{counter = Cnt} = Cfg, 255 | duplicates = Dupes0, 256 | committed_offset = COffs0, 257 | tracking = Trk0} = 258 | State0) -> 259 | 260 | %% process commands in reverse order 261 | case catch lists:foldr(fun handle_command/2, 262 | {State0, [], [], #{}, Trk0, []}, Commands) of 263 | {#?MODULE{log = Log0} = State1, Entries, Replies, Corrs, Trk1, Dupes} -> 264 | {Log1, Trk2} = osiris_log:evaluate_tracking_snapshot(Log0, Trk1), 265 | Now = erlang:system_time(millisecond), 266 | ThisBatchOffs = osiris_log:next_offset(Log1), 267 | {TrkData, Trk} = osiris_tracking:flush(Trk2), 268 | Log = case Entries of 269 | [] when TrkData =/= [] -> 270 | %% TODO: we could set a timer for explicit tracking delta 271 | %% chunks in order to writer fewer of them 272 | osiris_log:write([TrkData], 273 | ?CHNK_TRK_DELTA, 274 | Now, 275 | <<>>, 276 | Log1); 277 | _ -> 278 | osiris_log:write(Entries, 279 | ?CHNK_USER, 280 | Now, 281 | TrkData, 282 | Log1) 283 | end, 284 | State2 = 285 | update_pending(ThisBatchOffs, Corrs, 286 | State1#?MODULE{tracking = Trk, 287 | log = Log}), 288 | 289 | LastChId = 290 | case osiris_log:tail_info(State2#?MODULE.log) of 291 | {_, {_, TailChId, _TailTs}} -> 292 | TailChId; 293 | _ -> 294 | -1 295 | end, 296 | AllChIds = maps:fold(fun (_, {O, _}, Acc) -> 297 | [O | Acc] 298 | end, [LastChId], State2#?MODULE.replica_state), 299 | 300 | COffs = agreed_commit(AllChIds), 301 | 302 | RemDupes = handle_duplicates(COffs, Dupes ++ Dupes0, Cfg), 303 | %% if committed offset has increased - update 304 | State = case COffs > COffs0 of 305 | true -> 306 | P = State2#?MODULE.pending_corrs, 307 | ok = osiris_log:set_committed_chunk_id(Log, COffs), 308 | counters:put(Cnt, ?C_COMMITTED_OFFSET, COffs), 309 | Pending = notify_writers(P, COffs, Cfg), 310 | State2#?MODULE{committed_offset = COffs, 311 | duplicates = RemDupes, 312 | pending_corrs = Pending}; 313 | false -> 314 | State2#?MODULE{duplicates = RemDupes} 315 | end, 316 | {ok, [garbage_collect | Replies], 317 | notify_offset_listeners(notify_data_listeners(State))}; 318 | {stop, normal} -> 319 | {stop, normal} 320 | end. 321 | 322 | terminate(Reason, 323 | #?MODULE{log = Log, 324 | data_listeners = Listeners, 325 | cfg = #cfg{name = Name}}) -> 326 | ?INFO("osiris_writer:terminate/2: name ~ts reason: ~w", 327 | [Name, Reason]), 328 | _ = ets:delete(osiris_reader_context_cache, self()), 329 | ok = osiris_log:close(Log), 330 | [osiris_replica_reader:stop(Pid) || {Pid, _} <- Listeners], 331 | ok. 332 | 333 | format_status(#?MODULE{cfg = #cfg{name = Name, 334 | replicas = Replicas, 335 | reference = ExtRef}, 336 | log = Log, 337 | tracking = Trk, 338 | pending_corrs = PendingCorrs, 339 | replica_state = ReplicaState, 340 | data_listeners = DataListeners, 341 | offset_listeners = OffsetListeners, 342 | committed_offset = CommittedOffset}) -> 343 | #{name => Name, 344 | external_reference => ExtRef, 345 | replica_nodes => Replicas, 346 | log => osiris_log:format_status(Log), 347 | tracking => osiris_tracking:overview(Trk), 348 | replica_state => ReplicaState, 349 | %%TODO make lqueue for performance 350 | num_pending_correlations => queue:len(PendingCorrs), 351 | num_data_listeners => length(DataListeners), 352 | num_offset_listeners => length(OffsetListeners), 353 | committed_offset => CommittedOffset 354 | }. 355 | 356 | %% Internal 357 | 358 | update_pending(BatchOffs, Corrs, 359 | #?MODULE{cfg = #cfg{}, pending_corrs = Pending0} = State) -> 360 | case Corrs of 361 | _ when map_size(Corrs) == 0 -> 362 | State; 363 | _ -> 364 | State#?MODULE{pending_corrs = 365 | queue:in({BatchOffs, Corrs}, Pending0)} 366 | end. 367 | 368 | put_writer(undefined, _ChId, _Corr, Trk) -> 369 | Trk; 370 | put_writer(WriterId, ChunkId, Corr, Trk) when is_binary(WriterId) -> 371 | osiris_tracking:add(WriterId, sequence, Corr, ChunkId, Trk). 372 | 373 | handle_duplicates(_CommittedOffset, [], _Cfg) -> 374 | []; 375 | handle_duplicates(CommittedOffset, Dupes, #cfg{} = Cfg) 376 | when is_list(Dupes) -> 377 | %% turn list of dupes into corr map 378 | {Rem, Corrs} = 379 | lists:foldr(fun ({ChId, Pid, WriterId, Seq}, {Rem, Corrs0}) 380 | when ChId =< CommittedOffset -> 381 | Corrs = 382 | maps:update_with({Pid, WriterId}, 383 | fun(C) -> [Seq | C] end, 384 | [Seq], 385 | Corrs0), 386 | {Rem, Corrs}; 387 | (Dupe, {Rem, Corrs}) -> 388 | {[Dupe | Rem], Corrs} 389 | end, 390 | {[], #{}}, Dupes), 391 | send_written_events(Cfg, Corrs), 392 | Rem. 393 | 394 | handle_command({cast, {write, Pid, WriterId, Corr, R}}, 395 | {#?MODULE{log = Log} = State, Records, Replies, Corrs0, Trk, Dupes}) -> 396 | case is_duplicate(WriterId, Corr, State, Trk) of 397 | {false, _} -> 398 | Corrs = 399 | maps:update_with({Pid, WriterId}, 400 | fun(C) -> [Corr | C] end, 401 | [Corr], 402 | Corrs0), 403 | ChId = osiris_log:next_offset(Log), 404 | {State, 405 | [R | Records], 406 | Replies, 407 | Corrs, 408 | put_writer(WriterId, ChId, Corr, Trk), 409 | Dupes}; 410 | {true, ChId} -> 411 | %% add write to duplications list 412 | {State, 413 | Records, 414 | Replies, 415 | Corrs0, 416 | Trk, 417 | [{ChId, Pid, WriterId, Corr} | Dupes]} 418 | end; 419 | handle_command({cast, {write, R}}, 420 | {#?MODULE{} = State, Records, Replies, Corrs0, Trk, Dupes}) -> 421 | {State, [R | Records], Replies, Corrs0, Trk, Dupes}; 422 | handle_command({cast, {write_tracking, TrackingId, TrackingType, TrackingData}}, 423 | {#?MODULE{log = Log} = State, Records, Replies, Corrs, Trk0, Dupes}) -> 424 | ChunkId = osiris_log:next_offset(Log), 425 | Trk = osiris_tracking:add(TrackingId, TrackingType, TrackingData, ChunkId, Trk0), 426 | {State, Records, Replies, Corrs, Trk, Dupes}; 427 | handle_command({call, From, {read_tracking, TrackingType, TrackingId}}, 428 | {State, Records, Replies0, Corrs, Trk, Dupes}) -> 429 | %% need to merge pending tracking entries before read 430 | %% 431 | Tracking = case osiris_tracking:query(TrackingId, TrackingType, Trk) of 432 | {ok, T} -> 433 | {TrackingType, T}; 434 | {error, not_found} -> 435 | undefined 436 | end, 437 | Replies = [{reply, From, Tracking} | Replies0], 438 | {State, Records, Replies, Corrs, Trk, Dupes}; 439 | handle_command({call, From, read_tracking}, 440 | {State, Records, Replies0, Corrs, Trk, Dupes}) -> 441 | Tracking = osiris_tracking:overview(Trk), 442 | Replies = [{reply, From, Tracking} | Replies0], 443 | {State, Records, Replies, Corrs, Trk, Dupes}; 444 | handle_command({cast, {register_data_listener, Pid, Offset}}, 445 | {#?MODULE{data_listeners = Listeners} = State0, 446 | Records, 447 | Replies, 448 | Corrs, 449 | Trk, 450 | Dupes}) -> 451 | State = State0#?MODULE{data_listeners = [{Pid, Offset} | Listeners]}, 452 | {State, Records, Replies, Corrs, Trk, Dupes}; 453 | handle_command({cast, 454 | {register_offset_listener, Pid, EvtFormatter, Offset}}, 455 | {#?MODULE{offset_listeners = Listeners} = State0, 456 | Records, 457 | Replies, 458 | Corrs, 459 | Trk, 460 | Dupes}) -> 461 | State = 462 | State0#?MODULE{offset_listeners = 463 | [{Pid, Offset, EvtFormatter} | Listeners]}, 464 | {State, Records, Replies, Corrs, Trk, Dupes}; 465 | handle_command({cast, {ack, ReplicaNode, {Offset, _} = OffsetTs}}, 466 | {#?MODULE{replica_state = ReplicaState0} = State0, 467 | Records, 468 | Replies, 469 | Corrs, 470 | Trk, 471 | Dupes}) -> 472 | % ?DEBUG("osiris_writer ack from ~w at ~b", [ReplicaNode, Offset]), 473 | ReplicaState = 474 | case ReplicaState0 of 475 | #{ReplicaNode := {O, _}} when Offset > O -> 476 | ReplicaState0#{ReplicaNode => OffsetTs}; 477 | _ -> 478 | %% ignore anything else including acks from unknown replicas 479 | ReplicaState0 480 | end, 481 | State = State0#?MODULE{replica_state = ReplicaState}, 482 | {State, Records, Replies, Corrs, Trk, Dupes}; 483 | handle_command({call, From, get_reader_context}, 484 | {#?MODULE{cfg = 485 | #cfg{reference = Ref, 486 | name = Name, 487 | directory = Dir, 488 | counter = CntRef}, 489 | log = Log, 490 | committed_offset = COffs} = 491 | State, 492 | Records, 493 | Replies, 494 | Corrs, 495 | Trk, 496 | Dupes}) -> 497 | Shared = osiris_log:get_shared(Log), 498 | Reply = 499 | {reply, From, 500 | #{dir => Dir, 501 | name => Name, 502 | committed_offset => max(0, COffs), 503 | shared => Shared, 504 | reference => Ref, 505 | readers_counter_fun => fun(Inc) -> counters:add(CntRef, ?C_READERS, Inc) end 506 | }}, 507 | {State, Records, [Reply | Replies], Corrs, Trk, Dupes}; 508 | handle_command({call, From, {query_writers, QueryFun}}, 509 | {State, Records, Replies0, Corrs, Trk, Dupes}) -> 510 | %% need to merge pending tracking entries before read 511 | #{sequences := Writers} = osiris_tracking:overview(Trk), 512 | Result = 513 | try QueryFun(Writers) of 514 | R -> 515 | R 516 | catch 517 | Err -> 518 | Err 519 | end, 520 | Replies = [{reply, From, Result} | Replies0], 521 | {State, Records, Replies, Corrs, Trk, Dupes}; 522 | handle_command({call, From, query_replication_state}, 523 | {#?MODULE{log = Log, 524 | replica_state = R} = State, Records, Replies0, 525 | Corrs, Trk, Dupes}) -> 526 | %% need to merge pending tracking entries before read 527 | Result = case osiris_log:tail_info(Log) of 528 | {0, empty} -> 529 | R#{node() => {-1, 0}}; 530 | {_, {_E, O, T}} -> 531 | R#{node() => {O, T}} 532 | end, 533 | Replies = [{reply, From, Result} | Replies0], 534 | {State, Records, Replies, Corrs, Trk, Dupes}; 535 | handle_command({call, From, {update_retention, Retention}}, 536 | {#?MODULE{log = Log0} = State, 537 | Records, 538 | Replies0, 539 | Corrs, 540 | Trk, 541 | Dupes}) -> 542 | Log = osiris_log:update_retention(Retention, Log0), 543 | Replies = [{reply, From, ok} | Replies0], 544 | {State#?MODULE{log = Log}, Records, Replies, Corrs, Trk, Dupes}; 545 | handle_command(osiris_stop, _Acc) -> 546 | throw({stop, normal}); 547 | handle_command(_Unk, Acc) -> 548 | ?DEBUG("osiris_writer: unknown command ~w", [_Unk]), 549 | Acc. 550 | 551 | notify_data_listeners(#?MODULE{log = Seg, data_listeners = L0} = 552 | State) -> 553 | NextOffset = osiris_log:next_offset(Seg), 554 | {Notify, L} = 555 | lists:partition(fun({_Pid, O}) -> O < NextOffset end, L0), 556 | [gen_server:cast(P, {more_data, NextOffset}) || {P, _} <- Notify], 557 | State#?MODULE{data_listeners = L}. 558 | 559 | notify_offset_listeners(#?MODULE{cfg = 560 | #cfg{reference = Ref, 561 | event_formatter = EvtFmt}, 562 | committed_offset = COffs, 563 | offset_listeners = L0} = 564 | State) -> 565 | {Notify, L} = 566 | lists:partition(fun({_Pid, O, _}) -> O =< COffs end, L0), 567 | _ = [begin 568 | Evt = 569 | %% the per offset listener event formatter takes precedence of 570 | %% the process scoped one 571 | wrap_osiris_event( 572 | select_formatter(Fmt, EvtFmt), 573 | {osiris_offset, Ref, COffs}), 574 | P ! Evt 575 | end 576 | || {P, _, Fmt} <- Notify], 577 | State#?MODULE{offset_listeners = L}. 578 | 579 | select_formatter(undefined, Fmt) -> 580 | Fmt; 581 | select_formatter(Fmt, _) -> 582 | Fmt. 583 | 584 | notify_writers(Q0, COffs, Cfg) -> 585 | case queue:peek(Q0) of 586 | {value, {O, Corrs}} when O =< COffs -> 587 | send_written_events(Cfg, Corrs), 588 | {_, Q} = queue:out(Q0), 589 | notify_writers(Q, COffs, Cfg); 590 | _ -> 591 | Q0 592 | end. 593 | 594 | send_written_events(#cfg{reference = ExtRef, 595 | event_formatter = Fmt}, 596 | Corrs) -> 597 | maps:fold(fun({P, WId}, V, Acc) -> 598 | %% TODO: if the writer is on a remote node this could block 599 | %% which is bad but we'd have to consider the downsides of using 600 | %% send with noconnect and nosuspend here 601 | % ?DEBUG("send_written_events ~s ~w", [ExtRef, V]), 602 | P ! wrap_osiris_event(Fmt, 603 | {osiris_written, 604 | ExtRef, 605 | WId, 606 | lists:reverse(V)}), 607 | Acc 608 | end, ok, 609 | Corrs), 610 | ok. 611 | 612 | wrap_osiris_event(undefined, Evt) -> 613 | Evt; 614 | wrap_osiris_event({M, F, A}, Evt) -> 615 | apply(M, F, [Evt | A]). 616 | 617 | -spec agreed_commit([osiris:offset()]) -> osiris:offset(). 618 | agreed_commit(Indexes) -> 619 | SortedIdxs = lists:sort(fun erlang:'>'/2, Indexes), 620 | Nth = length(SortedIdxs) div 2 + 1, 621 | lists:nth(Nth, SortedIdxs). 622 | 623 | is_duplicate(undefined, _, _, _) -> 624 | {false, 0}; 625 | is_duplicate(WriterId, Corr, #?MODULE{log = _Log}, Trk) -> 626 | case osiris_tracking:query(WriterId, sequence, Trk) of 627 | {ok, {ChunkId, Seq}} -> 628 | {Corr =< Seq, ChunkId}; 629 | {error, not_found} -> 630 | {false, 0} 631 | end. 632 | -------------------------------------------------------------------------------- /src/osiris_replica.erl: -------------------------------------------------------------------------------- 1 | %% This Source Code Form is subject to the terms of the Mozilla Public 2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this 3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/. 4 | %% 5 | %% Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. 6 | %% 7 | 8 | -module(osiris_replica). 9 | 10 | -behaviour(gen_server). 11 | -behaviour(osiris_member). 12 | 13 | -include("osiris.hrl"). 14 | 15 | -define(DEFAULT_PORT_RANGE, {6000, 6500}). 16 | 17 | %% osiris replica, starts TCP listener ("server side" of the link), 18 | %% spawns remote reader, TCP listener replicates and 19 | %% confirms latest offset back to primary 20 | 21 | %% API functions 22 | -export([start/2, 23 | start_link/1 24 | ]). 25 | %% Test 26 | -export([get_port/1, combine_ips_hosts/4]). 27 | %% gen_server callbacks 28 | -export([init/1, 29 | handle_continue/2, 30 | handle_call/3, 31 | handle_cast/2, 32 | handle_info/2, 33 | terminate/2, 34 | code_change/3, 35 | format_status/1]). 36 | 37 | %% holds static or rarely changing fields 38 | -record(cfg, 39 | {name :: osiris:name(), 40 | leader_pid :: pid(), 41 | acceptor_pid :: pid(), 42 | replica_reader_pid :: pid(), 43 | directory :: file:filename_all(), 44 | port :: non_neg_integer(), 45 | transport :: osiris_log:transport(), 46 | socket :: undefined | gen_tcp:socket() | ssl:sslsocket(), 47 | gc_interval :: infinity | non_neg_integer(), 48 | reference :: term(), 49 | event_formatter :: undefined | mfa(), 50 | counter :: counters:counters_ref(), 51 | token :: undefined | binary()}). 52 | 53 | -type parse_state() :: 54 | undefined | 55 | binary() | 56 | {non_neg_integer(), iolist(), non_neg_integer()}. 57 | 58 | -record(?MODULE, 59 | {cfg :: #cfg{}, 60 | parse_state :: parse_state(), 61 | log :: osiris_log:state(), 62 | committed_chunk_id = -1 :: -1 | osiris:offset(), 63 | offset_listeners = [] :: 64 | [{pid(), osiris:offset(), mfa() | undefined}] 65 | }). 66 | 67 | -opaque state() :: #?MODULE{}. 68 | 69 | -export_type([state/0]). 70 | 71 | -define(C_COMMITTED_OFFSET, ?C_NUM_LOG_FIELDS + 1). 72 | -define(C_FORCED_GCS, ?C_NUM_LOG_FIELDS + 2). 73 | -define(C_PACKETS, ?C_NUM_LOG_FIELDS + 3). 74 | -define(C_READERS, ?C_NUM_LOG_FIELDS + 4). 75 | -define(C_EPOCH, ?C_NUM_LOG_FIELDS + 5). 76 | -define(ADD_COUNTER_FIELDS, 77 | [{committed_offset, ?C_COMMITTED_OFFSET, counter, "Last committed offset"}, 78 | {forced_gcs, ?C_FORCED_GCS, counter, "Number of garbage collection runs"}, 79 | {packets, ?C_PACKETS, counter, "Number of packets"}, 80 | {readers, ?C_READERS, counter, "Number of readers"}, 81 | {epoch, ?C_EPOCH, counter, "Current epoch"}]). 82 | 83 | -define(DEFAULT_ONE_TIME_TOKEN_TIMEOUT, 30000). 84 | -define(TOKEN_SIZE, 32). 85 | -define(DEF_REC_BUF, 408300 * 5). 86 | 87 | %%%=================================================================== 88 | %%% API functions 89 | %%%=================================================================== 90 | 91 | -spec start(node(), Config :: osiris:config()) -> 92 | supervisor:startchild_ret(). 93 | start(Node, Config = #{name := Name}) when ?IS_STRING(Name) -> 94 | case supervisor:start_child({?SUP, Node}, 95 | #{id => Name, 96 | start => {?MODULE, start_link, [Config]}, 97 | restart => temporary, 98 | shutdown => 5000, 99 | type => worker, 100 | modules => [?MODULE]}) of 101 | {ok, Pid} = Res -> 102 | %% make a dummy call to block until intialisation is complete 103 | _ = await(Pid), 104 | Res; 105 | {ok, _, Pid} = Res -> 106 | _ = await(Pid), 107 | Res; 108 | Err -> 109 | Err 110 | end. 111 | 112 | %%-------------------------------------------------------------------- 113 | %% @doc 114 | %% Starts the server 115 | %% 116 | %% @spec start_link() -> {ok, Pid} | ignore | {error, Error} 117 | %% @end 118 | %%-------------------------------------------------------------------- 119 | start_link(Conf) -> 120 | gen_server:start_link(?MODULE, Conf, []). 121 | 122 | get_port(Server) -> 123 | gen_server:call(Server, get_port). 124 | 125 | await(Server) -> 126 | gen_server:call(Server, await, infinity). 127 | 128 | %%%=================================================================== 129 | %%% gen_server callbacks 130 | %%%=================================================================== 131 | 132 | %%-------------------------------------------------------------------- 133 | %% @private 134 | %% @doc 135 | %% Initializes the server 136 | %% 137 | %% @spec init(Args) -> {ok, State} | 138 | %% {ok, State, Timeout} | 139 | %% ignore | 140 | %% {stop, Reason} 141 | %% @end 142 | %%-------------------------------------------------------------------- 143 | init(Config) -> 144 | {ok, undefined, {continue, Config}}. 145 | 146 | handle_continue(#{name := Name0, 147 | epoch := Epoch, 148 | leader_pid := LeaderPid, 149 | reference := ExtRef} = Config, undefined) 150 | when ?IS_STRING(Name0) -> 151 | Name = osiris_util:normalise_name(Name0), 152 | process_flag(trap_exit, true), 153 | process_flag(message_queue_data, off_heap), 154 | Node = node(LeaderPid), 155 | 156 | %% osiris_writer:overview/1 may need to call into the writer process and 157 | %% wait for it to initialise before replying. This could take some time 158 | %% so it makes no sense to have an upper bound on the timeout 159 | case rpc:call(Node, osiris_writer, overview, [LeaderPid], infinity) of 160 | {error, no_process} -> 161 | ?INFO_(Name, "Writer process not alive, exiting...", []), 162 | {stop, {shutdown, writer_unavailable}, undefined}; 163 | missing_file -> 164 | ?INFO_(Name, "missing file returned from writer, exiting...", []), 165 | {stop, {shutdown, missing_file}, undefined}; 166 | {error, _} = Err -> 167 | {stop, Err, undefined}; 168 | {badrpc, {'EXIT', shutdown}} -> 169 | ?INFO_(Name, "Writer process shutting down, exiting...", []), 170 | {stop, {shutdown, writer_unavailable}, undefined}; 171 | {badrpc, nodedown} -> 172 | ?INFO_(Name, "Writer process node is down, exiting...", []), 173 | {stop, {shutdown, writer_unavailable}, undefined}; 174 | {badrpc, Reason} -> 175 | {stop, {badrpc, Reason}, undefined}; 176 | {ok, {LeaderRange, LeaderEpochOffs}} -> 177 | {Min, Max} = application:get_env(osiris, port_range, 178 | ?DEFAULT_PORT_RANGE), 179 | Transport = application:get_env(osiris, replication_transport, tcp), 180 | Self = self(), 181 | CntName = {?MODULE, ExtRef}, 182 | 183 | Dir = osiris_log:directory(Config), 184 | Log = osiris_log:init_acceptor(LeaderRange, LeaderEpochOffs, 185 | Config#{dir => Dir, 186 | counter_spec => 187 | {CntName, ?ADD_COUNTER_FIELDS}}), 188 | CntRef = osiris_log:counters_ref(Log), 189 | {NextOffset, LastChunk} = TailInfo = osiris_log:tail_info(Log), 190 | 191 | case LastChunk of 192 | empty -> 193 | ok; 194 | {_, LastChId, LastTs} -> 195 | %% need to ack last chunk back to leader so that it can 196 | %% re-discover the committed offset 197 | osiris_writer:ack(LeaderPid, {LastChId, LastTs}) 198 | end, 199 | ?INFO_(Name, "osiris replica starting in epoch ~b, next offset ~b, tail info ~w", 200 | [Epoch, NextOffset, TailInfo]), 201 | 202 | %% HostName: append the HostName to the Ip(s) list: in some cases 203 | %% like NAT or redirect the local ip addresses are not enough. 204 | %% ex: In docker with host network configuration the `inet:getaddrs` 205 | %% are only the IP(s) inside docker but the dns lookup happens 206 | %% outside the docker image (host machine). 207 | %% The host name is the last to leave the compatibility. 208 | %% See: rabbitmq/rabbitmq-server#3510 209 | {ok, HostName} = inet:gethostname(), 210 | 211 | %% Ips: are the first values used to connect the 212 | %% replicas 213 | {ok, Ips} = inet:getaddrs(HostName, inet), 214 | 215 | %% HostNameFromHost: The hostname value from RABBITMQ_NODENAME 216 | %% can be different from the machine hostname. 217 | %% In case of docker with bridge and extra_hosts the use case can be: 218 | %% RABBITMQ_NODENAME=rabbit@my-domain 219 | %% docker hostname = "114f4317c264" 220 | %% the HostNameFromHost will be "my-domain". 221 | %% btw 99% of the time the HostNameFromHost is equal to HostName. 222 | %% see: rabbitmq/osiris/issues/53 for more details 223 | HostNameFromHost = osiris_util:hostname_from_node(), 224 | 225 | IpsHosts = combine_ips_hosts(Transport, Ips, HostName, 226 | HostNameFromHost), 227 | 228 | Token = crypto:strong_rand_bytes(?TOKEN_SIZE), 229 | ?DEBUG_(Name, "replica resolved host endpoints: ~0p", [IpsHosts]), 230 | {Port, LSock} = open_listener(Transport, {Min, Max}, 0), 231 | ?DEBUG_(Name, "replica listening on port '~b' using transport ~s", 232 | [Port, Transport]), 233 | Acceptor = spawn_link(fun() -> accept(Name, Transport, LSock, Self) end), 234 | ?DEBUG_(Name, "starting replica reader on node '~w'", [Node]), 235 | 236 | ReplicaReaderConf = #{hosts => IpsHosts, 237 | port => Port, 238 | transport => Transport, 239 | name => Name, 240 | replica_pid => self(), 241 | leader_pid => LeaderPid, 242 | start_offset => TailInfo, 243 | reference => ExtRef, 244 | connection_token => Token}, 245 | case osiris_replica_reader:start(Node, ReplicaReaderConf) of 246 | {ok, RRPid} -> 247 | true = link(RRPid), 248 | ?DEBUG_(Name, "started replica reader on node '~w'", [Node]), 249 | GcInterval0 = application:get_env(osiris, 250 | replica_forced_gc_default_interval, 251 | 4999), 252 | 253 | GcInterval1 = case is_integer(GcInterval0) of 254 | true -> 255 | _ = erlang:send_after(GcInterval0, self(), force_gc), 256 | GcInterval0; 257 | false -> 258 | infinity 259 | end, 260 | counters:put(CntRef, ?C_COMMITTED_OFFSET, -1), 261 | counters:put(CntRef, ?C_EPOCH, Epoch), 262 | Shared = osiris_log:get_shared(Log), 263 | osiris_util:cache_reader_context(self(), Dir, Name, Shared, ExtRef, 264 | fun(Inc) -> 265 | counters:add(CntRef, ?C_READERS, Inc) 266 | end), 267 | EvtFmt = maps:get(event_formatter, Config, undefined), 268 | {noreply, 269 | #?MODULE{cfg = 270 | #cfg{name = Name, 271 | leader_pid = LeaderPid, 272 | acceptor_pid = Acceptor, 273 | replica_reader_pid = RRPid, 274 | directory = Dir, 275 | port = Port, 276 | gc_interval = GcInterval1, 277 | reference = ExtRef, 278 | event_formatter = EvtFmt, 279 | counter = CntRef, 280 | token = Token, 281 | transport = Transport}, 282 | log = Log, 283 | parse_state = undefined}}; 284 | {error, {connection_refused = R, _}} -> 285 | %% we don't log details for connection_refused, 286 | %% they are already in the logs of the other node 287 | ?WARN_(Name, "failed to start replica reader on node '~w'. " 288 | "Reason ~0p.", [Node, R]), 289 | {stop, {shutdown, R}, undefined}; 290 | {error, Reason} -> 291 | ?WARN_(Name, "failed to start replica reader on node '~w'. " 292 | "Reason ~0p.", [Node, Reason]), 293 | {stop, {shutdown, Reason}, undefined} 294 | end 295 | end. 296 | 297 | combine_ips_hosts(tcp, IPs, HostName, HostNameFromHost) when 298 | HostName =/= HostNameFromHost -> 299 | lists:append(IPs, [HostName, HostNameFromHost]); 300 | combine_ips_hosts(tcp, IPs, HostName, _HostNameFromHost) -> 301 | lists:append(IPs, [HostName]); 302 | combine_ips_hosts(ssl, IPs, HostName, HostNameFromHost) when 303 | HostName =/= HostNameFromHost -> 304 | lists:append([HostName, HostNameFromHost], IPs); 305 | combine_ips_hosts(ssl, IPs, HostName, _HostNameFromHost) -> 306 | lists:append([HostName], IPs). 307 | 308 | open_listener(_Transport, Range, 100) -> 309 | throw({stop, {no_available_ports_in_range, Range}}); 310 | open_listener(Transport, {Min, Max} = Range, Attempts) -> 311 | Offs = rand:uniform(Max - Min), 312 | Port = Min + Offs, 313 | Options = listener_opts(Transport), 314 | case listen(Transport, Port, Options) of 315 | {ok, LSock} -> 316 | {Port, LSock}; 317 | {error, eaddrinuse} -> 318 | timer:sleep(Attempts), 319 | open_listener(Transport, Range, Attempts + 1); 320 | E -> 321 | throw({stop, E}) 322 | end. 323 | 324 | accept(Name, tcp, LSock, Process) -> 325 | case gen_tcp:accept(LSock) of 326 | {ok, Sock} -> 327 | try gen_tcp:close(LSock) of 328 | ok -> ok 329 | catch _:Err -> 330 | ?DEBUG_(Name, "gen_tcp:close/1 failed with ~0p", [Err]) 331 | end, 332 | ok = gen_tcp:controlling_process(Sock, Process), 333 | Process ! {socket, Sock}, 334 | ok; 335 | {error, Err} -> 336 | ?DEBUG_(Name, "gen_tcp:accept/1 failed with ~0p", [Err]), 337 | gen_tcp:close(LSock), 338 | ok 339 | end; 340 | accept(Name, ssl, LSock, Process) -> 341 | ?DEBUG_(Name, "Starting socket acceptor for replication over TLS", []), 342 | case ssl:transport_accept(LSock) of 343 | {ok, Sock0} -> 344 | SslOpts = application:get_env(osiris, replication_server_ssl_options, []), 345 | case ssl:handshake(Sock0, SslOpts) of 346 | {ok, Sock} -> 347 | _ = ssl:close(LSock), 348 | ok = ssl:controlling_process(Sock, Process), 349 | Process ! {socket, Sock}, 350 | ok; 351 | {error, {tls_alert, {handshake_failure, _}}} -> 352 | ?DEBUG_(Name, "Handshake failure, restarting listener...", 353 | []), 354 | _ = spawn_link(fun() -> accept(Name, ssl, LSock, Process) end), 355 | ok; 356 | {error, E} -> 357 | ?DEBUG_(Name, "Error during handshake ~w", [E]); 358 | H -> 359 | ?DEBUG_(Name, "Unexpected result from TLS handshake ~w", [H]) 360 | end, 361 | ok; 362 | {error, Err} -> 363 | ?DEBUG_(Name, "ssl:transport_accept/1 failed with ~0p", [Err]), 364 | _ = ssl:close(LSock), 365 | ok 366 | end. 367 | 368 | %%-------------------------------------------------------------------- 369 | %% @private 370 | %% @doc 371 | %% Handling call messages 372 | %% 373 | %% @spec handle_call(Request, From, State) -> 374 | %% {reply, Reply, State} | 375 | %% {reply, Reply, State, Timeout} | 376 | %% {noreply, State} | 377 | %% {noreply, State, Timeout} | 378 | %% {stop, Reason, Reply, State} | 379 | %% {stop, Reason, State} 380 | %% @end 381 | %%-------------------------------------------------------------------- 382 | handle_call(get_port, _From, 383 | #?MODULE{cfg = #cfg{port = Port}} = State) -> 384 | {reply, Port, State}; 385 | handle_call(get_reader_context, _From, 386 | #?MODULE{cfg = 387 | #cfg{name = Name, 388 | directory = Dir, 389 | reference = Ref, 390 | counter = CntRef}, 391 | committed_chunk_id = COffs, 392 | log = Log} = 393 | State) -> 394 | Shared = osiris_log:get_shared(Log), 395 | Reply = 396 | #{dir => Dir, 397 | name => Name, 398 | committed_offset => COffs, 399 | shared => Shared, 400 | reference => Ref, 401 | readers_counter_fun => fun(Inc) -> counters:add(CntRef, ?C_READERS, Inc) end}, 402 | {reply, Reply, State}; 403 | handle_call({update_retention, Retention}, _From, 404 | #?MODULE{log = Log0} = State) -> 405 | Log = osiris_log:update_retention(Retention, Log0), 406 | {reply, ok, State#?MODULE{log = Log}}; 407 | handle_call(await, _From, State) -> 408 | {reply, ok, State}; 409 | handle_call(Unknown, _From, 410 | #?MODULE{cfg = #cfg{name = Name}} = State) -> 411 | ?INFO_(Name, "unknown command ~W", [Unknown, 10]), 412 | {reply, {error, unknown_command}, State}. 413 | 414 | %%-------------------------------------------------------------------- 415 | %% @private 416 | %% @doc 417 | %% Handling cast messages 418 | %% 419 | %% @spec handle_cast(Msg, State) -> {noreply, State} | 420 | %% {noreply, State, Timeout} | 421 | %% {stop, Reason, State} 422 | %% @end 423 | %%-------------------------------------------------------------------- 424 | handle_cast({committed_offset, CommittedChId}, 425 | #?MODULE{cfg = #cfg{counter = Cnt}, 426 | log = Log, 427 | committed_chunk_id = LastCommittedChId} = 428 | State) -> 429 | case CommittedChId > LastCommittedChId of 430 | true -> 431 | %% notify offset listeners 432 | counters:put(Cnt, ?C_COMMITTED_OFFSET, CommittedChId), 433 | ok = osiris_log:set_committed_chunk_id(Log, CommittedChId), 434 | {noreply, 435 | notify_offset_listeners( 436 | State#?MODULE{committed_chunk_id = CommittedChId})}; 437 | false -> 438 | State 439 | end; 440 | handle_cast({register_offset_listener, Pid, EvtFormatter, Offset}, 441 | #?MODULE{cfg = #cfg{reference = Ref, 442 | event_formatter = DefaultFmt}, 443 | log = Log, 444 | offset_listeners = Listeners} = State) -> 445 | Max = max_readable_chunk_id(Log), 446 | case Offset =< Max of 447 | true -> 448 | %% only evaluate the request, the rest will be evaluated 449 | %% when data is written or committed 450 | Evt = wrap_osiris_event( 451 | select_formatter(EvtFormatter, DefaultFmt), 452 | {osiris_offset, Ref, Max}), 453 | Pid ! Evt, 454 | {noreply, State}; 455 | false -> 456 | %% queue the offset listener for later 457 | {noreply, 458 | State#?MODULE{offset_listeners = [{Pid, Offset, EvtFormatter} | 459 | Listeners]}} 460 | end; 461 | handle_cast(Msg, #?MODULE{cfg = #cfg{name = Name}} = State) -> 462 | ?DEBUG_(Name, "osiris_replica unhandled cast ~w", [Msg]), 463 | {noreply, State}. 464 | 465 | %%-------------------------------------------------------------------- 466 | %% @private 467 | %% @doc 468 | %% Handling all non call/cast messages 469 | %% 470 | %% @spec handle_info(Info, State) -> {noreply, State} | 471 | %% {noreply, State, Timeout} | 472 | %% {stop, Reason, State} 473 | %% @end 474 | %%-------------------------------------------------------------------- 475 | handle_info(force_gc, 476 | #?MODULE{cfg = #cfg{gc_interval = Interval, 477 | counter = Cnt}} = 478 | State) -> 479 | garbage_collect(), 480 | counters:add(Cnt, ?C_FORCED_GCS, 1), 481 | case is_integer(Interval) of 482 | true -> 483 | _ = erlang:send_after(Interval, self(), force_gc), 484 | ok; 485 | false -> 486 | ok 487 | end, 488 | {noreply, State}; 489 | handle_info({socket, Socket}, #?MODULE{cfg = #cfg{name = Name, 490 | token = Token, 491 | transport = Transport} = Cfg} = State) -> 492 | 493 | Timeout = application:get_env(osiris, one_time_token_timeout, 494 | ?DEFAULT_ONE_TIME_TOKEN_TIMEOUT), 495 | case recv(Transport, Socket, ?TOKEN_SIZE, Timeout) of 496 | {ok, Token} -> 497 | %% token validated, all good we can let the flood of data begin 498 | ok = setopts(Transport, Socket, [{active, 5}]), 499 | {noreply, State#?MODULE{cfg = Cfg#cfg{socket = Socket}}}; 500 | {ok, Other} -> 501 | ?WARN_(Name, "invalid token received ~w expected ~w", 502 | [Other, Token]), 503 | {stop, invalid_token, State}; 504 | {error, Reason} -> 505 | ?WARN_(Name, "error awaiting token ~w", [Reason]) 506 | end; 507 | handle_info({tcp, Socket, Bin}, State) -> 508 | handle_incoming_data(Socket, Bin, State); 509 | handle_info({ssl, Socket, Bin}, State) -> 510 | handle_incoming_data(Socket, Bin, State); 511 | handle_info({tcp_passive, Socket}, 512 | #?MODULE{cfg = #cfg{socket = Socket}} = State) -> 513 | %% we always top up before processing each packet so no need to do anything 514 | %% here 515 | {noreply, State}; 516 | handle_info({ssl_passive, Socket}, 517 | #?MODULE{cfg = #cfg{socket = Socket}} = State) -> 518 | %% we always top up before processing each packet so no need to do anything 519 | %% here 520 | {noreply, State}; 521 | handle_info({tcp_closed, Socket}, 522 | #?MODULE{cfg = #cfg{name = Name, socket = Socket}} = State) -> 523 | ?DEBUG_(Name, "Socket closed. Exiting...", []), 524 | {stop, normal, State}; 525 | handle_info({ssl_closed, Socket}, 526 | #?MODULE{cfg = #cfg{name = Name, socket = Socket}} = State) -> 527 | ?DEBUG_(Name, "TLS socket closed. Exiting...", []), 528 | {stop, normal, State}; 529 | handle_info({tcp_error, Socket, Error}, 530 | #?MODULE{cfg = #cfg{name = Name, socket = Socket}} = State) -> 531 | ?DEBUG_(Name, "osiris_replica: ~ts Socket error ~0p. Exiting...", 532 | [Error]), 533 | {stop, {tcp_error, Error}, State}; 534 | handle_info({ssl_error, Socket, Error}, 535 | #?MODULE{cfg = #cfg{name = Name, socket = Socket}} = State) -> 536 | ?DEBUG_(Name, "TLS socket error ~w. Exiting...", 537 | [Error]), 538 | {stop, {ssl_error, Error}, State}; 539 | handle_info({'DOWN', _Ref, process, Pid, Info}, 540 | #?MODULE{cfg = #cfg{name = Name}} = State) -> 541 | ?DEBUG_(Name, "DOWN received for Pid ~w, Info: ~w", 542 | [Pid, Info]), 543 | {noreply, State}; 544 | handle_info({'EXIT', RRPid, Info}, 545 | #?MODULE{cfg = #cfg{name = Name, 546 | replica_reader_pid = RRPid}} = State) -> 547 | %% any replica reader exit is troublesome and requires the replica to also 548 | %% terminate 549 | case lists:member(Info, [normal, shutdown]) of 550 | true -> 551 | ?DEBUG_(Name, "replica reader ~w exited with ~w", [RRPid, Info]); 552 | false -> 553 | ?ERROR_(Name, "replica reader ~w exited with ~w", [RRPid, Info]) 554 | end, 555 | {stop, {shutdown, Info}, State}; 556 | handle_info({'EXIT', Ref, normal}, 557 | #?MODULE{cfg = #cfg{name = Name}} = State) -> 558 | %% we assume any 'normal' EXIT is fine to ignore (port etc) 559 | ?DEBUG_(Name, "EXIT received for ~w with 'normal'", [Ref]), 560 | {noreply, State}; 561 | handle_info({'EXIT', Ref, Info}, 562 | #?MODULE{cfg = #cfg{name = Name}} = State) -> 563 | ?WARN_(Name, "unexpected linked process or port ~w exited with ~w", 564 | [Ref, Info]), 565 | {stop, unexpected_exit, State}. 566 | 567 | handle_incoming_data(Socket, Bin, 568 | #?MODULE{cfg = 569 | #cfg{socket = Socket, 570 | leader_pid = LeaderPid, 571 | transport = Transport, 572 | counter = Cnt}, 573 | parse_state = ParseState0, 574 | log = Log0} = 575 | State0) -> 576 | counters:add(Cnt, ?C_PACKETS, 1), 577 | %% deliberately ignoring return value here as it would fail if the 578 | %% tcp connection has been closed and we still want to try to process 579 | %% any messages still in the mailbox 580 | _ = setopts(Transport, Socket, [{active, 1}]), 581 | %% validate chunk 582 | {ParseState, OffsetChunks} = parse_chunk(Bin, ParseState0, []), 583 | {OffsetTimestamp, Log} = 584 | lists:foldl(fun({OffsetTs, B}, {_OffsTs, Acc0}) -> 585 | Acc = osiris_log:accept_chunk(B, Acc0), 586 | {OffsetTs, Acc} 587 | end, 588 | {undefined, Log0}, OffsetChunks), 589 | State1 = State0#?MODULE{log = Log, parse_state = ParseState}, 590 | case OffsetTimestamp of 591 | undefined -> 592 | {noreply, State1}; 593 | _ -> 594 | ok = osiris_writer:ack(LeaderPid, OffsetTimestamp), 595 | State = notify_offset_listeners(State1), 596 | {noreply, State} 597 | end. 598 | 599 | %%-------------------------------------------------------------------- 600 | %% @private 601 | %% @doc 602 | %% This function is called by a gen_server when it is about to 603 | %% terminate. It should be the opposite of Module:init/1 and do any 604 | %% necessary cleaning up. When it returns, the gen_server terminates 605 | %% with Reason. The return value is ignored. 606 | %% 607 | %% @spec terminate(Reason, State) -> void() 608 | %% @end 609 | %%-------------------------------------------------------------------- 610 | terminate(_Reason, undefined) -> 611 | %% if we crash in handle_continue we may end up here 612 | ok; 613 | terminate(Reason, #?MODULE{cfg = #cfg{name = Name, 614 | socket = Sock}, log = Log}) -> 615 | ?DEBUG_(Name, "terminating with ~w ", [Reason]), 616 | _ = ets:delete(osiris_reader_context_cache, self()), 617 | ok = osiris_log:close(Log), 618 | case Sock of 619 | undefined -> ok; 620 | _ -> 621 | ok = gen_tcp:close(Sock) 622 | end, 623 | ok. 624 | 625 | %%-------------------------------------------------------------------- 626 | %% @private 627 | %% @doc 628 | %% Convert process state when code is changed 629 | %% 630 | %% @spec code_change(OldVsn, State, Extra) -> {ok, NewState} 631 | %% @end 632 | %%-------------------------------------------------------------------- 633 | code_change(_OldVsn, State, _Extra) -> 634 | {ok, State}. 635 | 636 | format_status(#{state := #?MODULE{cfg = #cfg{name = Name, 637 | reference = ExtRef}, 638 | log = Log, 639 | parse_state = ParseState, 640 | offset_listeners = OffsetListeners, 641 | committed_chunk_id = CommittedOffset}} = Status) -> 642 | maps:update(state, 643 | #{name => Name, 644 | external_reference => ExtRef, 645 | has_parse_state => ParseState /= undefined, 646 | log => osiris_log:format_status(Log), 647 | num_offset_listeners => length(OffsetListeners), 648 | committed_offset => CommittedOffset 649 | }, 650 | Status); 651 | format_status(Status) -> 652 | %% Handle formatting the status when the server shut down before start-up, 653 | %% for example when the rpc call in `handle_continue/2' fails. 654 | Status. 655 | %%%=================================================================== 656 | %%% Internal functions 657 | %%%=================================================================== 658 | 659 | parse_chunk(<<>>, ParseState, Acc) -> 660 | {ParseState, lists:reverse(Acc)}; 661 | parse_chunk(<> = All, 678 | undefined, Acc) -> 679 | TotalSize = ?HEADER_SIZE_B + FSize + Size + TSize, 680 | <> = All, 681 | parse_chunk(Rem, undefined, [{{FirstOffset, Timestamp}, Chunk} | Acc]); 682 | parse_chunk(Bin, undefined, Acc) 683 | when byte_size(Bin) =< ?HEADER_SIZE_B -> 684 | {Bin, lists:reverse(Acc)}; 685 | parse_chunk(<> = All, 699 | undefined, Acc) -> 700 | {{{FirstOffset, Timestamp}, [All], FSize + Size + TSize - byte_size(Partial)}, 701 | lists:reverse(Acc)}; 702 | parse_chunk(Bin, PartialHeaderBin, Acc) 703 | when is_binary(PartialHeaderBin) -> 704 | %% slight inneficiency but partial headers should be relatively 705 | %% rare and fairly small - also ensures the header is always intact 706 | parse_chunk(<>, undefined, Acc); 707 | parse_chunk(Bin, {FirstOffsetTs, IOData, RemSize}, Acc) 708 | when byte_size(Bin) >= RemSize -> 709 | <> = Bin, 710 | parse_chunk(Rem, undefined, 711 | [{FirstOffsetTs, lists:reverse([Final | IOData])} | Acc]); 712 | parse_chunk(Bin, {FirstOffsetTs, IOData, RemSize}, Acc) -> 713 | %% there is not enough data to complete the partial chunk 714 | {{FirstOffsetTs, [Bin | IOData], RemSize - byte_size(Bin)}, 715 | lists:reverse(Acc)}. 716 | 717 | notify_offset_listeners(#?MODULE{cfg = #cfg{reference = Ref, 718 | event_formatter = EvtFmt}, 719 | committed_chunk_id = CommittedChId, 720 | log = Log, 721 | offset_listeners = L0} = State) -> 722 | Max = max_readable_chunk_id(Log), 723 | {Notify, L} = 724 | lists:partition(fun({_Pid, O, _}) -> O =< Max end, L0), 725 | _ = [begin 726 | Evt = 727 | %% the per offset listener event formatter takes precedence of 728 | %% the process scoped one 729 | wrap_osiris_event( 730 | select_formatter(Fmt, EvtFmt), 731 | {osiris_offset, Ref, CommittedChId}), 732 | P ! Evt 733 | end 734 | || {P, _, Fmt} <- Notify], 735 | State#?MODULE{offset_listeners = L}. 736 | 737 | max_readable_chunk_id(Log) -> 738 | min(osiris_log:committed_offset(Log), osiris_log:last_chunk_id(Log)). 739 | 740 | %% INTERNAL 741 | 742 | wrap_osiris_event(undefined, Evt) -> 743 | Evt; 744 | wrap_osiris_event({M, F, A}, Evt) -> 745 | apply(M, F, [Evt | A]). 746 | 747 | select_formatter(undefined, Fmt) -> 748 | Fmt; 749 | select_formatter(Fmt, _) -> 750 | Fmt. 751 | 752 | recv(tcp, Socket, Length, Timeout) -> 753 | gen_tcp:recv(Socket, Length, Timeout); 754 | recv(ssl, Socket, Length, Timeout) -> 755 | ssl:recv(Socket, Length, Timeout). 756 | 757 | setopts(tcp, Socket, Options) -> 758 | inet:setopts(Socket, Options); 759 | setopts(ssl, Socket, Options) -> 760 | ssl:setopts(Socket, Options). 761 | 762 | listener_opts(tcp) -> 763 | RcvBuf = application:get_env(osiris, replica_recbuf, ?DEF_REC_BUF), 764 | Buffer = application:get_env(osiris, replica_buffer, RcvBuf * 2), 765 | KeepAlive = application:get_env(osiris, replica_keepalive, false), 766 | ReuseAddr = application:get_env(osiris, replica_reuseaddr, true), 767 | Linger = application:get_env(osiris, replica_linger, true), 768 | LingerTimeout = application:get_env(osiris, replica_linger_timeout, 0), 769 | 770 | IPAddrFamily = osiris_util:get_replica_listener_inet_address_family(), 771 | 772 | [binary, 773 | IPAddrFamily, 774 | {reuseaddr, ReuseAddr}, 775 | {linger, {Linger, LingerTimeout}}, 776 | {backlog, 0}, 777 | {packet, raw}, 778 | {active, false}, 779 | {buffer, Buffer}, 780 | {recbuf, RcvBuf}, 781 | {keepalive, KeepAlive} 782 | ]; 783 | listener_opts(ssl) -> 784 | Opts = listener_opts(tcp), 785 | SslOptions = application:get_env(osiris, replication_server_ssl_options, []), 786 | Opts ++ SslOptions. 787 | 788 | listen(tcp, Port, Options) -> 789 | gen_tcp:listen(Port, Options); 790 | listen(ssl, Port, Options) -> 791 | ssl:listen(Port, Options). 792 | --------------------------------------------------------------------------------