├── .github └── workflows │ └── ci-workflow.yaml ├── .gitignore ├── .readthedocs.yaml ├── .vscode └── settings.json ├── CHANGELOG.rst ├── CONTRIBUTING.md ├── Dockerfile ├── LICENSE ├── Makefile ├── README.rst ├── allocator ├── alloc_state.go ├── alloc_state_test.go ├── allocator.go ├── allocator_key_space.go ├── allocator_key_space_test.go ├── allocator_test.go ├── announce.go ├── announce_test.go ├── benchmark_test.go ├── doc.go ├── item_state.go ├── item_state_test.go ├── scenarios_test.go ├── sparse_flow_network.go ├── sparse_flow_network_test.go └── sparse_push_relabel │ ├── push_relabel.go │ └── push_relabel_test.go ├── auth ├── auth.go └── auth_test.go ├── broker ├── append_api.go ├── append_api_test.go ├── append_flow_control.go ├── append_flow_control_test.go ├── append_fsm.go ├── append_fsm_test.go ├── client │ ├── append_service.go │ ├── append_service_test.go │ ├── append_service_unix.go │ ├── append_service_win.go │ ├── appender.go │ ├── appender_test.go │ ├── doc.go │ ├── list.go │ ├── list_test.go │ ├── reader.go │ ├── reader_test.go │ ├── retry_reader.go │ ├── retry_reader_test.go │ ├── route_cache.go │ └── route_cache_test.go ├── codecs │ ├── codecs.go │ └── zstandard_enable.go ├── doc.go ├── e2e_test.go ├── fragment │ ├── cover_set.go │ ├── cover_set_test.go │ ├── doc.go │ ├── fragment.go │ ├── index.go │ ├── index_test.go │ ├── persister.go │ ├── persister_test.go │ ├── spool.go │ ├── spool_test.go │ ├── spool_unix.go │ ├── spool_win.go │ ├── store_azure.go │ ├── store_fs.go │ ├── store_gcs.go │ ├── store_s3.go │ ├── stores.go │ └── stores_test.go ├── fragments_api.go ├── fragments_api_test.go ├── http_gateway │ ├── http_gateway.go │ └── http_gateway_test.go ├── journalspace │ ├── node.go │ └── node_test.go ├── key_space.go ├── key_space_test.go ├── list_apply_api.go ├── list_apply_api_test.go ├── pipeline.go ├── pipeline_test.go ├── protocol │ ├── auth.go │ ├── broker_spec_extensions.go │ ├── broker_spec_extensions_test.go │ ├── dispatcher.go │ ├── dispatcher_test.go │ ├── doc.go │ ├── endpoint.go │ ├── endpoint_test.go │ ├── ext │ │ ├── doc.go │ │ ├── header_extensions.go │ │ ├── header_extensions_test.go │ │ ├── route_extensions.go │ │ └── route_extensions_test.go │ ├── fragment_extensions.go │ ├── fragment_extensions_test.go │ ├── fragment_store.go │ ├── fragment_store_test.go │ ├── header_extensions.go │ ├── header_extensions_test.go │ ├── journal_spec_extensions.go │ ├── journal_spec_extensions_test.go │ ├── label_extensions.go │ ├── label_extensions_test.go │ ├── protocol.pb.go │ ├── protocol.pb.gw.go │ ├── protocol.proto │ ├── protocol_gateway.yaml │ ├── route_extensions.go │ ├── route_extensions_test.go │ ├── rpc_extensions.go │ ├── rpc_extensions_test.go │ └── validator.go ├── read_api.go ├── read_api_test.go ├── replica.go ├── replica_test.go ├── replicate_api.go ├── replicate_api_test.go ├── resolver.go ├── resolver_test.go ├── service.go ├── test_support_test.go └── teststub │ └── broker.go ├── brokertest ├── broker.go └── broker_test.go ├── cmd ├── gazctl │ ├── gazctlcmd │ │ ├── attach_uuids.go │ │ ├── edit_docs.go │ │ ├── editor │ │ │ └── editor.go │ │ ├── gazctl.go │ │ ├── journals_append.go │ │ ├── journals_apply.go │ │ ├── journals_edit.go │ │ ├── journals_fragments.go │ │ ├── journals_list.go │ │ ├── journals_prune.go │ │ ├── journals_read.go │ │ ├── journals_reset_head.go │ │ ├── journals_suspend.go │ │ ├── shards_apply.go │ │ ├── shards_edit.go │ │ ├── shards_list.go │ │ ├── shards_prune.go │ │ ├── shards_prune_test.go │ │ ├── shards_recover.go │ │ └── shards_unassign.go │ └── main.go └── gazette │ └── main.go ├── consumer ├── consumer_stub_test.go ├── interfaces.go ├── key_space.go ├── key_space_test.go ├── protocol │ ├── auth.go │ ├── checkpoint_extensions.go │ ├── checkpoint_extensions_test.go │ ├── doc.go │ ├── protocol.pb.go │ ├── protocol.pb.gw.go │ ├── protocol.proto │ ├── protocol_gateway.yaml │ ├── rpc_extensions.go │ ├── rpc_extensions_test.go │ ├── shard_spec_extensions.go │ └── shard_spec_extensions_test.go ├── recovery.go ├── recovery_test.go ├── recoverylog │ ├── doc.go │ ├── fsm.go │ ├── fsm_test.go │ ├── playback.go │ ├── playback_test.go │ ├── recorded_op.pb.go │ ├── recorded_op.proto │ ├── recorded_op_extensions.go │ ├── recorder.go │ ├── recorder_afero.go │ ├── recorder_test.go │ ├── segment.go │ └── segment_test.go ├── resolver.go ├── resolver_test.go ├── service.go ├── shard.go ├── shard_api.go ├── shard_api_test.go ├── shard_test.go ├── shardspace │ ├── shardspace.go │ └── shardspace_test.go ├── store-rocksdb │ ├── arena_iterator.cpp │ ├── arena_iterator.go │ ├── arena_iterator_test.go │ ├── hooked_env.cpp │ ├── hooked_env.go │ ├── recorder_rocksdb.go │ ├── recorder_rocksdb_test.go │ ├── store_rocksdb.go │ └── store_rocksdb_test.go ├── store-sqlite │ ├── CMakeLists.txt │ ├── doc.go │ ├── store.cpp │ ├── store.go │ ├── store.h │ ├── store_test.go │ ├── store_test_support.go │ └── testdata │ │ ├── test-attach-and-vacuum.sql │ │ ├── test-multi-db-txn.sql │ │ ├── test-simple.sql │ │ ├── test-sqllogic-full.sql │ │ ├── test-sqllogic.sql │ │ └── test-trim-and-rollback.sql ├── store_json_file.go ├── store_sql.go ├── store_sql_test.go ├── test_support_test.go ├── transaction.go └── transaction_test.go ├── consumertest ├── consumer.go └── consumer_test.go ├── docs ├── Makefile ├── _static │ ├── cmd-gazctl-attach-uuids.txt │ ├── cmd-gazctl-journals-append.txt │ ├── cmd-gazctl-journals-apply.txt │ ├── cmd-gazctl-journals-edit.txt │ ├── cmd-gazctl-journals-fragments.txt │ ├── cmd-gazctl-journals-list.txt │ ├── cmd-gazctl-journals-prune.txt │ ├── cmd-gazctl-journals-read.txt │ ├── cmd-gazctl-journals-reset-head.txt │ ├── cmd-gazctl-journals-suspend.txt │ ├── cmd-gazctl-print-config.txt │ ├── cmd-gazctl-shards-apply.txt │ ├── cmd-gazctl-shards-edit.txt │ ├── cmd-gazctl-shards-list.txt │ ├── cmd-gazctl-shards-prune.txt │ ├── cmd-gazctl.txt │ ├── cmd-gazette-print-config.txt │ ├── cmd-gazette-serve.txt │ └── logo_with_text.svg ├── architecture-design-goals-and-nongoals.rst ├── architecture-exactly-once.rst ├── architecture-operational-considerations.rst ├── architecture-transactional-appends.rst ├── brokers-concepts.rst ├── brokers-journalspecs.rst ├── brokers-tutorial-introduction.rst ├── changelog.rst ├── cncf_proposal.rst ├── conf.py ├── consumers-concepts.rst ├── consumers-shardspecs.rst ├── examples-bike-share.rst ├── examples-language-model.rst ├── examples-stream-sum.rst ├── examples.rst ├── index.rst ├── overview-build-and-test.rst ├── overview-intro.rst ├── overview-slides.rst ├── reference-api.rst ├── reference-gazctl.rst ├── reference-gazette.rst ├── requirements.txt └── table-of-contents.rst ├── etcdtest ├── etcd.go ├── etcd_default.go └── etcd_linux.go ├── examples ├── batch_download.py ├── bike-share │ ├── api.go │ ├── application.go │ ├── bike-share │ │ └── main.go │ └── sql_statements.go ├── doc.go ├── generate-zonemap-gke.sh ├── stream-sum │ ├── chunker │ │ └── chunker.go │ ├── stream_sum.go │ ├── stream_sum_test.go │ └── summer │ │ └── summer.go └── word-count │ ├── counter │ └── counter.go │ ├── word_count.go │ ├── word_count.pb.go │ ├── word_count.proto │ ├── word_count_test.go │ └── wordcountctl │ └── main.go ├── go.mod ├── go.sum ├── keyspace ├── doc.go ├── key_space.go ├── key_space_test.go ├── key_values.go └── key_values_test.go ├── kustomize ├── README.md ├── bases │ ├── broker │ │ ├── deployment.yaml │ │ ├── kustomization.yaml │ │ └── service.yaml │ ├── consumer │ │ ├── deployment.yaml │ │ ├── kustomization.yaml │ │ └── service.yaml │ ├── etcd │ │ ├── etcd-bootstrap.sh │ │ ├── etcd-lib.sh │ │ ├── etcd-shutdown.sh │ │ ├── kustomization.yaml │ │ ├── service.yaml │ │ └── statefulset.yaml │ ├── example-bike-share │ │ ├── apply-shard-specs.yaml │ │ ├── kustomization.yaml │ │ ├── patches.yaml │ │ ├── shard_specs.yaml │ │ ├── stream-rides.yaml │ │ └── stream_rides.sh │ ├── example-stream-sum │ │ ├── apply-shard-specs.yaml │ │ ├── chunker.yaml │ │ ├── generate-shards.sh │ │ ├── kustomization.yaml │ │ └── set-summer-name-label.yaml │ ├── example-word-count │ │ ├── apply-shard-specs.yaml │ │ ├── kustomization.yaml │ │ ├── patches.yaml │ │ ├── publish-docs.yaml │ │ ├── publish_docs.sh │ │ └── shard_specs.yaml │ ├── minio │ │ ├── create-bucket.sh │ │ ├── create-bucket.yaml │ │ ├── deployment.yaml │ │ ├── kustomization.yaml │ │ ├── service.yaml │ │ └── serviceaccount.yaml │ └── postgres │ │ ├── kustomization.yaml │ │ ├── service.yaml │ │ └── statefulset.yaml └── test │ ├── bases │ ├── crash-tester │ │ ├── kustomization.yaml │ │ ├── rbac.yaml │ │ ├── runner.yaml │ │ └── urkel.yaml │ └── environment │ │ ├── apply-journal-specs.yaml │ │ ├── examples.journalspace.yaml │ │ ├── kustomization.yaml │ │ ├── minio_config │ │ ├── minio_credentials │ │ ├── postgres_tables.sql │ │ └── random_node_zonemap.sh │ ├── deploy-bike-share │ ├── kustomization.yaml │ └── namespace.yaml │ ├── deploy-stream-sum-with-crash-tests │ ├── kustomization.yaml │ └── patches.yaml │ ├── deploy-stream-sum │ ├── kustomization.yaml │ └── namespace.yaml │ ├── deploy-word-count │ ├── kustomization.yaml │ └── namespace.yaml │ └── run-with-local-registry │ ├── always-pull.yaml │ └── kustomization.yaml ├── labels └── labels.go ├── mainboilerplate ├── client.go ├── cmdreg.go ├── config.go ├── diagnostics.go ├── etcd.go ├── logging.go ├── runconsumer │ └── run_consumer.go └── service.go ├── message ├── csv_framing.go ├── csv_framing_test.go ├── fixed_framing.go ├── fixed_framing_test.go ├── interfaces.go ├── json_framing.go ├── json_framing_test.go ├── publisher.go ├── publisher_test.go ├── reader.go ├── reader_test.go ├── routines.go ├── routines_test.go ├── sequencer.go ├── sequencer_test.go ├── uuid.go └── uuid_test.go ├── server └── server.go ├── task └── group.go └── test └── integration ├── crash_test.go └── partition_test.go /.github/workflows/ci-workflow.yaml: -------------------------------------------------------------------------------- 1 | name: Gazette Continuous Integration 2 | 3 | on: 4 | push: 5 | branches: [master] 6 | pull_request: 7 | branches: [master] 8 | 9 | jobs: 10 | build: 11 | runs-on: ubuntu-24.04 12 | steps: 13 | - uses: actions/checkout@v4 14 | with: 15 | fetch-depth: 0 # Full history. 16 | 17 | - uses: docker/setup-qemu-action@v3 18 | - uses: docker/setup-buildx-action@v3 19 | - name: Login to GitHub container registry 20 | run: | 21 | echo "${{ secrets.GITHUB_TOKEN }}" | \ 22 | docker login --username ${{ github.actor }} --password-stdin ghcr.io 23 | 24 | - uses: actions/setup-go@v4 25 | - run: sudo apt install -y protobuf-compiler librocksdb-dev libsqlite3-dev etcd-server 26 | - run: echo "VERSION=$(git describe --dirty --tags)" >> "$GITHUB_ENV" 27 | 28 | - run: make go-test-ci 29 | - run: make go-build go-build-arm64 30 | 31 | - uses: docker/build-push-action@v5 32 | with: 33 | context: .build 34 | file: Dockerfile 35 | target: broker 36 | platforms: linux/amd64,linux/arm64 37 | tags: "ghcr.io/gazette/broker:${{ env.VERSION }}" 38 | push: true 39 | 40 | - uses: docker/build-push-action@v5 41 | with: 42 | context: .build 43 | file: Dockerfile 44 | target: examples 45 | platforms: linux/amd64 46 | tags: "ghcr.io/gazette/examples:${{ env.VERSION }}" 47 | push: true -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .build 2 | .build-ci 3 | 4 | docs/_build 5 | 6 | .idea 7 | -------------------------------------------------------------------------------- /.readthedocs.yaml: -------------------------------------------------------------------------------- 1 | # Read the Docs configuration file 2 | # See https://docs.readthedocs.io/en/stable/config-file/v2.html for details 3 | 4 | # Required 5 | version: 2 6 | 7 | # Build documentation in the docs/ directory with Sphinx 8 | sphinx: 9 | configuration: docs/conf.py 10 | 11 | # Set the version of Python and requirements required to build docs. 12 | python: 13 | version: 3.7 14 | install: 15 | - requirements: docs/requirements.txt 16 | -------------------------------------------------------------------------------- /.vscode/settings.json: -------------------------------------------------------------------------------- 1 | { 2 | "go.buildTags": "libsqlite3" 3 | } -------------------------------------------------------------------------------- /CONTRIBUTING.md: -------------------------------------------------------------------------------- 1 | Contributing to Gazette 2 | 3 | When you contribute code, you affirm that the contribution is your original work and that you license the work to the project under the project's open source license. Whether or not you state this explicitly, by submitting any copyrighted material via pull request, email, or other means you agree to license the material under the project's open source license and warrant that you have the legal authority to do so. 4 | -------------------------------------------------------------------------------- /Dockerfile: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Gazette broker image. 3 | 4 | FROM ubuntu:24.04 AS broker 5 | 6 | ARG TARGETARCH 7 | 8 | RUN apt-get update -y \ 9 | && apt-get upgrade -y \ 10 | && apt-get install --no-install-recommends -y \ 11 | ca-certificates \ 12 | curl \ 13 | && rm -rf /var/lib/apt/lists/* 14 | 15 | COPY ${TARGETARCH}/gazette ${TARGETARCH}/gazctl /usr/local/bin 16 | 17 | # Run as non-privileged "gazette" user. 18 | RUN useradd gazette --create-home --shell /usr/sbin/nologin 19 | USER gazette 20 | WORKDIR /home/gazette 21 | 22 | ################################################################################ 23 | # Gazette examples image. 24 | 25 | FROM ubuntu:24.04 AS examples 26 | 27 | ARG TARGETARCH 28 | 29 | RUN apt-get update -y \ 30 | && apt-get upgrade -y \ 31 | && apt-get install --no-install-recommends -y \ 32 | ca-certificates \ 33 | curl \ 34 | librocksdb8.9 \ 35 | libsqlite3-0 \ 36 | && rm -rf /var/lib/apt/lists/* 37 | 38 | # Copy binaries to the image. 39 | COPY \ 40 | ${TARGETARCH}/bike-share \ 41 | ${TARGETARCH}/chunker \ 42 | ${TARGETARCH}/counter \ 43 | ${TARGETARCH}/gazctl \ 44 | ${TARGETARCH}/integration.test \ 45 | ${TARGETARCH}/summer \ 46 | ${TARGETARCH}/wordcountctl \ 47 | /usr/local/bin/ 48 | 49 | # Run as non-privileged "gazette" user. 50 | RUN useradd gazette --create-home --shell /usr/sbin/nologin 51 | USER gazette 52 | WORKDIR /home/gazette 53 | 54 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2017 LiveRamp 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | -------------------------------------------------------------------------------- /README.rst: -------------------------------------------------------------------------------- 1 | 2 | .. image:: docs/_static/logo_with_text.svg 3 | :alt: Gazette Logo 4 | 5 | 6 | .. image:: https://github.com/gazette/core/workflows/Gazette%20Continuous%20Integration/badge.svg 7 | :target: https://github.com/gazette/core/actions?query=workflow%3A%22Gazette+Continuous+Integration%22 8 | :alt: Gazette Continuous Integration 9 | .. image:: https://godoc.org/go.gazette.dev/core?status.svg 10 | :target: https://godoc.org/go.gazette.dev/core 11 | :alt: GoDoc 12 | .. image:: https://img.shields.io/badge/slack-@gazette/dev-yellow.svg?logo=slack 13 | :target: https://join.slack.com/t/gazette-dev/shared_invite/enQtNjQxMzgyNTEzNzk1LTU0ZjZlZmY5ODdkOTEzZDQzZWU5OTk3ZTgyNjY1ZDE1M2U1ZTViMWQxMThiMjU1N2MwOTlhMmVjYjEzMjEwMGQ 14 | :alt: Slack 15 | .. image:: https://goreportcard.com/badge/github.com/gazette/core 16 | :target: https://goreportcard.com/report/github.com/gazette/core 17 | :alt: Go Report Card 18 | 19 | 20 | Overview 21 | ========= 22 | 23 | Gazette makes it easy to build platforms that flexibly mix *SQL*, *batch*, 24 | and *millisecond-latency streaming* processing paradigms. It enables teams, 25 | applications, and analysts to work from a common catalog of data in the way 26 | that's most convenient **to them**. Gazette's core abstraction is a "journal" 27 | -- a streaming append log that's represented using regular files in a BLOB 28 | store (i.e., S3). 29 | 30 | The magic of this representation is that journals are simultaneously a 31 | low-latency data stream *and* a collection of immutable, organized files 32 | in cloud storage (aka, a data lake) -- a collection which can be directly 33 | integrated into familiar processing tools and SQL engines. 34 | 35 | Atop the journal *broker service*, Gazette offers a powerful *consumers 36 | framework* for building streaming applications in Go. Gazette has served 37 | production use cases for nearly five years, with deployments scaled to 38 | millions of streamed records per second. 39 | 40 | Where to Start 41 | =============== 42 | 43 | - Official documentation_, with tutorials and examples. 44 | - API Godocs_. 45 | 46 | .. _documentation: https://gazette.dev 47 | .. _Godocs: https://godoc.org/go.gazette.dev/core 48 | 49 | -------------------------------------------------------------------------------- /allocator/doc.go: -------------------------------------------------------------------------------- 1 | // Package allocator implements a distributed algorithm for assigning a number 2 | // of "Items" across a number of "Members", where each Member runs an instance of 3 | // the Allocator. Items and Members may come and go over time; each may have 4 | // constraints on desired replication and assignment limits which must be 5 | // satisfied, and replicas may be placed across distinct failure Zones. 6 | // Allocator coordinates through Etcd, and uses a greedy, incremental maximum- 7 | // flow solver to quickly determine minimal re-Assignments which best balance 8 | // Items across Members (subject to constraints). 9 | package allocator 10 | 11 | import ( 12 | "github.com/prometheus/client_golang/prometheus" 13 | "github.com/prometheus/client_golang/prometheus/promauto" 14 | ) 15 | 16 | var ( 17 | allocatorAssignmentAddedTotal = promauto.NewCounter(prometheus.CounterOpts{ 18 | Name: "gazette_allocator_assignment_added_total", 19 | Help: "Cumulative number of item / member assignments added by the allocator.", 20 | }) 21 | allocatorAssignmentPackedTotal = promauto.NewCounter(prometheus.CounterOpts{ 22 | Name: "gazette_allocator_assignment_packed_total", 23 | Help: "Cumulative number of item / member assignments packed by the allocator.", 24 | }) 25 | allocatorAssignmentRemovedTotal = promauto.NewCounter(prometheus.CounterOpts{ 26 | Name: "gazette_allocator_assignment_removed_total", 27 | Help: "Cumulative number of item / member assignments removed by the allocator.", 28 | }) 29 | allocatorConvergeTotal = promauto.NewCounter(prometheus.CounterOpts{ 30 | Name: "gazette_allocator_converge_total", 31 | Help: "Cumulative number of converge iterations.", 32 | }) 33 | allocatorMaxFlowRuntimeSeconds = promauto.NewHistogram(prometheus.HistogramOpts{ 34 | Name: "gazette_allocator_max_flow_runtime_seconds", 35 | Help: "Duration required to re-solve for maximum assignment.", 36 | }) 37 | allocatorNumItemSlots = promauto.NewGauge(prometheus.GaugeOpts{ 38 | Name: "gazette_allocator_desired_replication_slots", 39 | Help: "Number of desired item replication slots summed across all items.", 40 | }) 41 | allocatorNumItems = promauto.NewGauge(prometheus.GaugeOpts{ 42 | Name: "gazette_allocator_items", 43 | Help: "Number of items known to the allocator.", 44 | }) 45 | allocatorNumMembers = promauto.NewGauge(prometheus.GaugeOpts{ 46 | Name: "gazette_allocator_members", 47 | Help: "Number of members known to the allocator.", 48 | }) 49 | ) 50 | -------------------------------------------------------------------------------- /auth/auth_test.go: -------------------------------------------------------------------------------- 1 | package auth_test 2 | 3 | import ( 4 | "context" 5 | "testing" 6 | "time" 7 | 8 | "github.com/stretchr/testify/require" 9 | "go.gazette.dev/core/auth" 10 | pb "go.gazette.dev/core/broker/protocol" 11 | "google.golang.org/grpc/metadata" 12 | ) 13 | 14 | func TestKeyedAuthCases(t *testing.T) { 15 | ka1, err := auth.NewKeyedAuth("c2VjcmV0,b3RoZXI=") 16 | require.NoError(t, err) 17 | ka2, err := auth.NewKeyedAuth("b3RoZXI=,c2VjcmV0") 18 | require.NoError(t, err) 19 | kaM, err := auth.NewKeyedAuth("YXNkZg==,AA==") 20 | require.NoError(t, err) 21 | 22 | // Authorize with one KeyedAuth... 23 | ctx, err := ka1.Authorize(context.Background(), 24 | pb.Claims{ 25 | Capability: pb.Capability_APPEND | pb.Capability_LIST, 26 | Selector: pb.MustLabelSelector("hi=there"), 27 | }, time.Hour) 28 | require.NoError(t, err) 29 | 30 | var md, _ = metadata.FromOutgoingContext(ctx) 31 | ctx = metadata.NewIncomingContext(ctx, md) 32 | 33 | // ...and verify with the other. 34 | _, cancel, claims, err := ka2.Verify(ctx, pb.Capability_APPEND) 35 | require.NoError(t, err) 36 | cancel() 37 | require.Equal(t, pb.MustLabelSelector("hi=there"), claims.Selector) 38 | 39 | // Unless the capability doesn't match. 40 | _, _, _, err = ka2.Verify(ctx, pb.Capability_REPLICATE) 41 | require.EqualError(t, err, 42 | "rpc error: code = Unauthenticated desc = authorization is missing required REPLICATE capability") 43 | 44 | // A KeyedAuth with a diferent key rejects it. 45 | _, _, _, err = kaM.Verify(ctx, pb.Capability_APPEND) 46 | require.EqualError(t, err, 47 | "rpc error: code = Unauthenticated desc = verifying Authorization: token signature is invalid: signature is invalid") 48 | 49 | // A KeyedAuth that allows pass-through will accept a request without a token. 50 | _, cancel, claims, err = kaM.Verify(context.Background(), pb.Capability_READ) 51 | require.NoError(t, err) 52 | cancel() 53 | require.Equal(t, pb.MustLabelSelector(""), claims.Selector) 54 | } 55 | -------------------------------------------------------------------------------- /broker/client/append_service_unix.go: -------------------------------------------------------------------------------- 1 | // +build !windows 2 | 3 | package client 4 | 5 | import ( 6 | "bufio" 7 | "io/ioutil" 8 | "os" 9 | ) 10 | 11 | // newAppendBuffer creates and returns a temporary file which has already had its 12 | // one-and-only hard link removed from the file system. So long as the *os.File 13 | // remains open, the OS will defer collecting the allocated inode and reclaiming 14 | // disk resources, but the file becomes unaddressable and its resources released 15 | // to the OS after the os.File is garbage-collected (such that the runtime 16 | // finalizer calls Close on our behalf). 17 | var newAppendBuffer = func() (*appendBuffer, error) { 18 | if f, err := ioutil.TempFile("", "gazette-append"); err != nil { 19 | return nil, err 20 | } else if err = os.Remove(f.Name()); err != nil { 21 | return nil, err 22 | } else { 23 | var fb = &appendBuffer{file: f} 24 | fb.buf = bufio.NewWriterSize(fb, appendBufferSize) 25 | return fb, nil 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /broker/client/append_service_win.go: -------------------------------------------------------------------------------- 1 | // +build windows 2 | 3 | package client 4 | 5 | import ( 6 | "bufio" 7 | "io/ioutil" 8 | "os" 9 | "runtime" 10 | 11 | log "github.com/sirupsen/logrus" 12 | ) 13 | 14 | // newAppendBuffer creates and returns a temporary file which also has a configured 15 | // runtime finalizer that will remove the file from the file system. This 16 | // behavior is used because Windows does not support removing the one-and-only 17 | // hard link of a currently-open file. 18 | var newAppendBuffer = func() (*appendBuffer, error) { 19 | if f, err := ioutil.TempFile("", "gazette-append"); err != nil { 20 | return nil, err 21 | } else { 22 | runtime.SetFinalizer(f, removeFileFinalizer) 23 | var fb = &appendBuffer{file: f} 24 | fb.buf = bufio.NewWriterSize(fb, appendBufferSize) 25 | return fb, nil 26 | } 27 | } 28 | 29 | func removeFileFinalizer(f *os.File) { 30 | if err := f.Close(); err != nil { 31 | log.WithFields(log.Fields{"name": f.Name(), "err": err}).Error("failed to Close file in finalizer") 32 | } 33 | if err := os.Remove(f.Name()); err != nil { 34 | log.WithFields(log.Fields{"name": f.Name(), "err": err}).Error("failed to Remove file in finalizer") 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /broker/client/route_cache.go: -------------------------------------------------------------------------------- 1 | package client 2 | 3 | import ( 4 | "context" 5 | "time" 6 | 7 | "github.com/hashicorp/golang-lru" 8 | pb "go.gazette.dev/core/broker/protocol" 9 | ) 10 | 11 | // RouteCache caches observed Routes for items (eg, Journals, or Shards). 12 | // It implements the protocol.DispatchRouter interface, and where a cached 13 | // Route of an item is available, it enables applications to dispatch RPCs 14 | // directly to the most appropriate broker or consumer process. This reduces 15 | // the overall number of network hops, and especially the number of hops 16 | // crossing availability zones (which often cost more). 17 | // 18 | // For example, RouteCache can direct an application to a broker in its same 19 | // availability zone which is replicating a desired journal, and to which a 20 | // long-lived Read RPC can be dispatched. 21 | // 22 | // // Adapt a JournalClient to a RoutedJournalClient by using a RouteCache. 23 | // var jc protocol.JournalClient 24 | // var rjc = protocol.NewRoutedJournalClient(jc, NewRouteCache(256, time.Hour)) 25 | // 26 | type RouteCache struct { 27 | cache lru.Cache 28 | ttl time.Duration 29 | } 30 | 31 | // NewRouteCache returns a RouteCache of the given size (which must be > 0) 32 | // and caching Duration. 33 | func NewRouteCache(size int, ttl time.Duration) *RouteCache { 34 | var cache, err = lru.New(size) 35 | if err != nil { 36 | panic(err.Error()) // Only errors on size <= 0. 37 | } 38 | return &RouteCache{ 39 | cache: *cache, 40 | ttl: ttl, 41 | } 42 | } 43 | 44 | // UpdateRoute caches the provided Route for the item, or invalidates it if 45 | // the route is nil or empty. 46 | func (rc *RouteCache) UpdateRoute(item string, route *pb.Route) { 47 | if route == nil || len(route.Members) == 0 { 48 | rc.cache.Remove(item) 49 | } else { 50 | var cr = cachedRoute{ 51 | route: route.Copy(), 52 | at: timeNow(), 53 | } 54 | rc.cache.Add(item, cr) 55 | } 56 | } 57 | 58 | // Route queries for a cached Route of the item. 59 | func (rc *RouteCache) Route(_ context.Context, item string) pb.Route { 60 | if v, ok := rc.cache.Get(item); ok { 61 | // If the TTL has elapsed, treat as a cache miss and remove. 62 | if cr := v.(cachedRoute); cr.at.Add(rc.ttl).Before(timeNow()) { 63 | rc.cache.Remove(item) 64 | } else { 65 | return cr.route 66 | } 67 | } 68 | return pb.Route{Primary: -1} 69 | } 70 | 71 | // IsNoopRouter returns false. 72 | func (rc *RouteCache) IsNoopRouter() bool { return false } 73 | 74 | type cachedRoute struct { 75 | route pb.Route 76 | at time.Time 77 | } 78 | 79 | var timeNow = time.Now 80 | -------------------------------------------------------------------------------- /broker/client/route_cache_test.go: -------------------------------------------------------------------------------- 1 | package client 2 | 3 | import ( 4 | "context" 5 | "time" 6 | 7 | pb "go.gazette.dev/core/broker/protocol" 8 | gc "gopkg.in/check.v1" 9 | ) 10 | 11 | type RouteCacheSuite struct{} 12 | 13 | func (s *RouteCacheSuite) TestCachingCases(c *gc.C) { 14 | defer func(f func() time.Time) { timeNow = f }(timeNow) 15 | 16 | var fixedtime int64 = 1000 17 | timeNow = func() time.Time { return time.Unix(fixedtime, 0) } 18 | 19 | var ctx = context.Background() 20 | var rc = NewRouteCache(3, time.Minute) 21 | 22 | for _, s := range []string{"A", "B", "C", "D"} { 23 | rc.UpdateRoute(s, buildRouteFixture(s)) 24 | } 25 | c.Check(rc.cache.Len(), gc.Equals, 3) 26 | 27 | // Case: Cached routes are returned. 28 | c.Check(rc.Route(ctx, "D"), gc.DeepEquals, *buildRouteFixture("D")) // Hit. 29 | 30 | // Case: Routes which have fallen out of cache are not. 31 | c.Check(rc.Route(ctx, "A"), gc.DeepEquals, pb.Route{Primary: -1}) // Miss. 32 | 33 | // Case: Nil or empty routes invalidate the cache. 34 | rc.UpdateRoute("C", nil) 35 | rc.UpdateRoute("C", new(pb.Route)) 36 | c.Check(rc.Route(ctx, "C"), gc.DeepEquals, pb.Route{Primary: -1}) // Miss. 37 | 38 | // Case: TTLs are enforced. 39 | fixedtime += 31 40 | rc.UpdateRoute("B", buildRouteFixture("B")) 41 | 42 | // Precondition: both B and D are cached. 43 | c.Check(rc.Route(ctx, "B"), gc.DeepEquals, *buildRouteFixture("B")) 44 | c.Check(rc.Route(ctx, "D"), gc.DeepEquals, *buildRouteFixture("D")) 45 | 46 | fixedtime += 30 47 | 48 | // TTL for D has elapsed, but not for B. 49 | c.Check(rc.Route(ctx, "B"), gc.DeepEquals, *buildRouteFixture("B")) 50 | c.Check(rc.Route(ctx, "D"), gc.DeepEquals, pb.Route{Primary: -1}) 51 | } 52 | 53 | func buildRouteFixture(id string) *pb.Route { 54 | return &pb.Route{ 55 | Primary: 0, 56 | Members: []pb.ProcessSpec_ID{{Zone: id, Suffix: id}}, 57 | } 58 | } 59 | 60 | var _ = gc.Suite(&RouteCacheSuite{}) 61 | -------------------------------------------------------------------------------- /broker/codecs/codecs.go: -------------------------------------------------------------------------------- 1 | package codecs 2 | 3 | import ( 4 | "fmt" 5 | "io" 6 | "io/ioutil" 7 | 8 | "github.com/golang/snappy" 9 | "github.com/klauspost/compress/gzip" 10 | pb "go.gazette.dev/core/broker/protocol" 11 | ) 12 | 13 | // Decompressor is a ReadCloser where Close closes and releases Decompressor 14 | // state, but does not Close or affect the underlying Reader. 15 | type Decompressor io.ReadCloser 16 | 17 | // Compressor is a WriteCloser where Close closes and releases Compressor 18 | // state, potentially flushing final content to the underlying Writer, 19 | // but does not Close or otherwise affect the underlying Writer. 20 | type Compressor io.WriteCloser 21 | 22 | // NewCodecReader returns a Decompressor of the Reader encoded with CompressionCodec. 23 | func NewCodecReader(r io.Reader, codec pb.CompressionCodec) (Decompressor, error) { 24 | switch codec { 25 | case pb.CompressionCodec_NONE, pb.CompressionCodec_GZIP_OFFLOAD_DECOMPRESSION: 26 | return ioutil.NopCloser(r), nil 27 | case pb.CompressionCodec_GZIP: 28 | return gzip.NewReader(r) 29 | case pb.CompressionCodec_SNAPPY: 30 | return ioutil.NopCloser(snappy.NewReader(r)), nil 31 | case pb.CompressionCodec_ZSTANDARD: 32 | return zstdNewReader(r) 33 | default: 34 | return nil, fmt.Errorf("unsupported codec %s", codec.String()) 35 | } 36 | } 37 | 38 | // NewCodecWriter returns a Compressor wrapping the Writer encoding with CompressionCodec. 39 | func NewCodecWriter(w io.Writer, codec pb.CompressionCodec) (Compressor, error) { 40 | switch codec { 41 | case pb.CompressionCodec_NONE: 42 | return nopWriteCloser{w}, nil 43 | case pb.CompressionCodec_GZIP, pb.CompressionCodec_GZIP_OFFLOAD_DECOMPRESSION: 44 | return gzip.NewWriter(w), nil 45 | case pb.CompressionCodec_SNAPPY: 46 | return snappy.NewBufferedWriter(w), nil 47 | case pb.CompressionCodec_ZSTANDARD: 48 | return zstdNewWriter(w) 49 | default: 50 | return nil, fmt.Errorf("unsupported codec %s", codec.String()) 51 | } 52 | } 53 | 54 | type nopWriteCloser struct{ io.Writer } 55 | 56 | func (nopWriteCloser) Close() error { return nil } 57 | 58 | var ( 59 | zstdNewReader = func(io.Reader) (io.ReadCloser, error) { 60 | return nil, fmt.Errorf("ZSTANDARD was not enabled at compile time") 61 | } 62 | zstdNewWriter = func(io.Writer) (io.WriteCloser, error) { 63 | return nil, fmt.Errorf("ZSTANDARD was not enabled at compile time") 64 | } 65 | ) 66 | -------------------------------------------------------------------------------- /broker/codecs/zstandard_enable.go: -------------------------------------------------------------------------------- 1 | // +build !nozstd 2 | 3 | package codecs 4 | 5 | import ( 6 | "io" 7 | 8 | "github.com/DataDog/zstd" 9 | ) 10 | 11 | func init() { 12 | zstdNewReader = func(r io.Reader) (io.ReadCloser, error) { return zstd.NewReader(r), nil } 13 | zstdNewWriter = func(w io.Writer) (io.WriteCloser, error) { return zstd.NewWriter(w), nil } 14 | } 15 | -------------------------------------------------------------------------------- /broker/doc.go: -------------------------------------------------------------------------------- 1 | // Package broker implements the broker runtime and protocol.JournalServer APIs 2 | // (Read, Append, Replicate, List, Apply). Its `pipeline` type manages the 3 | // coordination of write transactions, and `resolver` the mapping of journal 4 | // names to Routes of responsible brokers. `replica` is a top-level collection 5 | // of runtime state and maintenance tasks associated with the processing of a 6 | // journal. gRPC proxy support is also implemented by this package. 7 | package broker 8 | 9 | import ( 10 | "github.com/prometheus/client_golang/prometheus" 11 | "github.com/prometheus/client_golang/prometheus/promauto" 12 | ) 13 | 14 | var ( 15 | journalServerStarted = promauto.NewCounterVec(prometheus.CounterOpts{ 16 | Name: "gazette_journal_server_started_totals", 17 | Help: "Total number of started JournalServer RPC invocations, by operation.", 18 | }, []string{"operation"}) 19 | journalServerCompleted = promauto.NewCounterVec(prometheus.CounterOpts{ 20 | Name: "gazette_journal_server_completed_totals", 21 | Help: "Total number of completed JournalServer RPC invocations, by operation & response status", 22 | }, []string{"operation", "status"}) 23 | writeHeadGauge = promauto.NewGaugeVec(prometheus.GaugeOpts{ 24 | Name: "gazette_write_head", 25 | Help: "Current write head of the journal (i.e., next byte offset to be written).", 26 | }, []string{"journal"}) 27 | ) 28 | -------------------------------------------------------------------------------- /broker/fragment/fragment.go: -------------------------------------------------------------------------------- 1 | package fragment 2 | 3 | import ( 4 | "io" 5 | 6 | "go.gazette.dev/core/broker/protocol" 7 | ) 8 | 9 | // Fragment wraps the protocol.Fragment type with a nil-able backing local File. 10 | type Fragment struct { 11 | protocol.Fragment 12 | // Local uncompressed file of the Fragment, or nil iff the Fragment is remote. 13 | File File 14 | } 15 | 16 | // File is the subset of os.File used in backing Fragments with local files. 17 | type File interface { 18 | io.ReaderAt 19 | io.Seeker 20 | io.WriterAt 21 | io.Writer 22 | io.Closer 23 | } 24 | -------------------------------------------------------------------------------- /broker/fragment/spool_unix.go: -------------------------------------------------------------------------------- 1 | //go:build !windows 2 | 3 | package fragment 4 | 5 | import ( 6 | "os" 7 | ) 8 | 9 | // newSpoolFile creates and returns a temporary file which has already had its 10 | // one-and-only hard link removed from the file system. So long as the *os.File 11 | // remains open, the OS will defer collecting the allocated inode and reclaiming 12 | // disk resources, but the file becomes unaddressable and its resources released 13 | // to the OS after an explicit call to Close, or if the os.File is garbage- 14 | // collected (such that the runtime finalizer calls Close on our behalf). 15 | var newSpoolFile = func() (File, error) { 16 | if f, err := os.CreateTemp("", "spool"); err != nil { 17 | return nil, err 18 | } else { 19 | return f, os.Remove(f.Name()) 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /broker/fragment/spool_win.go: -------------------------------------------------------------------------------- 1 | //go:build windows 2 | 3 | package fragment 4 | 5 | import ( 6 | "os" 7 | "runtime" 8 | 9 | log "github.com/sirupsen/logrus" 10 | ) 11 | 12 | // newSpoolFile creates and returns a temporary file which also has a configured 13 | // runtime finalizer which will remove the file from the file system. This 14 | // behavior is used because Windows does not support removing the one-and-only 15 | // hard link of a currently-open file. 16 | var newSpoolFile = func() (File, error) { 17 | if f, err := os.CreateTemp("", "spool"); err != nil { 18 | return nil, err 19 | } else { 20 | runtime.SetFinalizer(f, removeFileFinalizer) 21 | return f, nil 22 | } 23 | } 24 | 25 | func removeFileFinalizer(f *os.File) { 26 | if err := f.Close(); err != nil { 27 | log.WithFields(log.Fields{"name": f.Name(), "err": err}).Error("failed to Close file in finalizer") 28 | } 29 | if err := os.Remove(f.Name()); err != nil { 30 | log.WithFields(log.Fields{"name": f.Name(), "err": err}).Error("failed to Remove file in finalizer") 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /broker/key_space_test.go: -------------------------------------------------------------------------------- 1 | package broker 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/stretchr/testify/require" 7 | "go.gazette.dev/core/allocator" 8 | pb "go.gazette.dev/core/broker/protocol" 9 | "go.gazette.dev/core/keyspace" 10 | ) 11 | 12 | func TestJournalConsistencyCases(t *testing.T) { 13 | var routes [3]pb.Route 14 | var assignments keyspace.KeyValues 15 | 16 | for i := range routes { 17 | routes[i].Primary = 0 18 | routes[i].Members = []pb.ProcessSpec_ID{ 19 | {Zone: "zone/a", Suffix: "member/1"}, 20 | {Zone: "zone/a", Suffix: "member/3"}, 21 | {Zone: "zone/b", Suffix: "member/2"}, 22 | } 23 | assignments = append(assignments, keyspace.KeyValue{ 24 | Decoded: allocator.Assignment{ 25 | AssignmentValue: &routes[i], 26 | MemberSuffix: routes[0].Members[i].Suffix, 27 | MemberZone: routes[0].Members[i].Zone, 28 | Slot: i, 29 | }, 30 | }) 31 | } 32 | 33 | // Case: all assignments match. 34 | require.True(t, JournalIsConsistent(allocator.Item{}, keyspace.KeyValue{}, assignments)) 35 | // Case: vary indicated primary of Route. 36 | routes[0].Primary = 1 37 | require.False(t, JournalIsConsistent(allocator.Item{}, keyspace.KeyValue{}, assignments)) 38 | routes[0].Primary = 0 39 | // Case: vary members. 40 | routes[1].Members = append(routes[1].Members, pb.ProcessSpec_ID{Zone: "zone/b", Suffix: "member/4"}) 41 | require.False(t, JournalIsConsistent(allocator.Item{}, keyspace.KeyValue{}, assignments)) 42 | } 43 | -------------------------------------------------------------------------------- /broker/protocol/broker_spec_extensions.go: -------------------------------------------------------------------------------- 1 | package protocol 2 | 3 | // Validate returns an error if the ProcessSpec_ID is not well-formed. 4 | func (m ProcessSpec_ID) Validate() error { 5 | if err := ValidateToken(m.Zone, TokenSymbols, minZoneLen, maxZoneLen); err != nil { 6 | return ExtendContext(err, "Zone") 7 | } else if err := ValidateToken(m.Suffix, TokenSymbols, minBrokerSuffixLen, maxBrokerSuffixLen); err != nil { 8 | return ExtendContext(err, "Suffix") 9 | } 10 | return nil 11 | } 12 | 13 | // Less returns whether the ProcessSpec_ID is less than the argument 14 | // BrokerSpec_ID, under (Zone, Suffix) ordering. 15 | func (m ProcessSpec_ID) Less(other ProcessSpec_ID) bool { 16 | if m.Zone != other.Zone { 17 | return m.Zone < other.Zone 18 | } 19 | return m.Suffix < other.Suffix 20 | } 21 | 22 | // Validate returns an error if the ProcessSpec is not well-formed. 23 | func (m *ProcessSpec) Validate() error { 24 | if err := m.Id.Validate(); err != nil { 25 | return ExtendContext(err, "Id") 26 | } else if err = m.Endpoint.Validate(); err != nil { 27 | return ExtendContext(err, "Endpoint") 28 | } 29 | return nil 30 | } 31 | 32 | // Validate returns an error if the BrokerSpec is not well-formed. 33 | func (m *BrokerSpec) Validate() error { 34 | if err := m.ProcessSpec.Validate(); err != nil { 35 | return err 36 | } else if m.JournalLimit > maxBrokerJournalLimit { 37 | return NewValidationError("invalid JournalLimit (%d; expected 0 <= JournalLimit <= %d)", 38 | m.JournalLimit, maxBrokerJournalLimit) 39 | } 40 | return nil 41 | } 42 | 43 | // MarshalString returns the marshaled encoding of the JournalSpec as a string. 44 | func (m *BrokerSpec) MarshalString() string { 45 | var d, err = m.Marshal() 46 | if err != nil { 47 | panic(err.Error()) // Cannot happen, as we use no custom marshalling. 48 | } 49 | return string(d) 50 | } 51 | 52 | // ZeroLimit zeros the BrokerSpec JournalLimit. 53 | func (m *BrokerSpec) ZeroLimit() { m.JournalLimit = 0 } 54 | 55 | // v3_allocator.MemberValue implementation. 56 | func (m *BrokerSpec) ItemLimit() int { return int(m.JournalLimit) } 57 | 58 | const ( 59 | minZoneLen = 1 60 | maxZoneLen = 32 61 | minBrokerSuffixLen = 4 62 | maxBrokerSuffixLen = 128 63 | maxBrokerJournalLimit = 1 << 17 64 | ) 65 | -------------------------------------------------------------------------------- /broker/protocol/broker_spec_extensions_test.go: -------------------------------------------------------------------------------- 1 | package protocol 2 | 3 | import ( 4 | gc "gopkg.in/check.v1" 5 | ) 6 | 7 | type BrokerSpecSuite struct{} 8 | 9 | func (s *BrokerSpecSuite) TestIDValidationCases(c *gc.C) { 10 | var cases = []struct { 11 | id ProcessSpec_ID 12 | expect string 13 | }{ 14 | {ProcessSpec_ID{Zone: "a-zone", Suffix: "a-name"}, ""}, // Success. 15 | {ProcessSpec_ID{Zone: "", Suffix: "a-name"}, "Zone: invalid length .*"}, 16 | {ProcessSpec_ID{Zone: "&*", Suffix: "a-name"}, `Zone: not a valid token .*`}, 17 | {ProcessSpec_ID{Zone: "a-very-very-very-very-looong-zone", Suffix: "a-name"}, "Zone: invalid length .*"}, 18 | {ProcessSpec_ID{Zone: "a-zone", Suffix: "ae"}, "Suffix: invalid length .*"}, 19 | {ProcessSpec_ID{Zone: "a-zone", Suffix: "&*($"}, "Suffix: not a valid token .*"}, 20 | } 21 | for _, tc := range cases { 22 | if tc.expect == "" { 23 | c.Check(tc.id.Validate(), gc.IsNil) 24 | } else { 25 | c.Check(tc.id.Validate(), gc.ErrorMatches, tc.expect) 26 | } 27 | } 28 | } 29 | 30 | func (s *BrokerSpecSuite) TestSpecValidationCases(c *gc.C) { 31 | var model = BrokerSpec{ 32 | ProcessSpec: ProcessSpec{ 33 | Id: ProcessSpec_ID{Zone: "a-zone", Suffix: "a-name"}, 34 | Endpoint: "http://foo", 35 | }, 36 | JournalLimit: 5, 37 | } 38 | c.Check(model.Validate(), gc.Equals, nil) 39 | c.Check(model.ItemLimit(), gc.Equals, 5) 40 | 41 | model.Id.Zone = "" 42 | c.Check(model.Validate(), gc.ErrorMatches, "Id.Zone: invalid length .*") 43 | 44 | model.Id.Zone = "a-zone" 45 | model.Endpoint = "invalid" 46 | c.Check(model.Validate(), gc.ErrorMatches, "Endpoint: not absolute: .*") 47 | 48 | model.Endpoint = "http://foo" 49 | model.JournalLimit = maxBrokerJournalLimit + 1 50 | c.Check(model.Validate(), gc.ErrorMatches, `invalid JournalLimit \(\d+; expected 0 <= JournalLimit <= \d+\)`) 51 | } 52 | 53 | var _ = gc.Suite(&BrokerSpecSuite{}) 54 | -------------------------------------------------------------------------------- /broker/protocol/doc.go: -------------------------------------------------------------------------------- 1 | // Package protocol defines the core broker datamodel, validation behaviors, 2 | // and gRPC APIs which are shared across clients and broker servers. Datamodel 3 | // types and APIs are implemented as generated protobuf messages and stubs, 4 | // typically extended with additional parsing, validation, and shared 5 | // implementation behaviors. A central goal of this package to be highly 6 | // exacting in defining allowed "shapes" that types & messages may take (through 7 | // implementations of the Validator interface), providing strong guarantees to 8 | // brokers and clients that messages are well-formed without need for additional 9 | // ad-hoc, repetitive checks (which often become a maintenance burden). 10 | // 11 | // The package also implements a gRPC "dispatcher" load balancer which provides 12 | // a tight integration between available routes of a journal and gRPC's 13 | // selection of an appropriate transport for a given RPC. Use of the balancer, 14 | // identified by DispatcherGRPCBalancerName, allows clients and servers to use 15 | // a single *grpc.ClientConn through which all RPCs are dispatched. Context 16 | // metadata, attached via WithDispatch*(), informs the balancer of the set of 17 | // specific servers suitable for serving a request. The balancer can factor 18 | // considerations such as which servers have ready transports, or whether the 19 | // RPC will cross availability zones to make a final transport selection. 20 | // 21 | // By convention, this package is usually imported as `pb`, short for 22 | // "Protocol of Broker", due to it's ubiquity and to distinguish it from 23 | // package go.gazette.dev/core/consumer/protocol (imported as `pc`). Eg, 24 | // 25 | // import pb "go.gazette.dev/core/broker/protocol" 26 | // 27 | package protocol 28 | -------------------------------------------------------------------------------- /broker/protocol/endpoint.go: -------------------------------------------------------------------------------- 1 | package protocol 2 | 3 | import ( 4 | "net/url" 5 | ) 6 | 7 | // Endpoint defines an accessible service address. It is a URL, where the 8 | // scheme defines the network transport and semantics of the host, path, 9 | // and query components. At present, supported schemes are: 10 | // 11 | // - http://host(:port)/path?query 12 | // - unix://host/absolute/path/to/unix/domain/socket 13 | type Endpoint string 14 | 15 | // Validate returns an error if the Endpoint is not well-formed. 16 | func (ep Endpoint) Validate() error { 17 | var _, err = ep.parse() 18 | return err 19 | } 20 | 21 | // URL returns the Endpoint as a URL. The Endpoint must Validate, or URL panics. 22 | func (ep Endpoint) URL() *url.URL { 23 | if url, err := ep.parse(); err == nil { 24 | return url 25 | } else { 26 | panic(err.Error()) 27 | } 28 | } 29 | 30 | // GRPCAddr maps this Endpoint into an address form suitable for gRPC to dial. 31 | func (ep Endpoint) GRPCAddr() string { 32 | // gRPC wants the host/authority of a unix:// URL to be empty, 33 | // whereas we populate it with the hostname. Strip it. 34 | var addr string 35 | if u := ep.URL(); u.Scheme == "unix" { 36 | addr = "unix://" + u.Path 37 | } else if u.Port() == "" && u.Scheme == "https" { 38 | addr = u.Host + ":443" 39 | } else if u.Port() == "" && u.Scheme == "http" { 40 | addr = u.Host + ":80" 41 | } else { 42 | addr = u.Host 43 | } 44 | return addr 45 | } 46 | 47 | func (ep Endpoint) parse() (*url.URL, error) { 48 | var url, err = url.Parse(string(ep)) 49 | if err != nil { 50 | return nil, &ValidationError{Err: err} 51 | } else if !url.IsAbs() { 52 | return nil, NewValidationError("not absolute: %s", ep) 53 | } else if url.Host == "" { 54 | return nil, NewValidationError("missing host: %s", ep) 55 | } 56 | return url, nil 57 | } 58 | -------------------------------------------------------------------------------- /broker/protocol/endpoint_test.go: -------------------------------------------------------------------------------- 1 | package protocol 2 | 3 | import ( 4 | gc "gopkg.in/check.v1" 5 | ) 6 | 7 | type EndpointSuite struct{} 8 | 9 | func (s *EndpointSuite) TestValidation(c *gc.C) { 10 | var cases = []struct { 11 | ep Endpoint 12 | expect string 13 | }{ 14 | {"http://host:1234/path?query", ""}, // Success. 15 | {":garbage: :garbage:", "parse .* missing protocol scheme"}, 16 | {"/baz/bing", "not absolute: .*"}, 17 | {"http:///baz/bing", "missing host: .*"}, 18 | } 19 | for _, tc := range cases { 20 | if tc.expect == "" { 21 | c.Check(tc.ep.Validate(), gc.IsNil) 22 | } else { 23 | c.Check(tc.ep.Validate(), gc.ErrorMatches, tc.expect) 24 | } 25 | } 26 | } 27 | 28 | func (s *EndpointSuite) TestURLConversion(c *gc.C) { 29 | var ep Endpoint = "http://host:1234/path?query" 30 | c.Check(ep.URL().Host, gc.Equals, "host:1234") 31 | 32 | ep = "/baz/bing" 33 | c.Check(func() { ep.URL() }, gc.PanicMatches, "not absolute: .*") 34 | } 35 | 36 | func (s *EndpointSuite) TestGRPCConversion(c *gc.C) { 37 | var ep Endpoint = "http://host:1234/path?query" 38 | c.Check(ep.GRPCAddr(), gc.Equals, "host:1234") 39 | 40 | ep = "unix://some-host/a/path" 41 | c.Check(ep.GRPCAddr(), gc.Equals, "unix:///a/path") 42 | } 43 | 44 | var _ = gc.Suite(&EndpointSuite{}) 45 | -------------------------------------------------------------------------------- /broker/protocol/ext/doc.go: -------------------------------------------------------------------------------- 1 | // Package ext defines extensions to broker/protocol that depend on keyspace 2 | // and allocator (which in turn depend on etcd). This package has been separated 3 | // from broker/protocol and broker/client to avoid incurring the dependency on 4 | // etcd for client users. 5 | package ext 6 | -------------------------------------------------------------------------------- /broker/protocol/ext/header_extensions.go: -------------------------------------------------------------------------------- 1 | package ext 2 | 3 | import ( 4 | epb "go.etcd.io/etcd/api/v3/etcdserverpb" 5 | "go.gazette.dev/core/allocator" 6 | pb "go.gazette.dev/core/broker/protocol" 7 | ) 8 | 9 | // FromEtcdResponseHeader converts an etcd ResponseHeader to an equivalent Header_Etcd. 10 | func FromEtcdResponseHeader(h epb.ResponseHeader) pb.Header_Etcd { 11 | return pb.Header_Etcd{ 12 | ClusterId: h.ClusterId, 13 | MemberId: h.MemberId, 14 | Revision: h.Revision, 15 | RaftTerm: h.RaftTerm, 16 | } 17 | } 18 | 19 | // NewUnroutedHeader returns a Header with its ProcessId and Etcd fields derived 20 | // from the v3_allocator.State, and Route left as zero-valued. It is a helper for 21 | // APIs which do not utilize item resolution but still return Headers (eg, List 22 | // and Update). 23 | // 24 | // You MUST hold a lock over State.KS.Mu when calling this function! 25 | func NewUnroutedHeader(s *allocator.State) (hdr pb.Header) { 26 | if s.LocalMemberInd != -1 { 27 | var member = s.Members[s.LocalMemberInd].Decoded.(allocator.Member) 28 | hdr.ProcessId = pb.ProcessSpec_ID{Zone: member.Zone, Suffix: member.Suffix} 29 | } 30 | hdr.Route = pb.Route{Primary: -1} 31 | hdr.Etcd = FromEtcdResponseHeader(s.KS.Header) 32 | return 33 | } 34 | -------------------------------------------------------------------------------- /broker/protocol/ext/header_extensions_test.go: -------------------------------------------------------------------------------- 1 | package ext 2 | 3 | import ( 4 | "go.etcd.io/etcd/api/v3/etcdserverpb" 5 | "go.gazette.dev/core/allocator" 6 | pb "go.gazette.dev/core/broker/protocol" 7 | "go.gazette.dev/core/keyspace" 8 | gc "gopkg.in/check.v1" 9 | ) 10 | 11 | type HeaderSuite struct{} 12 | 13 | func (s *HeaderSuite) TestEtcdConversion(c *gc.C) { 14 | c.Check(pb.Header_Etcd{ 15 | ClusterId: 12, 16 | MemberId: 34, 17 | Revision: 56, 18 | RaftTerm: 78, 19 | }, gc.Equals, FromEtcdResponseHeader( 20 | etcdserverpb.ResponseHeader{ 21 | ClusterId: 12, 22 | MemberId: 34, 23 | Revision: 56, 24 | RaftTerm: 78, 25 | }, 26 | )) 27 | } 28 | 29 | func (s *HeaderSuite) TestUnroutedHeader(c *gc.C) { 30 | var etcd = etcdserverpb.ResponseHeader{ 31 | ClusterId: 12, 32 | MemberId: 34, 33 | Revision: 56, 34 | RaftTerm: 78, 35 | } 36 | var fixture = &allocator.State{ 37 | LocalMemberInd: 0, 38 | Members: keyspace.KeyValues{ 39 | {Decoded: allocator.Member{Zone: "zone", Suffix: "suffix"}}, 40 | }, 41 | KS: &keyspace.KeySpace{ 42 | Header: etcd, 43 | }, 44 | } 45 | c.Check(NewUnroutedHeader(fixture), gc.DeepEquals, pb.Header{ 46 | ProcessId: pb.ProcessSpec_ID{Zone: "zone", Suffix: "suffix"}, 47 | Route: pb.Route{Primary: -1}, 48 | Etcd: FromEtcdResponseHeader(etcd), 49 | }) 50 | } 51 | 52 | var _ = gc.Suite(&HeaderSuite{}) 53 | -------------------------------------------------------------------------------- /broker/protocol/ext/route_extensions.go: -------------------------------------------------------------------------------- 1 | package ext 2 | 3 | import ( 4 | "go.gazette.dev/core/allocator" 5 | pb "go.gazette.dev/core/broker/protocol" 6 | "go.gazette.dev/core/keyspace" 7 | ) 8 | 9 | // Init Initializes Route with the provided allocator Assignments. 10 | func Init(route *pb.Route, assignments keyspace.KeyValues) { 11 | *route = pb.Route{Primary: -1, Members: route.Members[:0]} 12 | 13 | for _, kv := range assignments { 14 | var a = kv.Decoded.(allocator.Assignment) 15 | if a.Slot == 0 { 16 | route.Primary = int32(len(route.Members)) 17 | } 18 | 19 | route.Members = append(route.Members, pb.ProcessSpec_ID{ 20 | Zone: a.MemberZone, 21 | Suffix: a.MemberSuffix, 22 | }) 23 | } 24 | } 25 | 26 | // AttachEndpoints maps Route members through the KeySpace to their respective 27 | // specs, and attaches the associated Endpoint of each to the Route. 28 | // KeySpace must already be read-locked. 29 | func AttachEndpoints(route *pb.Route, ks *keyspace.KeySpace) { 30 | if len(route.Members) != 0 { 31 | route.Endpoints = make([]pb.Endpoint, len(route.Members)) 32 | } 33 | for i, b := range route.Members { 34 | if member, ok := allocator.LookupMember(ks, b.Zone, b.Suffix); !ok { 35 | continue // Assignment with missing Member. Ignore. 36 | } else { 37 | route.Endpoints[i] = member.MemberValue.(interface { 38 | GetEndpoint() pb.Endpoint 39 | }).GetEndpoint() 40 | } 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /broker/protocol/ext/route_extensions_test.go: -------------------------------------------------------------------------------- 1 | package ext 2 | 3 | import ( 4 | "go.etcd.io/etcd/api/v3/mvccpb" 5 | "go.gazette.dev/core/allocator" 6 | pb "go.gazette.dev/core/broker/protocol" 7 | "go.gazette.dev/core/keyspace" 8 | gc "gopkg.in/check.v1" 9 | ) 10 | 11 | type RouteSuite struct{} 12 | 13 | func (s *RouteSuite) TestInitialization(c *gc.C) { 14 | 15 | var rt = pb.Route{} 16 | Init(&rt, nil) 17 | c.Check(rt, gc.DeepEquals, pb.Route{Primary: -1, Members: nil}) 18 | 19 | var kv = keyspace.KeyValues{ 20 | {Decoded: allocator.Assignment{MemberZone: "zone-a", MemberSuffix: "member-1", Slot: 1}}, 21 | {Decoded: allocator.Assignment{MemberZone: "zone-a", MemberSuffix: "member-3", Slot: 3}}, 22 | {Decoded: allocator.Assignment{MemberZone: "zone-b", MemberSuffix: "member-2", Slot: 2}}, 23 | {Decoded: allocator.Assignment{MemberZone: "zone-c", MemberSuffix: "member-4", Slot: 0}}, 24 | } 25 | Init(&rt, kv) 26 | 27 | c.Check(rt, gc.DeepEquals, pb.Route{ 28 | Primary: 3, 29 | Members: []pb.ProcessSpec_ID{ 30 | {Zone: "zone-a", Suffix: "member-1"}, 31 | {Zone: "zone-a", Suffix: "member-3"}, 32 | {Zone: "zone-b", Suffix: "member-2"}, 33 | {Zone: "zone-c", Suffix: "member-4"}, 34 | }, 35 | }) 36 | 37 | kv = kv[:3] // This time, remove the primary Assignment. 38 | Init(&rt, kv) 39 | 40 | c.Check(rt, gc.DeepEquals, pb.Route{ 41 | Primary: -1, 42 | Members: []pb.ProcessSpec_ID{ 43 | {Zone: "zone-a", Suffix: "member-1"}, 44 | {Zone: "zone-a", Suffix: "member-3"}, 45 | {Zone: "zone-b", Suffix: "member-2"}, 46 | }, 47 | }) 48 | } 49 | 50 | func (s *RouteSuite) TestEndpointAttachmentAndCopy(c *gc.C) { 51 | var ks = keyspace.KeySpace{ 52 | Root: "/root", 53 | KeyValues: keyspace.KeyValues{ 54 | {Raw: mvccpb.KeyValue{Key: []byte("/root/members/zone-a#member-1")}, 55 | Decoded: allocator.Member{Zone: "zone-a", Suffix: "member-1", 56 | MemberValue: &pb.BrokerSpec{ProcessSpec: pb.ProcessSpec{Endpoint: "http://host-a:port/path"}}}}, 57 | {Raw: mvccpb.KeyValue{Key: []byte("/root/members/zone-b#member-2")}, 58 | Decoded: allocator.Member{Zone: "zone-b", Suffix: "member-2", 59 | MemberValue: &pb.BrokerSpec{ProcessSpec: pb.ProcessSpec{Endpoint: "http://host-b:port/path"}}}}, 60 | }, 61 | } 62 | var rt = pb.Route{ 63 | Members: []pb.ProcessSpec_ID{ 64 | {Zone: "zone-a", Suffix: "member-1"}, 65 | {Zone: "zone-a", Suffix: "member-3"}, 66 | {Zone: "zone-b", Suffix: "member-2"}, 67 | }, 68 | } 69 | 70 | AttachEndpoints(&rt, &ks) 71 | c.Check(rt.Endpoints, gc.DeepEquals, []pb.Endpoint{"http://host-a:port/path", "", "http://host-b:port/path"}) 72 | 73 | var other = rt.Copy() 74 | 75 | // Expect |other| is deeply equal while referencing different slices. 76 | c.Check(rt.Members, gc.DeepEquals, other.Members) 77 | c.Check(rt.Endpoints, gc.DeepEquals, other.Endpoints) 78 | c.Check(&rt.Members[0], gc.Not(gc.Equals), &other.Members[0]) 79 | c.Check(&rt.Endpoints[0], gc.Not(gc.Equals), &other.Endpoints[0]) 80 | } 81 | -------------------------------------------------------------------------------- /broker/protocol/fragment_store.go: -------------------------------------------------------------------------------- 1 | package protocol 2 | 3 | import ( 4 | "net/url" 5 | "strings" 6 | ) 7 | 8 | // FragmentStore defines a storage backend base path for Journal Fragments. 9 | // It is a URL, where the scheme defines the storage backend service. As 10 | // FragmentStores "root" remote storage locations of fragments, their path 11 | // component must end in a trailing slash. 12 | // 13 | // Currently supported schemes are "gs" for Google Cloud Storage, "s3" for 14 | // Amazon S3, "azure" for Azure Cloud Storage, and "file" for a local file-system / NFS mount. Eg: 15 | // 16 | // * s3://bucket-name/a/sub-path/?profile=a-shared-credentials-profile 17 | // * gs://bucket-name/a/sub-path/? 18 | // * file:///a/local/volume/mount 19 | // 20 | // FragmentStore implementations may support additional configuration which 21 | // can be declared via URL query arguments. The meaning of these query 22 | // arguments and values are specific to the store in question; consult 23 | // FileStoreConfig, S3StoreConfig, and GSStoreConfig of the fragment 24 | // package for details of available configuration. 25 | type FragmentStore string 26 | 27 | // Validate returns an error if the FragmentStore is not well-formed. 28 | func (fs FragmentStore) Validate() error { 29 | var _, err = fs.parse() 30 | return err 31 | } 32 | 33 | // URL returns the FragmentStore as a URL. The FragmentStore must Validate, or URL panics. 34 | func (fs FragmentStore) URL() *url.URL { 35 | if url, err := fs.parse(); err == nil { 36 | return url 37 | } else { 38 | panic(err.Error()) 39 | } 40 | } 41 | 42 | func (fs FragmentStore) parse() (*url.URL, error) { 43 | var url, err = url.Parse(string(fs)) 44 | if err != nil { 45 | return nil, &ValidationError{Err: err} 46 | } else if !url.IsAbs() { 47 | return nil, NewValidationError("not absolute (%s)", fs) 48 | } 49 | 50 | switch url.Scheme { 51 | case "s3", "gs", "azure": 52 | if url.Host == "" { 53 | return nil, NewValidationError("missing bucket (%s)", fs) 54 | } 55 | case "azure-ad": 56 | var splitPath = strings.Split(url.Path[1:], "/") 57 | if url.Host == "" { 58 | return nil, NewValidationError("missing tenant ID (%s)", fs) 59 | } else if splitPath[0] == "" { 60 | return nil, NewValidationError("missing storage account (%s)", fs) 61 | } else if splitPath[1] == "" { 62 | return nil, NewValidationError("missing storage container (%s)", fs) 63 | } 64 | case "file": 65 | if url.Host != "" { 66 | return nil, NewValidationError("file scheme cannot have host (%s)", fs) 67 | } 68 | default: 69 | return nil, NewValidationError("invalid scheme (%s)", url.Scheme) 70 | } 71 | 72 | if path := url.Path; len(path) == 0 || path[len(path)-1] != '/' { 73 | return nil, NewValidationError("path component doesn't end in '/' (%s)", url.Path) 74 | } 75 | return url, nil 76 | } 77 | 78 | func fragmentStoresEq(a, b []FragmentStore) bool { 79 | if len(a) != len(b) { 80 | return false 81 | } 82 | for i := range a { 83 | if a[i] != b[i] { 84 | return false 85 | } 86 | } 87 | return true 88 | } 89 | -------------------------------------------------------------------------------- /broker/protocol/fragment_store_test.go: -------------------------------------------------------------------------------- 1 | package protocol 2 | 3 | import ( 4 | "net/url" 5 | 6 | gc "gopkg.in/check.v1" 7 | ) 8 | 9 | type FragmentStoreSuite struct{} 10 | 11 | func (s *FragmentStoreSuite) TestValidation(c *gc.C) { 12 | var cases = []struct { 13 | fs FragmentStore 14 | expect string 15 | }{ 16 | {"s3://my-bucket/?query", ""}, // Success. 17 | {"s3://my-bucket/subpath/?query", ""}, // Success (non-empty prefix). 18 | {"file:///mnt/path/", ``}, // Success. 19 | {"file:///mnt/path/?query", ``}, // Success. 20 | 21 | {"s3://my-bucket", `path component doesn't end in '/' \(\)`}, 22 | {"s3://my-bucket/subpath?query", `path component doesn't end in '/' \(/subpath\)`}, 23 | {":garbage: :garbage:", "parse .* missing protocol scheme"}, 24 | {"foobar://baz/", `invalid scheme \(foobar\)`}, 25 | {"/baz/bing/", `not absolute \(/baz/bing/\)`}, 26 | {"gs:///baz/bing/", `missing bucket \(gs:///baz/bing/\)`}, 27 | {"file://host/mnt/path/", `file scheme cannot have host \(file://host/mnt/path/\)`}, 28 | {"file:///mnt/path", `path component doesn't end in '/' \(/mnt/path\)`}, 29 | } 30 | for _, tc := range cases { 31 | if tc.expect == "" { 32 | c.Check(tc.fs.Validate(), gc.IsNil) 33 | } else { 34 | c.Check(tc.fs.Validate(), gc.ErrorMatches, tc.expect) 35 | } 36 | } 37 | } 38 | 39 | func (s *FragmentStoreSuite) TestURLConversion(c *gc.C) { 40 | var fs FragmentStore = "s3://bucket/sub/path/?query" 41 | c.Check(fs.URL(), gc.DeepEquals, &url.URL{ 42 | Scheme: "s3", 43 | Host: "bucket", 44 | Path: "/sub/path/", 45 | RawQuery: "query", 46 | }) 47 | 48 | fs = "/baz/bing/" 49 | c.Check(func() { fs.URL() }, gc.PanicMatches, `not absolute \(/baz/bing/\)`) 50 | } 51 | 52 | var _ = gc.Suite(&FragmentStoreSuite{}) 53 | -------------------------------------------------------------------------------- /broker/protocol/header_extensions.go: -------------------------------------------------------------------------------- 1 | package protocol 2 | 3 | // Validate returns an error if the Header is not well-formed. 4 | func (m Header) Validate() error { 5 | if m.ProcessId != (ProcessSpec_ID{}) { 6 | if err := m.ProcessId.Validate(); err != nil { 7 | return ExtendContext(err, "ProcessId") 8 | } 9 | } 10 | if err := m.Route.Validate(); err != nil { 11 | return ExtendContext(err, "Route") 12 | } else if err = m.Etcd.Validate(); err != nil { 13 | return ExtendContext(err, "Etcd") 14 | } 15 | return nil 16 | } 17 | 18 | // Validate returns an error if the Header_Etcd is not well-formed. 19 | func (m Header_Etcd) Validate() error { 20 | if m.ClusterId == 0 { 21 | return NewValidationError("invalid ClusterId (expected != 0)") 22 | } else if m.MemberId == 0 { 23 | return NewValidationError("invalid MemberId (expected != 0)") 24 | } else if m.Revision <= 0 { 25 | return NewValidationError("invalid Revision (%d; expected 0 < revision)", m.Revision) 26 | } else if m.RaftTerm == 0 { 27 | return NewValidationError("invalid RaftTerm (expected != 0)") 28 | } 29 | return nil 30 | } 31 | -------------------------------------------------------------------------------- /broker/protocol/header_extensions_test.go: -------------------------------------------------------------------------------- 1 | package protocol 2 | 3 | import ( 4 | gc "gopkg.in/check.v1" 5 | ) 6 | 7 | type HeaderSuite struct{} 8 | 9 | func (s *HeaderSuite) TestEtcdValidationCases(c *gc.C) { 10 | var model = Header_Etcd{ 11 | ClusterId: 0, 12 | MemberId: 0, 13 | Revision: -42, 14 | RaftTerm: 0, 15 | } 16 | c.Check(model.Validate(), gc.ErrorMatches, `invalid ClusterId \(expected != 0\)`) 17 | model.ClusterId = 12 18 | c.Check(model.Validate(), gc.ErrorMatches, `invalid MemberId \(expected != 0\)`) 19 | model.MemberId = 34 20 | c.Check(model.Validate(), gc.ErrorMatches, `invalid Revision \(-42; expected 0 < revision\)`) 21 | model.Revision = 56 22 | c.Check(model.Validate(), gc.ErrorMatches, `invalid RaftTerm \(expected != 0\)`) 23 | model.RaftTerm = 78 24 | 25 | c.Check(model.Validate(), gc.IsNil) 26 | } 27 | 28 | func (s *HeaderSuite) TestHeaderValidationCases(c *gc.C) { 29 | var model = Header{ 30 | ProcessId: ProcessSpec_ID{Zone: "no-suffix", Suffix: ""}, 31 | Route: Route{Primary: 2, Members: []ProcessSpec_ID{{Zone: "zone", Suffix: "name"}}}, 32 | Etcd: Header_Etcd{ 33 | ClusterId: 0, 34 | MemberId: 34, 35 | Revision: 56, 36 | RaftTerm: 78, 37 | }, 38 | } 39 | c.Check(model.Validate(), gc.ErrorMatches, `ProcessId.Suffix: invalid length .*`) 40 | model.ProcessId = ProcessSpec_ID{Zone: "zone", Suffix: "name"} 41 | c.Check(model.Validate(), gc.ErrorMatches, `Route: invalid Primary .*`) 42 | model.Route.Primary = 0 43 | c.Check(model.Validate(), gc.ErrorMatches, `Etcd: invalid ClusterId .*`) 44 | model.Etcd.ClusterId = 12 45 | 46 | c.Check(model.Validate(), gc.IsNil) 47 | 48 | // Empty ProcessId is permitted. 49 | model.ProcessId = ProcessSpec_ID{} 50 | c.Check(model.Validate(), gc.IsNil) 51 | } 52 | 53 | var _ = gc.Suite(&HeaderSuite{}) 54 | -------------------------------------------------------------------------------- /broker/protocol/protocol_gateway.yaml: -------------------------------------------------------------------------------- 1 | type: google.api.Service 2 | config_version: 3 3 | 4 | http: 5 | rules: 6 | - selector: protocol.Journal.List 7 | post: /v1/journals/list 8 | body: "*" 9 | - selector: protocol.Journal.Read 10 | post: /v1/journals/read 11 | body: "*" 12 | -------------------------------------------------------------------------------- /broker/protocol/route_extensions.go: -------------------------------------------------------------------------------- 1 | package protocol 2 | 3 | // Copy returns a deep copy of the Route. 4 | func (m Route) Copy() Route { 5 | return Route{ 6 | Members: append([]ProcessSpec_ID(nil), m.Members...), 7 | Primary: m.Primary, 8 | Endpoints: append([]Endpoint(nil), m.Endpoints...), 9 | } 10 | } 11 | 12 | // Validate returns an error if the Route is not well-formed. 13 | func (m Route) Validate() error { 14 | for i, b := range m.Members { 15 | if err := b.Validate(); err != nil { 16 | return ExtendContext(err, "Members[%d]", i) 17 | } 18 | if i != 0 && !m.Members[i-1].Less(b) { 19 | return NewValidationError("Members not in unique, sorted order (index %d; %+v <= %+v)", 20 | i, m.Members[i-1], m.Members[i]) 21 | } 22 | } 23 | 24 | if m.Primary < -1 || m.Primary >= int32(len(m.Members)) { 25 | return NewValidationError("invalid Primary (%+v; expected -1 <= Primary < %d)", 26 | m.Primary, len(m.Members)) 27 | } 28 | 29 | if l := len(m.Endpoints); l != 0 && l != len(m.Members) { 30 | return NewValidationError("len(Endpoints) != 0, and != len(Members) (%d vs %d)", 31 | l, len(m.Members)) 32 | } 33 | 34 | for i, ep := range m.Endpoints { 35 | if ep == "" { 36 | continue 37 | } else if err := ep.Validate(); err != nil { 38 | return ExtendContext(err, "Endpoints[%d]", i) 39 | } 40 | } 41 | return nil 42 | } 43 | 44 | // Equivalent returns true if the Routes have equivalent broker Names, Zones, 45 | // and current Primary. It does not compare broker Endpoints. 46 | func (m Route) Equivalent(other *Route) bool { 47 | if other == nil { 48 | return false 49 | } else if m.Primary != other.Primary { 50 | return false 51 | } else if len(m.Members) != len(other.Members) { 52 | return false 53 | } 54 | for i := range m.Members { 55 | if m.Members[i] != other.Members[i] { 56 | return false 57 | } 58 | } 59 | return true 60 | } 61 | 62 | // MarshalString returns the marshaled encoding of the Route as a string. 63 | func (m Route) MarshalString() string { 64 | var d, err = m.Marshal() 65 | if err != nil { 66 | panic(err.Error()) // Cannot happen, as we use no custom marshalling. 67 | } 68 | return string(d) 69 | } 70 | -------------------------------------------------------------------------------- /broker/protocol/route_extensions_test.go: -------------------------------------------------------------------------------- 1 | package protocol 2 | 3 | import ( 4 | gc "gopkg.in/check.v1" 5 | ) 6 | 7 | type RouteSuite struct{} 8 | 9 | func (s *RouteSuite) TestValidationCases(c *gc.C) { 10 | var rt = Route{ 11 | Primary: -1, 12 | Members: []ProcessSpec_ID{ 13 | {Zone: "zone-a", Suffix: "member-1"}, 14 | {Zone: "zone-a", Suffix: "member-3"}, 15 | {Zone: "zone-b", Suffix: "member-2"}, 16 | }, 17 | Endpoints: []Endpoint{"http://foo", "http://bar", "http://baz"}, 18 | } 19 | c.Check(rt.Validate(), gc.IsNil) 20 | 21 | rt.Members[0].Zone = "invalid zone" 22 | c.Check(rt.Validate(), gc.ErrorMatches, `Members\[0\].Zone: not a valid token \(invalid zone\)`) 23 | rt.Members[0].Zone = "zone-a" 24 | 25 | rt.Members[1], rt.Members[2] = rt.Members[2], rt.Members[1] 26 | c.Check(rt.Validate(), gc.ErrorMatches, `Members not in unique, sorted order \(index 2; {Zone.*?} <= {Zone.*?}\)`) 27 | rt.Members[1], rt.Members[2] = rt.Members[2], rt.Members[1] 28 | 29 | rt.Primary = -2 30 | c.Check(rt.Validate(), gc.ErrorMatches, `invalid Primary \(-2; expected -1 <= Primary < 3\)`) 31 | rt.Primary = 3 32 | c.Check(rt.Validate(), gc.ErrorMatches, `invalid Primary \(3; expected -1 <= Primary < 3\)`) 33 | rt.Primary = 2 34 | 35 | rt.Endpoints = append(rt.Endpoints, "http://bing") 36 | c.Check(rt.Validate(), gc.ErrorMatches, `len\(Endpoints\) != 0, and != len\(Members\) \(4 vs 3\)`) 37 | 38 | rt.Endpoints = rt.Endpoints[:3] 39 | rt.Endpoints[2] = "invalid" 40 | c.Check(rt.Validate(), gc.ErrorMatches, `Endpoints\[2\]: not absolute: invalid`) 41 | rt.Endpoints[2] = "http://baz" 42 | } 43 | 44 | func (s *RouteSuite) TestEquivalencyCases(c *gc.C) { 45 | var rt = Route{ 46 | Primary: -1, 47 | Members: []ProcessSpec_ID{ 48 | {Zone: "zone-a", Suffix: "member-1"}, 49 | {Zone: "zone-a", Suffix: "member-3"}, 50 | {Zone: "zone-b", Suffix: "member-2"}, 51 | }, 52 | Endpoints: []Endpoint{"http://foo", "http://bar", "http://baz"}, 53 | } 54 | var other = rt.Copy() 55 | other.Endpoints = nil // Endpoints are optional for equivalency. 56 | 57 | c.Check(rt.Equivalent(&other), gc.Equals, true) 58 | c.Check(other.Equivalent(&rt), gc.Equals, true) 59 | 60 | rt.Primary = 1 61 | c.Check(rt.Equivalent(&other), gc.Equals, false) 62 | c.Check(other.Equivalent(&rt), gc.Equals, false) 63 | 64 | other.Primary = 1 65 | other.Members[1].Zone = "zone-aa" 66 | c.Check(rt.Equivalent(&other), gc.Equals, false) 67 | c.Check(other.Equivalent(&rt), gc.Equals, false) 68 | 69 | rt.Members[1].Zone = "zone-aa" 70 | c.Check(rt.Equivalent(&other), gc.Equals, true) 71 | c.Check(other.Equivalent(&rt), gc.Equals, true) 72 | } 73 | 74 | var _ = gc.Suite(&RouteSuite{}) 75 | -------------------------------------------------------------------------------- /cmd/gazctl/gazctlcmd/edit_docs.go: -------------------------------------------------------------------------------- 1 | package gazctlcmd 2 | 3 | import ( 4 | "bytes" 5 | "text/template" 6 | ) 7 | 8 | var journalsEditLongDesc, shardsEditLongDesc string 9 | 10 | type editDescription struct { 11 | Type, HelpCommand, Examples string 12 | } 13 | 14 | func init() { 15 | // Avoid heavy duplication of text between "journals edit" and 16 | // "shards edit" commands by templating their long descriptions. 17 | var editTemplate = template.Must(template.New("template").Parse(`Edit and apply {{ .Type }} specifications. 18 | 19 | The edit command allows you to directly edit {{ .Type }} specifications matching 20 | the supplied LabelSelector. It will open the editor defined by your GAZ_EDITOR or 21 | EDITOR environment variables or fall back to 'vi'. Editing from Windows is 22 | currently not supported. 23 | 24 | Upon exiting the editor, if the file has been changed, it will be validated and 25 | applied. If the file is invalid or fails to apply, the editor is re-opened. 26 | Exiting the editor with no changes or saving an empty file are interpreted as 27 | the user aborting the edit attempt. 28 | 29 | Use --selector to supply a LabelSelector which constrains the set of returned 30 | {{ .Type }} specifications. See "{{ .HelpCommand }}" for details and examples. 31 | 32 | {{ .Examples }} 33 | `)) 34 | var journalData = editDescription{ 35 | Type: "journal", 36 | HelpCommand: "journals list --help", 37 | Examples: `Edit specifications of journals having an exact name: 38 | > gazctl journals edit --selector "name in (foo/bar, baz/bing)" 39 | 40 | Use an alternative editor 41 | > GAZ_EDITOR=nano gazctl journals edit --selector "prefix = my/prefix/"`, 42 | } 43 | var shardData = editDescription{ 44 | Type: "shard", 45 | HelpCommand: "shards list --help", 46 | Examples: `Edit specifications of shards having an exact ID: 47 | > gazctl shards edit --selector "id in (foo, bar)" 48 | 49 | Use an alternative editor 50 | > GAZ_EDITOR=nano gazctl shards edit --selector "id = baz"`, 51 | } 52 | 53 | // Save the template output to package vars. 54 | var buf = &bytes.Buffer{} 55 | if err := editTemplate.Execute(buf, journalData); err != nil { 56 | panic(err) 57 | } 58 | journalsEditLongDesc = buf.String() + maxTxnSizeWarning 59 | buf.Reset() 60 | if err := editTemplate.Execute(buf, shardData); err != nil { 61 | panic(err) 62 | } 63 | shardsEditLongDesc = buf.String() + maxTxnSizeWarning 64 | } 65 | -------------------------------------------------------------------------------- /cmd/gazctl/gazctlcmd/journals_edit.go: -------------------------------------------------------------------------------- 1 | package gazctlcmd 2 | 3 | import ( 4 | "bytes" 5 | "context" 6 | "io" 7 | 8 | log "github.com/sirupsen/logrus" 9 | "go.gazette.dev/core/broker/client" 10 | "go.gazette.dev/core/broker/journalspace" 11 | pb "go.gazette.dev/core/broker/protocol" 12 | "go.gazette.dev/core/cmd/gazctl/gazctlcmd/editor" 13 | pc "go.gazette.dev/core/consumer/protocol" 14 | mbp "go.gazette.dev/core/mainboilerplate" 15 | "gopkg.in/yaml.v2" 16 | ) 17 | 18 | type cmdJournalsEdit struct { 19 | EditConfig 20 | } 21 | 22 | func init() { 23 | CommandRegistry.AddCommand("journals", "edit", "Edit journal specifications", journalsEditLongDesc, &cmdJournalsEdit{}) 24 | } 25 | 26 | func (cmd *cmdJournalsEdit) Execute([]string) error { 27 | startup(JournalsCfg.BaseConfig) 28 | var ctx = context.Background() 29 | var rjc = JournalsCfg.Broker.MustRoutedJournalClient(ctx) 30 | 31 | return editor.EditRetryLoop(editor.RetryLoopArgs{ 32 | FilePrefix: "gazctl-journals-edit-", 33 | JournalClient: rjc, 34 | SelectFn: cmd.selectSpecs, 35 | ApplyFn: cmd.applySpecs, 36 | AbortIfUnchanged: true, 37 | }) 38 | } 39 | 40 | // selectSpecs returns the hoisted YAML specs of journals matching the selector. 41 | func (cmd *cmdJournalsEdit) selectSpecs(_ pc.ShardClient, journalClient pb.JournalClient) io.Reader { 42 | var resp = listJournals(journalClient, cmd.Selector) 43 | 44 | if len(resp.Journals) == 0 { 45 | log.WithField("selector", cmd.Selector).Panic("no journals match selector") 46 | } 47 | var buf = &bytes.Buffer{} 48 | writeHoistedJournalSpecTree(buf, resp) 49 | 50 | return buf 51 | } 52 | 53 | func (cmd *cmdJournalsEdit) applySpecs(b []byte, _ pc.ShardClient, journalClient pb.JournalClient) error { 54 | var tree journalspace.Node 55 | if err := yaml.UnmarshalStrict(b, &tree); err != nil { 56 | return err 57 | } 58 | if err := tree.Validate(); err != nil { 59 | return err 60 | } 61 | 62 | var req = newJournalSpecApplyRequest(&tree) 63 | if err := req.Validate(); err != nil { 64 | return err 65 | } 66 | 67 | var ctx = context.Background() 68 | var resp, err = client.ApplyJournalsInBatches(ctx, journalClient, req, cmd.MaxTxnSize) 69 | mbp.Must(err, "failed to apply journals") 70 | log.WithField("revision", resp.Header.Etcd.Revision).Info("successfully applied") 71 | 72 | return nil 73 | } 74 | -------------------------------------------------------------------------------- /cmd/gazctl/gazctlcmd/journals_suspend.go: -------------------------------------------------------------------------------- 1 | package gazctlcmd 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | 7 | log "github.com/sirupsen/logrus" 8 | "go.gazette.dev/core/broker/client" 9 | pb "go.gazette.dev/core/broker/protocol" 10 | mbp "go.gazette.dev/core/mainboilerplate" 11 | "golang.org/x/sync/errgroup" 12 | ) 13 | 14 | type cmdJournalSuspend struct { 15 | Selector string `long:"selector" short:"l" required:"true" description:"Label Selector query to filter on"` 16 | Force bool `long:"force" short:"f" description:"Suspend journals even if they have local fragments"` 17 | } 18 | 19 | func init() { 20 | CommandRegistry.AddCommand("journals", "suspend", "Suspend idle journals", ` 21 | Suspend idle journals to use fewer or zero broker replicas. 22 | 23 | Suspension updates the 'suspend' field of the JournalSpec with its suspended 24 | status and its resumption offset. When applying other updates to JournalSpecs, 25 | operators utilizing journal suspension must take care to not overwrite the 26 | journal's suspension configuration. 27 | 28 | Typically this means reading the current JournalSpec and its ModRevision, 29 | copying the current 'suspend' field alongside other changes being made, 30 | and then applying the updated JournalSpec with ExpectModRevision. 31 | 32 | The 'journals edit' subcommand uses this workflow and is safe to use with suspension. 33 | `, &cmdJournalSuspend{}) 34 | } 35 | 36 | func (cmd *cmdJournalSuspend) Execute([]string) error { 37 | startup(JournalsCfg.BaseConfig) 38 | 39 | var err error 40 | var ctx = context.Background() 41 | var rjc = JournalsCfg.Broker.MustRoutedJournalClient(ctx) 42 | 43 | // Get the list of journals which match this selector. 44 | var listRequest pb.ListRequest 45 | listRequest.Selector, err = pb.ParseLabelSelector(cmd.Selector) 46 | mbp.Must(err, "failed to parse label selector", "selector", cmd.Selector) 47 | 48 | var listResp *pb.ListResponse 49 | listResp, err = client.ListAllJournals(ctx, rjc, listRequest) 50 | mbp.Must(err, "failed to resolve journals from selector", cmd.Selector) 51 | 52 | group, ctx := errgroup.WithContext(ctx) 53 | 54 | for _, journal := range listResp.Journals { 55 | if journal.Spec.Suspend.GetLevel() == pb.JournalSpec_Suspend_FULL { 56 | continue 57 | } 58 | group.Go(func() error { 59 | return suspendJournal(ctx, rjc, journal.Spec.Name, cmd.Force) 60 | }) 61 | } 62 | return group.Wait() 63 | } 64 | 65 | func suspendJournal( 66 | ctx context.Context, 67 | rjc pb.RoutedJournalClient, 68 | journal pb.Journal, 69 | force bool, 70 | ) error { 71 | 72 | var mode = pb.AppendRequest_SUSPEND_IF_FLUSHED 73 | if force { 74 | mode = pb.AppendRequest_SUSPEND_NOW 75 | } 76 | 77 | var a = client.NewAppender(ctx, rjc, pb.AppendRequest{ 78 | Journal: journal, 79 | Suspend: mode, 80 | }) 81 | var err = a.Close() 82 | 83 | if err == nil || a.Response.Status == pb.Status_SUSPENDED { 84 | log.WithFields(log.Fields{ 85 | "journal": journal, 86 | "status": a.Response.Status, 87 | }).Info("journal suspension status") 88 | return nil 89 | } else { 90 | return fmt.Errorf("failed to suspend journal %s: %w", journal, err) 91 | } 92 | } 93 | -------------------------------------------------------------------------------- /cmd/gazctl/gazctlcmd/shards_apply.go: -------------------------------------------------------------------------------- 1 | package gazctlcmd 2 | 3 | import ( 4 | "context" 5 | "os" 6 | 7 | "github.com/gogo/protobuf/proto" 8 | log "github.com/sirupsen/logrus" 9 | "go.gazette.dev/core/consumer" 10 | pc "go.gazette.dev/core/consumer/protocol" 11 | "go.gazette.dev/core/consumer/shardspace" 12 | mbp "go.gazette.dev/core/mainboilerplate" 13 | ) 14 | 15 | type cmdShardsApply struct { 16 | ApplyConfig 17 | } 18 | 19 | func init() { 20 | CommandRegistry.AddCommand("shards", "apply", "Apply shard specifications", ` 21 | Apply a collection of ShardSpec creations, updates, or deletions. 22 | 23 | ShardSpecs should be provided as a YAML list, the same format produced by 24 | "gazctl shards list". Consumers verify that the Etcd "revision" field of each 25 | ShardSpec is correct, and will fail the entire apply operation if any have since 26 | been updated. A common operational pattern is to list, edit, and re-apply a 27 | collection of ShardSpecs; this check ensures concurrent modifications are caught. 28 | 29 | You may explicitly inform the broker to apply your ShardSpecs regardless of the 30 | current state of specifications in Etcd by passing in a revision value of -1. 31 | This commonly done when operators keep ShardSpecs in version control as their 32 | source of truth. 33 | 34 | ShardSpecs may be created by setting "revision" to zero or omitting it altogether. 35 | 36 | ShardSpecs may be deleted by setting their field "delete" to true. 37 | `+maxTxnSizeWarning, &cmdShardsApply{}) 38 | } 39 | 40 | func (cmd *cmdShardsApply) Execute([]string) error { 41 | startup(ShardsCfg.BaseConfig) 42 | 43 | var set shardspace.Set 44 | mbp.Must(cmd.decode(&set), "failed to decode shardspace from YAML") 45 | 46 | var ctx = context.Background() 47 | var req = newShardSpecApplyRequest(set) 48 | 49 | mbp.Must(req.Validate(), "failed to validate ApplyRequest") 50 | mbp.Must(consumer.VerifyReferencedJournals(ctx, ShardsCfg.Broker.MustJournalClient(ctx), req), 51 | "failed to validate journals of the ApplyRequest") 52 | 53 | if cmd.DryRun { 54 | _ = proto.MarshalText(os.Stdout, req) 55 | return nil 56 | } 57 | 58 | var resp, err = consumer.ApplyShardsInBatches(ctx, ShardsCfg.Consumer.MustShardClient(ctx), req, cmd.MaxTxnSize) 59 | mbp.Must(err, "failed to apply shards") 60 | log.WithField("rev", resp.Header.Etcd.Revision).Info("successfully applied") 61 | 62 | return nil 63 | } 64 | 65 | // newShardSpecApplyRequest builds the ApplyRequest. 66 | func newShardSpecApplyRequest(set shardspace.Set) *pc.ApplyRequest { 67 | set.PushDown() 68 | 69 | var req = new(pc.ApplyRequest) 70 | for i := range set.Shards { 71 | var change = pc.ApplyRequest_Change{ExpectModRevision: set.Shards[i].Revision} 72 | 73 | if set.Shards[i].Delete != nil && *set.Shards[i].Delete == true { 74 | change.Delete = set.Shards[i].Spec.Id 75 | } else { 76 | change.Upsert = &set.Shards[i].Spec 77 | } 78 | req.Changes = append(req.Changes, change) 79 | } 80 | return req 81 | } 82 | -------------------------------------------------------------------------------- /cmd/gazctl/gazctlcmd/shards_edit.go: -------------------------------------------------------------------------------- 1 | package gazctlcmd 2 | 3 | import ( 4 | "bytes" 5 | "context" 6 | "io" 7 | 8 | "github.com/pkg/errors" 9 | log "github.com/sirupsen/logrus" 10 | pb "go.gazette.dev/core/broker/protocol" 11 | "go.gazette.dev/core/cmd/gazctl/gazctlcmd/editor" 12 | "go.gazette.dev/core/consumer" 13 | pc "go.gazette.dev/core/consumer/protocol" 14 | "go.gazette.dev/core/consumer/shardspace" 15 | mbp "go.gazette.dev/core/mainboilerplate" 16 | "gopkg.in/yaml.v2" 17 | ) 18 | 19 | type cmdShardsEdit struct { 20 | EditConfig 21 | } 22 | 23 | func init() { 24 | CommandRegistry.AddCommand("shards", "edit", "Edit shard specifications", shardsEditLongDesc, &cmdShardsEdit{}) 25 | } 26 | 27 | func (cmd *cmdShardsEdit) Execute([]string) error { 28 | startup(ShardsCfg.BaseConfig) 29 | 30 | var ctx = context.Background() 31 | var rsc = ShardsCfg.Consumer.MustRoutedShardClient(ctx) 32 | var rjc = ShardsCfg.Broker.MustRoutedJournalClient(ctx) 33 | 34 | return editor.EditRetryLoop(editor.RetryLoopArgs{ 35 | ShardClient: rsc, 36 | JournalClient: rjc, 37 | FilePrefix: "gazctl-shards-edit-", 38 | SelectFn: cmd.selectSpecs, 39 | ApplyFn: cmd.applyShardSpecYAML, 40 | AbortIfUnchanged: true, 41 | }) 42 | } 43 | 44 | func (cmd *cmdShardsEdit) selectSpecs(client pc.ShardClient, _ pb.JournalClient) io.Reader { 45 | var resp = listShards(client, cmd.Selector) 46 | 47 | var buf = &bytes.Buffer{} 48 | if len(resp.Shards) == 0 { 49 | log.WithField("selector", cmd.Selector).Panic("no shards match selector") 50 | } 51 | writeHoistedYAMLShardSpace(buf, resp) 52 | 53 | return buf 54 | } 55 | 56 | func (cmd *cmdShardsEdit) applyShardSpecYAML(b []byte, shardClient pc.ShardClient, journalClient pb.JournalClient) error { 57 | var set shardspace.Set 58 | if err := yaml.UnmarshalStrict(b, &set); err != nil { 59 | return err 60 | } 61 | 62 | var ctx = context.Background() 63 | var req = newShardSpecApplyRequest(set) 64 | 65 | if err := req.Validate(); err != nil { 66 | return err 67 | } else if err = consumer.VerifyReferencedJournals(ctx, journalClient, req); err != nil { 68 | return errors.WithMessage(err, "verifying referenced journals") 69 | } 70 | 71 | var resp, err = consumer.ApplyShardsInBatches(ctx, shardClient, req, cmd.MaxTxnSize) 72 | mbp.Must(err, "failed to apply shards") 73 | log.WithField("rev", resp.Header.Etcd.Revision).Info("successfully applied") 74 | return nil 75 | } 76 | -------------------------------------------------------------------------------- /cmd/gazctl/gazctlcmd/shards_recover.go: -------------------------------------------------------------------------------- 1 | package gazctlcmd 2 | 3 | import ( 4 | "context" 5 | "os" 6 | "os/signal" 7 | "syscall" 8 | 9 | log "github.com/sirupsen/logrus" 10 | "go.gazette.dev/core/broker/client" 11 | pb "go.gazette.dev/core/broker/protocol" 12 | "go.gazette.dev/core/consumer" 13 | pc "go.gazette.dev/core/consumer/protocol" 14 | "go.gazette.dev/core/consumer/recoverylog" 15 | mbp "go.gazette.dev/core/mainboilerplate" 16 | "google.golang.org/grpc" 17 | ) 18 | 19 | type cmdShardsRecover struct { 20 | ID string `long:"id" required:"true" description:"Shard ID"` 21 | Dir string `long:"dir" short:"d" required:"true" description:"Directory to write the played recovery log into"` 22 | } 23 | 24 | func init() { 25 | CommandRegistry.AddCommand("shards", "recover", "Recover the latest checkpoint from shard", ` 26 | Recover the latest checkpoint of a shard by playing its recoverylog. 27 | 28 | Given a shard name, reads the shard recovery logs and plays them using a recoverylog.Player, 29 | writing the played logs into a chosen directory. 30 | 31 | Examples: 32 | 33 | # Play a recovery log into logs directory 34 | gazctl shards recover --id=your/shard/id --dir=path/to/dir 35 | `, &cmdShardsRecover{}) 36 | } 37 | 38 | func (cmd *cmdShardsRecover) Execute([]string) error { 39 | startup(ShardsCfg.BaseConfig) 40 | 41 | // Install a signal handler which cancels a top-level |ctx|. 42 | var signalCh = make(chan os.Signal, 1) 43 | signal.Notify(signalCh, syscall.SIGTERM, syscall.SIGINT) 44 | 45 | var ctx, cancel = context.WithCancel(context.Background()) 46 | go func() { 47 | <-signalCh 48 | cancel() 49 | }() 50 | 51 | var shardClient = ShardsCfg.Consumer.MustShardClient(ctx) 52 | var shardResp, err = shardClient.List(pb.WithDispatchDefault(ctx), &pc.ListRequest{ 53 | Selector: pb.LabelSelector{ 54 | Include: pb.MustLabelSet("id", cmd.ID), 55 | }, 56 | }, grpc.WaitForReady(true)) 57 | mbp.Must(err, "failed to fetch shard spec") 58 | if len(shardResp.Shards) != 1 { 59 | log.Fatal("no shard exists with the given id") 60 | } 61 | 62 | hintResp, err := consumer.FetchHints(ctx, shardClient, &pc.GetHintsRequest{ 63 | Shard: pc.ShardID(cmd.ID), 64 | }) 65 | mbp.Must(err, "failed to fetch hints for shard") 66 | 67 | log.WithField("hints", hintResp).Debug("fetched shard recovery hints") 68 | 69 | var recoveryLog = shardResp.Shards[0].Spec.RecoveryLog() 70 | var hints = consumer.PickFirstHints(hintResp, recoveryLog) 71 | var rjc = ShardsCfg.Broker.MustRoutedJournalClient(ctx) 72 | var ajc = client.NewAppendService(ctx, rjc) 73 | var player = recoverylog.NewPlayer() 74 | player.FinishAtWriteHead() 75 | err = player.Play(ctx, hints, cmd.Dir, ajc) 76 | mbp.Must(err, "failed to play recoverylog") 77 | 78 | return nil 79 | } 80 | -------------------------------------------------------------------------------- /cmd/gazctl/gazctlcmd/shards_unassign.go: -------------------------------------------------------------------------------- 1 | package gazctlcmd 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "time" 7 | 8 | log "github.com/sirupsen/logrus" 9 | pb "go.gazette.dev/core/broker/protocol" 10 | pc "go.gazette.dev/core/consumer/protocol" 11 | ) 12 | 13 | type cmdShardsUnassign struct { 14 | Failed bool `long:"failed" description:"Only remove assignments from failed shards"` 15 | DryRun bool `long:"dry-run" description:"Perform a dry-run, printing matching shards"` 16 | Selector string `long:"selector" short:"l" required:"true" description:"Label Selector query to filter on"` 17 | } 18 | 19 | func init() { 20 | CommandRegistry.AddCommand("shards", "unassign", "Remove a shard assignment", ` 21 | Removes the assignment of a shard from its primary consumer process, causing it 22 | to be shortly rescheduled by the allocator. This can be used to restart a failed 23 | shard, or to move a shard off an overloaded node. 24 | 25 | Use --selector to supply a LabelSelector which constrains the set of returned 26 | shards. Shard selectors support an additional meta-label "id". See the 'shards 27 | list' command for more details about label selectors. 28 | `, &cmdShardsUnassign{}) 29 | } 30 | 31 | func (cmd *cmdShardsUnassign) Execute([]string) (err error) { 32 | startup(ShardsCfg.BaseConfig) 33 | 34 | var ctx, cancel = context.WithTimeout(context.Background(), 10*time.Second) 35 | defer cancel() 36 | var rsc = ShardsCfg.Consumer.MustRoutedShardClient(ctx) 37 | 38 | var listResp = listShards(rsc, cmd.Selector) 39 | if listResp.Status != pc.Status_OK { 40 | return fmt.Errorf("unexpected listShard status: %v", listResp.Status.String()) 41 | } 42 | 43 | // Compute the set of shard IDs which should have assignments removed. 44 | var shardIDs []pc.ShardID 45 | 46 | for _, shard := range listResp.Shards { 47 | var isFailed bool 48 | for _, status := range shard.Status { 49 | if status.Code == pc.ReplicaStatus_FAILED { 50 | isFailed = true 51 | } 52 | } 53 | if !cmd.Failed || isFailed { 54 | shardIDs = append(shardIDs, shard.Spec.Id) 55 | } 56 | } 57 | 58 | // Walk the set of filtered shards in batches. 59 | for { 60 | var chunk = len(shardIDs) 61 | 62 | if chunk == 0 { 63 | break 64 | } else if chunk > 100 { 65 | chunk = 100 66 | } 67 | 68 | var resp, err = rsc.Unassign(pb.WithDispatchDefault(ctx), &pc.UnassignRequest{ 69 | Shards: shardIDs[:chunk], 70 | OnlyFailed: cmd.Failed, 71 | DryRun: cmd.DryRun, 72 | }) 73 | shardIDs = shardIDs[chunk:] 74 | 75 | if err != nil { 76 | return fmt.Errorf("unassigning shard: %w", err) 77 | } else if err := resp.Validate(); err != nil { 78 | return fmt.Errorf("invalid response: %w", err) 79 | } 80 | 81 | for _, shardId := range resp.Shards { 82 | for _, origShard := range listResp.Shards { 83 | if shardId == origShard.Spec.Id { 84 | log.Infof("Successfully unassigned shard: id=%v. Previous status=%v, previous route members=%v", origShard.Spec.Id.String(), origShard.Status, origShard.Route.Members) 85 | } 86 | } 87 | } 88 | } 89 | 90 | return nil 91 | } 92 | -------------------------------------------------------------------------------- /cmd/gazctl/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "github.com/jessevdk/go-flags" 5 | 6 | "go.gazette.dev/core/cmd/gazctl/gazctlcmd" 7 | mbp "go.gazette.dev/core/mainboilerplate" 8 | ) 9 | 10 | const iniFilename = "gazctl.ini" 11 | 12 | func main() { 13 | parser := flags.NewParser(nil, flags.Default) 14 | 15 | mbp.AddPrintConfigCmd(parser, iniFilename) 16 | mbp.Must(gazctlcmd.AddCmdAttachUUIDs(parser.Command), "could not add attach-uuids subcommand") 17 | 18 | parser.LongDescription = `gazctl is a tool for interacting with Gazette brokers and consumer applications. 19 | 20 | See --help pages of each sub-command for documentation and usage examples. 21 | Optionally configure gazctl with a '` + iniFilename + `' file in the current working directory, 22 | or with '~/.config/gazette/` + iniFilename + `'. Use the 'print-config' sub-command to inspect 23 | the tool's current configuration. 24 | ` 25 | 26 | // Create these journals and shards commands to contain sub-commands 27 | _ = mustAddCmd(parser.Command, "journals", "Interact with broker journals", "", gazctlcmd.JournalsCfg) 28 | _ = mustAddCmd(parser.Command, "shards", "Interact with consumer shards", "", gazctlcmd.ShardsCfg) 29 | 30 | // Add all registered commands to the root parser.Command 31 | mbp.Must(gazctlcmd.CommandRegistry.AddCommands("", parser.Command, true), "could not add subcommand") 32 | 33 | // Parse config and start app 34 | mbp.MustParseConfig(parser, iniFilename) 35 | } 36 | 37 | func mustAddCmd(cmd *flags.Command, name, short, long string, cfg interface{}) *flags.Command { 38 | cmd, err := cmd.AddCommand(name, short, long, cfg) 39 | mbp.Must(err, "failed to add command") 40 | return cmd 41 | } 42 | -------------------------------------------------------------------------------- /consumer/consumer_stub_test.go: -------------------------------------------------------------------------------- 1 | package consumer 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/stretchr/testify/require" 7 | pb "go.gazette.dev/core/broker/protocol" 8 | pc "go.gazette.dev/core/consumer/protocol" 9 | "go.gazette.dev/core/server" 10 | "go.gazette.dev/core/task" 11 | ) 12 | 13 | // shardServerStub stubs the read and write loops of ShardServer RPCs. 14 | // C.f. teststub.Broker 15 | type shardServerStub struct { 16 | t require.TestingT 17 | tasks *task.Group 18 | srv *server.Server 19 | 20 | StatFunc func(context.Context, *pc.StatRequest) (*pc.StatResponse, error) 21 | ListFunc func(context.Context, *pc.ListRequest) (*pc.ListResponse, error) 22 | ApplyFunc func(context.Context, *pc.ApplyRequest) (*pc.ApplyResponse, error) 23 | GetHintsFunc func(context.Context, *pc.GetHintsRequest) (*pc.GetHintsResponse, error) 24 | UnassignFunc func(context.Context, *pc.UnassignRequest) (*pc.UnassignResponse, error) 25 | } 26 | 27 | // newShardServerStub returns a shardServerStub instance served by a local GRPC server. 28 | func newShardServerStub(t require.TestingT) *shardServerStub { 29 | var s = &shardServerStub{ 30 | t: t, 31 | tasks: task.NewGroup(context.Background()), 32 | srv: server.MustLoopback(), 33 | } 34 | pc.RegisterShardServer(s.srv.GRPCServer, s) 35 | s.srv.QueueTasks(s.tasks) 36 | s.tasks.GoRun() 37 | return s 38 | } 39 | 40 | // Client returns a JournalClient wrapping the GRPCLoopback. 41 | func (s *shardServerStub) client() pc.ShardClient { return pc.NewShardClient(s.srv.GRPCLoopback) } 42 | 43 | // Endpoint returns the server Endpoint. 44 | func (s *shardServerStub) endpoint() pb.Endpoint { return s.srv.Endpoint() } 45 | 46 | // cleanup cancels the shardServerStub task.Group and asserts that it exits cleanly. 47 | func (s *shardServerStub) cleanup() { 48 | s.tasks.Cancel() 49 | s.srv.BoundedGracefulStop() 50 | require.NoError(s.t, s.srv.GRPCLoopback.Close()) 51 | require.NoError(s.t, s.tasks.Wait()) 52 | } 53 | 54 | // Stat implements the shardServerStub interface by proxying through StatFunc. 55 | func (s *shardServerStub) Stat(ctx context.Context, req *pc.StatRequest) (*pc.StatResponse, error) { 56 | return s.StatFunc(ctx, req) 57 | } 58 | 59 | // List implements the shardServerStub interface by proxying through ListFunc. 60 | func (s *shardServerStub) List(ctx context.Context, req *pc.ListRequest) (*pc.ListResponse, error) { 61 | return s.ListFunc(ctx, req) 62 | } 63 | 64 | // Apply implements the shardServerStub interface by proxying through ApplyFunc. 65 | func (s *shardServerStub) Apply(ctx context.Context, req *pc.ApplyRequest) (*pc.ApplyResponse, error) { 66 | return s.ApplyFunc(ctx, req) 67 | } 68 | 69 | // GetHints implements the shardServerStub interface by proxying through GetHintsFunc. 70 | func (s *shardServerStub) GetHints(ctx context.Context, req *pc.GetHintsRequest) (*pc.GetHintsResponse, error) { 71 | return s.GetHintsFunc(ctx, req) 72 | } 73 | 74 | // Unassign implements the shardServerStub interface by proxying through UnassignFunc. 75 | func (s *shardServerStub) Unassign(ctx context.Context, req *pc.UnassignRequest) (*pc.UnassignResponse, error) { 76 | return s.UnassignFunc(ctx, req) 77 | } 78 | -------------------------------------------------------------------------------- /consumer/key_space_test.go: -------------------------------------------------------------------------------- 1 | package consumer 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/stretchr/testify/require" 7 | "go.gazette.dev/core/allocator" 8 | pc "go.gazette.dev/core/consumer/protocol" 9 | "go.gazette.dev/core/keyspace" 10 | ) 11 | 12 | func TestShardConsistencyCases(t *testing.T) { 13 | var status, primaryStatus = new(pc.ReplicaStatus), &pc.ReplicaStatus{Code: pc.ReplicaStatus_PRIMARY} 14 | var asn = keyspace.KeyValue{Decoded: allocator.Assignment{Slot: 1, AssignmentValue: status}} 15 | var all = keyspace.KeyValues{asn, {Decoded: allocator.Assignment{Slot: 0, AssignmentValue: primaryStatus}}} 16 | 17 | require.False(t, ShardIsConsistent(allocator.Item{}, asn, all)) 18 | status.Code = pc.ReplicaStatus_STANDBY 19 | require.True(t, ShardIsConsistent(allocator.Item{}, asn, all)) 20 | status.Code = pc.ReplicaStatus_PRIMARY 21 | require.True(t, ShardIsConsistent(allocator.Item{}, asn, all)) 22 | 23 | // If we're FAILED, we're consistent only if the primary is also. 24 | status.Code = pc.ReplicaStatus_FAILED 25 | require.False(t, ShardIsConsistent(allocator.Item{}, asn, all)) 26 | primaryStatus.Code = pc.ReplicaStatus_FAILED 27 | require.True(t, ShardIsConsistent(allocator.Item{}, asn, all)) 28 | } 29 | -------------------------------------------------------------------------------- /consumer/protocol/checkpoint_extensions.go: -------------------------------------------------------------------------------- 1 | package protocol 2 | 3 | import ( 4 | pb "go.gazette.dev/core/broker/protocol" 5 | "go.gazette.dev/core/message" 6 | ) 7 | 8 | // BuildCheckpointArgs are arguments of BuildCheckpoint. 9 | type BuildCheckpointArgs struct { 10 | ReadThrough pb.Offsets 11 | ProducerStates []message.ProducerState 12 | AckIntents []message.AckIntent 13 | } 14 | 15 | // BuildCheckpoint builds a Checkpoint message instance from the arguments. 16 | func BuildCheckpoint(args BuildCheckpointArgs) Checkpoint { 17 | var cp = Checkpoint{ 18 | Sources: make(map[pb.Journal]Checkpoint_Source, len(args.ReadThrough)), 19 | AckIntents: make(map[pb.Journal][]byte, len(args.AckIntents)), 20 | } 21 | for j, o := range args.ReadThrough { 22 | cp.Sources[j] = Checkpoint_Source{ReadThrough: o} 23 | } 24 | for _, p := range args.ProducerStates { 25 | var source = cp.Sources[p.Journal] 26 | source.Producers = append(source.Producers, 27 | Checkpoint_Source_ProducerEntry{ 28 | Id: append([]byte(nil), p.Producer[:]...), 29 | State: Checkpoint_ProducerState{ 30 | LastAck: p.LastAck, 31 | Begin: p.Begin, 32 | }, 33 | }) 34 | // Update because |source| is by-value. 35 | cp.Sources[p.Journal] = source 36 | } 37 | for _, ack := range args.AckIntents { 38 | cp.AckIntents[ack.Journal] = ack.Intent 39 | } 40 | return cp 41 | } 42 | 43 | // FlattenProducerStates returns a []ProducerState drawn from the Checkpoint. 44 | func FlattenProducerStates(cp Checkpoint) []message.ProducerState { 45 | var out []message.ProducerState 46 | 47 | for j, s := range cp.Sources { 48 | for _, p := range s.Producers { 49 | var producer message.ProducerID 50 | copy(producer[:], p.Id) 51 | 52 | out = append(out, message.ProducerState{ 53 | JournalProducer: message.JournalProducer{ 54 | Journal: j, 55 | Producer: producer, 56 | }, 57 | LastAck: p.State.LastAck, 58 | Begin: p.State.Begin, 59 | }) 60 | } 61 | } 62 | return out 63 | } 64 | 65 | // FlattenReadThrough returns Offsets drawn from the Checkpoint. 66 | func FlattenReadThrough(cp Checkpoint) pb.Offsets { 67 | var out = make(pb.Offsets, len(cp.Sources)) 68 | for j, s := range cp.Sources { 69 | out[j] = s.ReadThrough 70 | } 71 | return out 72 | } 73 | -------------------------------------------------------------------------------- /consumer/protocol/doc.go: -------------------------------------------------------------------------------- 1 | // Package protocol defines the consumer datamodel, validation behaviors, 2 | // and gRPC APIs which are shared across clients and consumer application servers. 3 | // Datamodel types and APIs are implemented as generated protobuf messages and 4 | // stubs, typically extended with additional parsing, validation, and shared 5 | // implementation behaviors. A central goal of this package to be highly 6 | // exacting in defining allowed "shapes" that types & messages may take (through 7 | // implementations of the Validator interface), providing strong guarantees to 8 | // consumers and clients that messages are well-formed without need for additional 9 | // ad-hoc, repetitive checks (which often become a maintenance burden). 10 | // 11 | // By convention, this package is usually imported as `pc`, short for 12 | // "Protocol of Consumer", due to it's ubiquity and to distinguish it from 13 | // package go.gazette.dev/core/broker/protocol (imported as `pb`). Eg, 14 | // 15 | // import pc "go.gazette.dev/core/consumer/protocol" 16 | // 17 | package protocol 18 | -------------------------------------------------------------------------------- /consumer/protocol/protocol_gateway.yaml: -------------------------------------------------------------------------------- 1 | type: google.api.Service 2 | config_version: 3 3 | 4 | http: 5 | rules: 6 | - selector: consumer.Shard.List 7 | post: /v1/shards/list 8 | body: "*" 9 | - selector: consumer.Shard.Stat 10 | post: /v1/shards/stat 11 | body: "*" 12 | -------------------------------------------------------------------------------- /consumer/recoverylog/doc.go: -------------------------------------------------------------------------------- 1 | // Package recoverylog specifies a finite state machine for recording 2 | // and replaying observed filesystem operations into a Gazette journal. 3 | package recoverylog 4 | 5 | import ( 6 | "github.com/prometheus/client_golang/prometheus" 7 | "github.com/prometheus/client_golang/prometheus/promauto" 8 | ) 9 | 10 | var ( 11 | recoveredBytesTotal = promauto.NewCounter(prometheus.CounterOpts{ 12 | Name: "gazette_recoverylog_recovered_bytes_total", 13 | Help: "Cumulative number of bytes recovered to local disk from recovery logs.", 14 | }) 15 | ) 16 | -------------------------------------------------------------------------------- /consumer/recoverylog/recorded_op_extensions.go: -------------------------------------------------------------------------------- 1 | package recoverylog 2 | 3 | import ( 4 | pb "go.gazette.dev/core/broker/protocol" 5 | ) 6 | 7 | // Validate validates a FSMHints instance. 8 | func (h *FSMHints) Validate() error { 9 | for _, node := range h.LiveNodes { 10 | for _, segment := range node.Segments { 11 | if segment.Log == "" { 12 | // FSMHints.Log is implied value of segments without Log. 13 | segment.Log = h.Log // Temporarily set (note this a stack copy!). 14 | } else if segment.Log == h.Log { 15 | // Segments must not repeat the already-implied FSMHints.Log. 16 | return pb.NewValidationError("contained segment.Log duplicates FSMHints.Log") 17 | } 18 | 19 | if err := segment.Validate(); err != nil { 20 | return pb.ExtendContext(err, "Segment") 21 | } 22 | } 23 | } 24 | return nil 25 | } 26 | -------------------------------------------------------------------------------- /consumer/recoverylog/recorder_afero.go: -------------------------------------------------------------------------------- 1 | package recoverylog 2 | 3 | import ( 4 | "fmt" 5 | "os" 6 | 7 | "github.com/spf13/afero" 8 | ) 9 | 10 | // RecordedAferoFS adapts a Recorder to wrap a afero.Fs instance. 11 | type RecordedAferoFS struct { 12 | *Recorder 13 | afero.Fs 14 | } 15 | 16 | func (r RecordedAferoFS) Create(name string) (afero.File, error) { 17 | var file, err = r.Fs.Create(name) 18 | if err != nil { 19 | return file, err 20 | } 21 | 22 | return &recordedAferoFile{ 23 | FileRecorder: FileRecorder{ 24 | Recorder: r.Recorder, 25 | Fnode: r.Recorder.RecordCreate(name), 26 | }, 27 | File: file, 28 | }, nil 29 | } 30 | 31 | func (r RecordedAferoFS) OpenFile(name string, flags int, perm os.FileMode) (afero.File, error) { 32 | var file, err = r.Fs.OpenFile(name, flags, perm) 33 | if err != nil { 34 | return file, err 35 | } 36 | 37 | if (flags & os.O_RDONLY) != 0 { 38 | return file, err // Not recorded. 39 | } else if (flags & os.O_APPEND) != 0 { 40 | // NOTE(johnny): No current use-case. Could be supported by stating the file here. 41 | return nil, fmt.Errorf("O_APPEND not supported by RecordedAferoFS") 42 | } 43 | return &recordedAferoFile{ 44 | FileRecorder: FileRecorder{ 45 | Recorder: r.Recorder, 46 | Fnode: r.Recorder.RecordOpenFile(name, flags), 47 | }, 48 | File: file, 49 | }, nil 50 | } 51 | 52 | func (r RecordedAferoFS) Remove(name string) error { 53 | if err := r.Fs.Remove(name); err != nil { 54 | return err 55 | } 56 | r.Recorder.RecordRemove(name) 57 | return nil 58 | } 59 | 60 | func (r RecordedAferoFS) RemoveAll(path string) error { 61 | return fmt.Errorf("removeAll not supported by RecordedAferoFS") 62 | } 63 | 64 | func (r RecordedAferoFS) Rename(oldname, newname string) error { 65 | if err := r.Fs.Rename(oldname, newname); err != nil { 66 | return err 67 | } 68 | r.Recorder.RecordRename(oldname, newname) 69 | return nil 70 | } 71 | 72 | // recordedAferoFile adapts a FileRecorder to wrap a afero.File instance. 73 | type recordedAferoFile struct { 74 | FileRecorder 75 | afero.File 76 | } 77 | 78 | func (r *recordedAferoFile) Write(p []byte) (n int, err error) { 79 | n, err = r.File.Write(p) 80 | r.FileRecorder.RecordWrite(p[:n]) 81 | return 82 | } 83 | 84 | func (r *recordedAferoFile) WriteAt(p []byte, off int64) (n int, err error) { 85 | n, err = r.File.WriteAt(p, off) 86 | r.FileRecorder.RecordWriteAt(p, off) 87 | return 88 | } 89 | 90 | func (r *recordedAferoFile) Seek(offset int64, whence int) (int64, error) { 91 | offset, err := r.File.Seek(offset, whence) 92 | r.FileRecorder.Offset = offset 93 | return offset, err 94 | } 95 | 96 | func (r *recordedAferoFile) Sync() error { 97 | if err := r.File.Sync(); err != nil { 98 | return err 99 | } 100 | var txn = r.Recorder.Barrier(nil) 101 | <-txn.Done() 102 | return txn.Err() 103 | } 104 | 105 | func (r *recordedAferoFile) Truncate(size int64) error { 106 | // NOTE(johnny): the size == 0 case could be implemented as RecordCreate. 107 | return fmt.Errorf("truncate not supported by RecordedAferoFS") 108 | } 109 | 110 | func (r *recordedAferoFile) WriteString(s string) (int, error) { 111 | return r.Write([]byte(s)) 112 | } 113 | -------------------------------------------------------------------------------- /consumer/store-rocksdb/arena_iterator.cpp: -------------------------------------------------------------------------------- 1 | #include 2 | #include 3 | #include 4 | 5 | using rocksdb::Iterator; 6 | using rocksdb::Slice; 7 | 8 | extern "C" { 9 | 10 | // Note: this definition is copied from github.com/facebook/rocksdb/db/c.cc:89 11 | // We must copy, as this struct is defined in a .cc we don't have access too. 12 | struct rocksdb_iterator_t { Iterator* rep; }; 13 | 14 | size_t arena_iter_next(rocksdb_iterator_t* cit, size_t arena_len, char* arena_out); 15 | 16 | } // extern "C" 17 | 18 | // Macro which appends rocksdb::Slice |s| to |arena_out|, prefixed by size. 19 | #define APPEND_SLICE(s) ( \ 20 | { \ 21 | size_t l = s.size(); \ 22 | arena_out[0] = char((l)>>24); \ 23 | arena_out[1] = char((l)>>16); \ 24 | arena_out[2] = char((l)>>8); \ 25 | arena_out[3] = char((l)>>0); \ 26 | memcpy(arena_out + 4, s.data(), l); \ 27 | arena_out = arena_out + 4 + l; \ 28 | arena_off += 4 + l; \ 29 | } \ 30 | ) 31 | 32 | // arena_iter_next fills up to |arena_len| bytes of |arena_out| with successive 33 | // length-prefixed keys and values of |cit|. The input |cit| must be Valid(). 34 | // At return, the last key & value stored in |arena_out| is at the current 35 | // iterator offset. If |arena_len| is insufficient to store the next key, 36 | // then the iterator is still stepped but |arena_out| is left empty. 37 | size_t arena_iter_next(rocksdb_iterator_t* cit, size_t arena_len, char* arena_out) { 38 | 39 | Iterator* it = cit->rep; 40 | size_t arena_off = 0; 41 | 42 | if (!it->Valid()) { 43 | return arena_off; 44 | } 45 | 46 | while(true) { 47 | it->Next(); 48 | 49 | if (!it->Valid()) { 50 | break; 51 | } 52 | 53 | Slice key = it->key(); 54 | Slice val = it->value(); 55 | 56 | if ((arena_off + 8 + key.size() + val.size()) > arena_len) { 57 | // Insufficient arena buffer remains for this key/value. 58 | if (arena_off > 0) { 59 | // At least one key/value is already being returned. 60 | // Step backwards to save this iterator offset for later. 61 | it->Prev(); 62 | } 63 | break; 64 | } 65 | 66 | APPEND_SLICE(key); 67 | APPEND_SLICE(val); 68 | } 69 | return arena_off; 70 | } 71 | -------------------------------------------------------------------------------- /consumer/store-rocksdb/recorder_rocksdb.go: -------------------------------------------------------------------------------- 1 | package store_rocksdb 2 | 3 | import ( 4 | "go.gazette.dev/core/consumer/recoverylog" 5 | ) 6 | 7 | // NewRecorder adapts a recoverylog.Recorder to an EnvObserver. 8 | func NewRecorder(recorder *recoverylog.Recorder) EnvObserver { 9 | return recordedDB{Recorder: recorder} 10 | } 11 | 12 | type recordedDB struct{ *recoverylog.Recorder } 13 | type recordedFile struct{ recoverylog.FileRecorder } 14 | 15 | func (r recordedDB) NewWritableFile(path string) WritableFileObserver { 16 | return &recordedFile{ 17 | FileRecorder: recoverylog.FileRecorder{ 18 | Recorder: r.Recorder, 19 | Fnode: r.Recorder.RecordCreate(path), 20 | }, 21 | } 22 | } 23 | func (r recordedDB) DeleteFile(path string) { r.RecordRemove(path) } 24 | func (r recordedDB) DeleteDir(dirname string) { panic("not supported") } 25 | func (r recordedDB) RenameFile(src, target string) { r.RecordRename(src, target) } 26 | func (r recordedDB) LinkFile(src, target string) { r.RecordLink(src, target) } 27 | 28 | func (r *recordedFile) Append(data []byte) { r.RecordWrite(data) } 29 | func (r *recordedFile) Close() {} // No-op. 30 | func (r *recordedFile) Sync() { <-r.Recorder.Barrier(nil).Done() } 31 | func (r *recordedFile) Fsync() { <-r.Recorder.Barrier(nil).Done() } 32 | func (r *recordedFile) RangeSync(offset, nbytes uint64) { <-r.Recorder.Barrier(nil).Done() } 33 | -------------------------------------------------------------------------------- /consumer/store-rocksdb/store_rocksdb_test.go: -------------------------------------------------------------------------------- 1 | package store_rocksdb 2 | 3 | import ( 4 | "os" 5 | "testing" 6 | 7 | "github.com/stretchr/testify/require" 8 | pb "go.gazette.dev/core/broker/protocol" 9 | pc "go.gazette.dev/core/consumer/protocol" 10 | "go.gazette.dev/core/consumer/recoverylog" 11 | ) 12 | 13 | func TestStoreWriteAndReadKeysAndOffsets(t *testing.T) { 14 | var bk, cleanup = newBrokerAndLog(t) 15 | defer cleanup() 16 | 17 | var fsm, _ = recoverylog.NewFSM(recoverylog.FSMHints{Log: aRecoveryLog}) 18 | var rep = newTestReplica(t, bk) 19 | var recorder = recoverylog.NewRecorder(aRecoveryLog, fsm, rep.author, rep.tmpdir, rep.client) 20 | var store = NewStore(recorder) 21 | require.NoError(t, store.Open()) 22 | 23 | store.WriteBatch.Put([]byte("foo"), []byte("bar")) 24 | store.WriteBatch.Put([]byte("baz"), []byte("bing")) 25 | 26 | require.NoError(t, store.StartCommit(nil, pc.Checkpoint{ 27 | Sources: map[pb.Journal]pc.Checkpoint_Source{ 28 | "journal/A": {ReadThrough: 1234}, 29 | }, 30 | }, nil).Err()) 31 | 32 | r, err := store.DB.Get(store.ReadOptions, []byte("foo")) 33 | require.NoError(t, err) 34 | require.Equal(t, []byte("bar"), r.Data()) 35 | r.Free() 36 | 37 | require.NoError(t, store.StartCommit(nil, pc.Checkpoint{ 38 | Sources: map[pb.Journal]pc.Checkpoint_Source{ 39 | "journal/B": {ReadThrough: 5678}, 40 | }, 41 | }, nil).Err()) 42 | 43 | cp, err := store.RestoreCheckpoint(nil) 44 | require.NoError(t, err) 45 | require.Equal(t, pc.Checkpoint{ 46 | Sources: map[pb.Journal]pc.Checkpoint_Source{ 47 | "journal/B": {ReadThrough: 5678}, 48 | }, 49 | }, cp) 50 | 51 | store.Destroy() 52 | 53 | // Assert the store directory was removed. 54 | _, err = os.Stat(recorder.Dir()) 55 | require.True(t, os.IsNotExist(err)) 56 | } 57 | -------------------------------------------------------------------------------- /consumer/store-sqlite/CMakeLists.txt: -------------------------------------------------------------------------------- 1 | cmake_minimum_required(VERSION 3.14) 2 | project(store_sqlite) 3 | 4 | set(CMAKE_CXX_STANDARD 14) 5 | 6 | include_directories(.) 7 | 8 | add_executable(store_sqlite 9 | store.cpp 10 | store.h) 11 | -------------------------------------------------------------------------------- /consumer/store-sqlite/store.h: -------------------------------------------------------------------------------- 1 | #ifndef VFS_H 2 | #define VFS_H 3 | 4 | #include 5 | #include 6 | 7 | #ifdef __cplusplus 8 | extern "C" { 9 | #endif 10 | 11 | static const int kHeaderSize = 100; // As per sqlite file format spec. 12 | static const int kMinPageSize = 1<<9; // 512 bytes. 13 | 14 | // recFS is a sqlite3_vfs which records mutations of stateful SQLite DB files. 15 | typedef struct recFS recFS; 16 | // recFile is a sqlite3_file which records its mutations. 17 | typedef struct recFile recFile; 18 | 19 | // newRecFS returns a new recFS* of the given registered |name| and path |root|. 20 | sqlite3_vfs *newRecFS(const char *name, const char *root); 21 | // recFSFree frees a recFS instance. 22 | void recFSFree(sqlite3_vfs*); 23 | 24 | // sqlite3_vfs hooks follow: 25 | int recFSOpen(sqlite3_vfs *p, const char *name, sqlite3_file *f, int flags, int *outFlags); 26 | int recFSDelete(sqlite3_vfs *p, const char *name, int dirSync); 27 | int recFSAccess(sqlite3_vfs *p, const char *name, int flags, int *out); 28 | int recFSFullPathname(sqlite3_vfs *p, const char *name, int n, char *out); 29 | 30 | // sqlite3_file hooks follow: 31 | void initRecFile(recFile *rf, sqlite3_vfs *vfs, const char *name); 32 | int recFileClose(sqlite3_file*); 33 | 34 | // configureAsPageFile configures the file to use pageFile* VFS hooks. 35 | void configureAsPageFile(sqlite3_file *f, rocksdb_t *db, rocksdb_column_family_handle_t *cf, size_t target_batch_size); 36 | int pageFileRead(sqlite3_file *f, void *v, int n, sqlite3_int64 offset); 37 | int pageFileWrite(sqlite3_file *f, const void *b, int n, sqlite3_int64 offset); 38 | int pageFileTruncate(sqlite3_file *f, sqlite3_int64 size); 39 | int pageFileSync(sqlite3_file *f, int flags); 40 | int pageFileSize(sqlite3_file *f, sqlite3_int64 *out); 41 | int pageFileControl(sqlite3_file *f, int op, void *pArg); 42 | int pageFileDeviceChar(sqlite3_file *f); 43 | 44 | int pageFileFlush(sqlite3_file *f); 45 | int pageFileRefreshHeader(sqlite3_file *f); 46 | 47 | // configureAsLogFile configures the file to use logFile* VFS hooks. 48 | void configureAsLogFile(sqlite3_file *f, uintptr_t id, int64_t fnode, size_t buffer); 49 | int logFileWrite(sqlite3_file *f, const void *b, int n, sqlite3_int64 offset); 50 | int logFileTruncate(sqlite3_file *f, sqlite3_int64 size); 51 | int logFileSync(sqlite3_file *f, int flags); 52 | int logFileFlush(sqlite3_file *f, sqlite3_int64 next_offset); 53 | 54 | // pageFileHeader is a representation of the SQLite DB header. 55 | typedef struct pageFileHeader { 56 | uint32_t page_size; 57 | uint32_t page_count; 58 | uint32_t change_counter; 59 | uint32_t freelist_head; 60 | uint32_t freelist_count; 61 | } pageFileHeader; 62 | 63 | void parsePageFileHeader(const char *b, pageFileHeader *hdr); 64 | 65 | #ifdef __cplusplus 66 | } /* end extern "C" */ 67 | #endif 68 | 69 | #endif 70 | -------------------------------------------------------------------------------- /consumer/store-sqlite/store_test_support.go: -------------------------------------------------------------------------------- 1 | package store_sqlite 2 | 3 | /* 4 | #include "store.h" 5 | #include 6 | */ 7 | import "C" 8 | import ( 9 | "unsafe" 10 | ) 11 | 12 | // This file contains (very) thin wrappers around store.cpp functions to 13 | // facilitate testing (CGO can't be called from store_test.go directly). 14 | type sqlite3File C.sqlite3_file 15 | 16 | func testRecFSOpen(p *C.sqlite3_vfs, name string, flags int) (*sqlite3File, int) { 17 | var f = (*sqlite3File)(C.malloc(C.ulong(p.szOsFile))) 18 | var cName = C.CString(name) 19 | defer C.free(unsafe.Pointer(cName)) 20 | 21 | var rc = C.recFSOpen(p, cName, (*C.sqlite3_file)(f), C.int(flags), nil) 22 | return f, int(rc) 23 | } 24 | 25 | func testPageFileRead(f *sqlite3File, b []byte, offset int64) int { 26 | return int(C.pageFileRead( 27 | (*C.sqlite3_file)(f), 28 | unsafe.Pointer(&b[0]), 29 | C.int(len(b)), 30 | C.longlong(offset))) 31 | } 32 | 33 | func testPageFileWrite(f *sqlite3File, b []byte, offset int64) int { 34 | return int(C.pageFileWrite( 35 | (*C.sqlite3_file)(f), 36 | unsafe.Pointer(&b[0]), 37 | C.int(len(b)), 38 | C.longlong(offset))) 39 | } 40 | 41 | func testPageFileSync(f *sqlite3File) int { 42 | return int(C.pageFileSync((*C.sqlite3_file)(f), 0)) 43 | } 44 | 45 | func testPageFileSize(f *sqlite3File) (size, code int) { 46 | var out C.longlong 47 | var rc = C.pageFileSize((*C.sqlite3_file)(f), &out) 48 | return int(out), int(rc) 49 | } 50 | 51 | func testPageFileControl(f *sqlite3File, op int) int { 52 | return int(C.pageFileControl((*C.sqlite3_file)(f), C.int(op), nil)) 53 | } 54 | 55 | func testLogFileWrite(f *sqlite3File, b []byte, offset int64) int { 56 | return int(C.logFileWrite( 57 | (*C.sqlite3_file)(f), 58 | unsafe.Pointer(&b[0]), 59 | C.int(len(b)), 60 | C.longlong(offset))) 61 | } 62 | 63 | func testLogFileTruncate(f *sqlite3File, size int64) int { 64 | return int(C.logFileTruncate((*C.sqlite3_file)(f), C.longlong(size))) 65 | } 66 | 67 | func testLogFileSync(f *sqlite3File) int { 68 | return int(C.logFileSync((*C.sqlite3_file)(f), 0)) 69 | } 70 | 71 | func testParsePageFileHeader(b []byte) (pageSize, pageCount, changeCtr, freeHead, freeCount int) { 72 | var hdr C.pageFileHeader 73 | C.parsePageFileHeader((*C.char)(unsafe.Pointer(&b[0])), &hdr) 74 | 75 | pageSize = int(hdr.page_size) 76 | pageCount = int(hdr.page_count) 77 | changeCtr = int(hdr.change_counter) 78 | freeHead = int(hdr.freelist_head) 79 | freeCount = int(hdr.freelist_count) 80 | return 81 | } 82 | -------------------------------------------------------------------------------- /consumer/store-sqlite/testdata/test-multi-db-txn.sql: -------------------------------------------------------------------------------- 1 | 2 | ATTACH DATABASE 'ATTACH_DB1' AS db1; 3 | ATTACH DATABASE 'ATTACH_DB2' AS db2; 4 | 5 | BEGIN; 6 | CREATE TABLE foo (id INTEGER NOT NULL primary key, name text); 7 | CREATE TABLE db1.foo (id INTEGER NOT NULL primary key, name text); 8 | CREATE TABLE db2.foo (id INTEGER NOT NULL primary key, name text); 9 | COMMIT; 10 | 11 | BEGIN; 12 | INSERT INTO foo (id, name) VALUES (0, 'primary DB'); 13 | INSERT INTO db1.foo (id, name) VALUES (1, 'db1 value'); 14 | INSERT INTO db2.foo (id, name) VALUES (2, 'db2 value'); 15 | COMMIT; 16 | 17 | SELECT * FROM foo UNION SELECT * FROM db1.foo UNION SELECT * FROM db2.foo; 18 | 19 | BEGIN; 20 | INSERT INTO foo (id, name) VALUES (10, 'primary DB two'); 21 | INSERT INTO db1.foo (id, name) VALUES (11, 'db1 value two'); 22 | INSERT INTO db2.foo (id, name) VALUES (12, 'db2 value two'); 23 | 24 | CRASH_AND_RECOVER; 25 | ATTACH DATABASE 'ATTACH_DB1' AS db1; 26 | ATTACH DATABASE 'ATTACH_DB2' AS db2; 27 | 28 | SELECT * FROM foo UNION SELECT * FROM db1.foo UNION SELECT * FROM db2.foo; 29 | 30 | -------------------------------------------------------------------------------- /consumer/store-sqlite/testdata/test-simple.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE foo ( 2 | id INTEGER NOT NULL primary key, 3 | name text 4 | ); 5 | 6 | CREATE INDEX fooIdx ON foo (name); 7 | 8 | BEGIN; 9 | INSERT INTO foo (id, name) VALUES (0, 'hello, sqlite 000'); 10 | INSERT INTO foo (id, name) VALUES (1, 'hello, sqlite 001'); 11 | INSERT INTO foo (id, name) VALUES (2, 'hello, sqlite 002'); 12 | INSERT INTO foo (id, name) VALUES (3, 'hello, sqlite 003'); 13 | INSERT INTO foo (id, name) VALUES (4, 'hello, sqlite 004'); 14 | INSERT INTO foo (id, name) VALUES (5, 'hello, sqlite 005'); 15 | COMMIT; 16 | 17 | BEGIN; 18 | INSERT INTO foo (id, name) VALUES (99, 'this is dropped'); 19 | INSERT INTO foo (id, name) VALUES (999, 'so is this'); 20 | CRASH_AND_RECOVER; 21 | 22 | INSERT INTO foo (id, name) VALUES (6, 'hello, sqlite 006'); 23 | INSERT INTO foo (id, name) VALUES (7, 'hello, sqlite 007'); 24 | INSERT INTO foo (id, name) VALUES (8, 'hello, sqlite 008'); 25 | 26 | SELECT id, name FROM foo ORDER BY id DESC; 27 | -------------------------------------------------------------------------------- /consumer/store-sqlite/testdata/test-trim-and-rollback.sql: -------------------------------------------------------------------------------- 1 | PRAGMA cache_size = 2; 2 | PRAGMA cell_size_check = 1; 3 | PRAGMA journal_mode = TRUNCATE; 4 | 5 | CREATE TABLE foo (id INT, name text); 6 | CREATE INDEX fooExpensiveIdx1 ON foo (id, name); 7 | CREATE INDEX fooExpensiveIdx2 ON foo (name, id); 8 | 9 | BEGIN; 10 | INSERT INTO foo 11 | WITH RECURSIVE generator(id, name) 12 | AS ( 13 | SELECT 1, 1 * 1000 14 | UNION ALL 15 | SELECT id+1, (id+1)*100000000 FROM generator 16 | LIMIT 1000 17 | ) 18 | SELECT id, name from generator; 19 | COMMIT; 20 | 21 | DROP INDEX fooExpensiveIdx1; 22 | DROP INDEX fooExpensiveIdx2; 23 | VACUUM; 24 | 25 | BEGIN; 26 | INSERT INTO foo 27 | SELECT id + 1000, name + "-extra" FROM foo; 28 | ROLLBACK; 29 | 30 | SELECT * from foo; 31 | -------------------------------------------------------------------------------- /docs/Makefile: -------------------------------------------------------------------------------- 1 | # Minimal makefile for Sphinx documentation 2 | # 3 | 4 | # You can set these variables from the command line. 5 | SPHINXOPTS = 6 | SPHINXBUILD = sphinx-build 7 | SOURCEDIR = . 8 | BUILDDIR = _build 9 | 10 | # Put it first so that "make" without argument is like "make help". 11 | help: 12 | @$(SPHINXBUILD) -M help "$(SOURCEDIR)" "$(BUILDDIR)" $(SPHINXOPTS) $(O) 13 | 14 | .PHONY: help Makefile 15 | 16 | # Catch-all target: route all unknown targets to Sphinx using the new 17 | # "make mode" option. $(O) is meant as a shortcut for $(SPHINXOPTS). 18 | %: Makefile 19 | @$(SPHINXBUILD) -M $@ "$(SOURCEDIR)" "$(BUILDDIR)" $(SPHINXOPTS) $(O) 20 | -------------------------------------------------------------------------------- /docs/_static/cmd-gazctl-attach-uuids.txt: -------------------------------------------------------------------------------- 1 | Usage: 2 | gazctl [OPTIONS] attach-uuids [attach-uuids-OPTIONS] [Paths...] 3 | 4 | For each line of each argument input file, generate a RFC 4122 v1 compatible 5 | UUID and, using the --template, combine it with the input line into output 6 | written to stdout. If no input file arguments are given, stdin is read instead. 7 | 8 | Exactly-once processing semantics require that messages carry a v1 UUID which 9 | is authored by Gazette. The UUID encodes a unique producer ID, monotonic Clock, 10 | and transaction flags. 11 | 12 | attach-uuids facilitates pre-processing text files or unix pipelines in 13 | preparation for appending to a journal, by associating each input with a 14 | corresponding UUID. UUIDs are flagged as committed, meaning they will be 15 | processed immediately by readers. attach-uuids may be used directly in a 16 | pipeline of streamed records. 17 | 18 | When processing files in preparation for append to Gazette, it's best practice 19 | to attach UUIDs into new temporary file(s), and then append the temporary files 20 | to journals. This ensures messages are processed only once even if one or both 21 | of the attach-uuids or append steps fail partway through and are restarted. 22 | 23 | However avoid appending many small files in this way, as each invocation of 24 | attach-uuids generates a new random producer ID, and each producer ID requires 25 | that consumers track a very small amount of state (eg, its Clock). Instead, 26 | first combine many small files into few large ones before attaching UUIDs. 27 | 28 | Prefix CSV rows with a UUID (using the default --template): 29 | > gazctl attach-uuids inputOne.csv inputTwo.csv inputN.csv 30 | 31 | Prefix CSV rows, but skip a initial header row of each input: 32 | > gazctl attach-uuids --skip-header inputOne.csv inputTwo.csv 33 | 34 | Postfix CSV rows with a UUID (use $'..' to correctly handle newline escape): 35 | > gazctl attach-uuids input.csv --template=$'{{.Line}},{{.UUID}}\n' 36 | 37 | Wrap JSON inputs with a UUID: 38 | > gazctl attach-uuids input.json --template=$'{"uuid": "{{.UUID}}","record":{{.Line}}}\n' 39 | 40 | Optionally compose with "jq" to un-nest the JSON objects: 41 | > gazctl attach-uuids input.json --template=$'{"uuid": "{{.UUID}}","record":{{.Line}}}\n' \ 42 | > | jq -c '{uuid: .uuid} + .record' 43 | 44 | 45 | Help Options: 46 | -h, --help Show this help message 47 | 48 | [attach-uuids command options] 49 | --template= Go text/template for output (default: "{{.UUID}},{{.Line}}\n") 50 | --max-length= Maximum allowed byte-length of an input line (default: 4194304) 51 | --skip-header Omit the first line of each input file 52 | 53 | -------------------------------------------------------------------------------- /docs/_static/cmd-gazctl-journals-prune.txt: -------------------------------------------------------------------------------- 1 | Usage: 2 | gazctl [OPTIONS] journals [journals-OPTIONS] prune [prune-OPTIONS] 3 | 4 | Deletes fragments across all configured fragment stores of matching journals that are older than the configured retention. 5 | 6 | There is a caveat when pruning journals. For a given journal, there could be multiple fragments covering the same offset. These fragments contain 7 | identical data at a given offset, but the brokers are tracking only the largest fragment, i.e. the fragment that covers the largest span of 8 | offsets. As a result, the prune command will delete only this tracked fragment, leaving the smaller fragments untouched. As a workaround, 9 | operators can wait for the fragment listing to refresh and prune the journals again. 10 | 11 | Use --selector to supply a LabelSelector to select journals to prune. 12 | See "journals list --help" for details and examples. 13 | 14 | 15 | Help Options: 16 | -h, --help Show this help message 17 | 18 | [journals command options] 19 | 20 | Interact with broker journals: 21 | --zone= Availability zone within which this process is running (default: local) [$ZONE] 22 | 23 | Logging: 24 | --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] 25 | --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] 26 | 27 | Broker: 28 | --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] 29 | --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] 30 | --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] 31 | --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates 32 | [$BROKER_TRUSTED_CA_FILE] 33 | --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign 34 | Authorization tokens. [$BROKER_AUTH_KEYS] 35 | --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) 36 | (default: 0) [$BROKER_CACHE_SIZE] 37 | --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] 38 | 39 | [prune command options] 40 | -l, --selector= Label Selector query to filter on 41 | --dry-run Perform a dry-run of the apply 42 | 43 | -------------------------------------------------------------------------------- /docs/_static/cmd-gazctl-journals-suspend.txt: -------------------------------------------------------------------------------- 1 | Usage: 2 | gazctl [OPTIONS] journals [journals-OPTIONS] suspend [suspend-OPTIONS] 3 | 4 | Suspend idle journals to use fewer or zero broker replicas. 5 | 6 | Suspension updates the 'suspend' field of the JournalSpec with its suspended 7 | status and its resumption offset. When applying other updates to JournalSpecs, 8 | operators utilizing journal suspension must take care to not overwrite the 9 | journal's suspension configuration. 10 | 11 | Typically this means reading the current JournalSpec and its ModRevision, 12 | copying the current 'suspend' field alongside other changes being made, 13 | and then applying the updated JournalSpec with ExpectModRevision. 14 | 15 | The 'journals edit' subcommand uses this workflow and is safe to use with suspension. 16 | 17 | 18 | Help Options: 19 | -h, --help Show this help message 20 | 21 | [journals command options] 22 | 23 | Interact with broker journals: 24 | --zone= Availability zone within which this process is running (default: local) [$ZONE] 25 | 26 | Logging: 27 | --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] 28 | --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] 29 | 30 | Broker: 31 | --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] 32 | --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] 33 | --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] 34 | --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates 35 | [$BROKER_TRUSTED_CA_FILE] 36 | --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign 37 | Authorization tokens. [$BROKER_AUTH_KEYS] 38 | --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) 39 | (default: 0) [$BROKER_CACHE_SIZE] 40 | --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] 41 | 42 | [suspend command options] 43 | -l, --selector= Label Selector query to filter on 44 | -f, --force Suspend journals even if they have local fragments 45 | 46 | -------------------------------------------------------------------------------- /docs/_static/cmd-gazctl-print-config.txt: -------------------------------------------------------------------------------- 1 | Usage: 2 | gazctl [OPTIONS] print-config 3 | 4 | print-config parses the combined configuration from gazctl.ini, flags, 5 | and environment variables, and then writes the configuration to stdout in INI format. 6 | 7 | 8 | Help Options: 9 | -h, --help Show this help message 10 | 11 | -------------------------------------------------------------------------------- /docs/_static/cmd-gazctl.txt: -------------------------------------------------------------------------------- 1 | Usage: 2 | gazctl [OPTIONS] 3 | 4 | gazctl is a tool for interacting with Gazette brokers and consumer applications. 5 | 6 | See --help pages of each sub-command for documentation and usage examples. 7 | Optionally configure gazctl with a 'gazctl.ini' file in the current working directory, 8 | or with '~/.config/gazette/gazctl.ini'. Use the 'print-config' sub-command to inspect 9 | the tool's current configuration. 10 | 11 | 12 | Help Options: 13 | -h, --help Show this help message 14 | 15 | Available commands: 16 | attach-uuids Generate and attach UUIDs to text input records 17 | journals Interact with broker journals 18 | print-config Print combined configuration and exit 19 | shards Interact with consumer shards 20 | 21 | -------------------------------------------------------------------------------- /docs/changelog.rst: -------------------------------------------------------------------------------- 1 | Changelog 2 | ========== 3 | 4 | .. include:: ../CHANGELOG.rst 5 | -------------------------------------------------------------------------------- /docs/consumers-shardspecs.rst: -------------------------------------------------------------------------------- 1 | ShardSpecs 2 | ============ 3 | 4 | ShardSpecs declare and configure shards managed by a Gazette consumer application. 5 | Users generally work with ShardSpecs in YAML form using the ``gazctl`` tool, 6 | in a analogous way to how Kubernetes resources are managed with ``kubectl``. 7 | 8 | A collection of shards often share configuration. To make them easier 9 | to work with, the ``gazctl`` tool converts to and from a hoisted ``common`` 10 | YAML map for end-user presentation. Each shard having no specific value for a 11 | configuration property inherits from the ``common`` map. 12 | 13 | This conversion happens entirely within the tool -- ShardSpecs sent to or 14 | queried from consumer member processes are full and complete individual 15 | specifications. 16 | 17 | A common operational pattern is to define ShardSpecs that maintain a 18 | one-to-one correspondence with journal partitions of a topic, as is done with 19 | the below example. However ShardSpecs are highly configurable and a variety 20 | of other patterns are also possible. Labels can be used to attach metadata to 21 | ShardSpecs, and applications can interpret labels of the spec to drive customized 22 | processing behaviors. 23 | 24 | Example YAML 25 | -------------- 26 | 27 | .. literalinclude:: ../kustomize/bases/example-word-count/shard_specs.yaml 28 | :language: yaml 29 | 30 | Etcd Revisions 31 | --------------- 32 | 33 | Like JournalSpecs, ShardSpecs retrieved by the ``gazctl`` tool will include their 34 | respective Etcd modification revisions as field ``revision`` within the rendered YAML. 35 | 36 | Consult the JournalSpecs documentation for more detail; ShardSpecs work analogously. 37 | 38 | Deleting ShardSpecs 39 | -------------------- 40 | 41 | Like JournalSpecs, ShardSpecs may be deleted by adding a ``delete: true`` stanza to 42 | the YAML returned by ``gazctl`` and then applying it. 43 | 44 | The field may be applied to an individual spec, or to the hoisted ``common`` map. 45 | Consult the JournalSpecs documentation for more detail. 46 | -------------------------------------------------------------------------------- /docs/examples-stream-sum.rst: -------------------------------------------------------------------------------- 1 | Summing over Multiplexed File Chunks 2 | ==================================== 3 | 4 | This example application implements a SHA-summing consumer which incrementally 5 | sums file "chunks" and publishes their final sum. It's implemented using 6 | the Gazette consumer_ framework. 7 | 8 | The complete implementation can be found in package stream-sum_. 9 | 10 | .. _consumer: https://godoc.org/go.gazette.dev/core/consumer 11 | .. _stream-sum: https://godoc.org/go.gazette.dev/core/examples/stream-sum 12 | 13 | Objectives 14 | ---------- 15 | 16 | A number of "chunker" jobs will randomly create and publish "chunks" of number of files. 17 | Each chunker will publish incremental chunks of many simultaneous files. A "summer" 18 | consumer must accumulate the running SHA-sum of each file and, when the file is 19 | completed, it must emit a final SHA-sum of its content. Each chunker job will 20 | independently verify that a final and correct SHA-sum is published in a timely fashion. 21 | 22 | This is a fully self-contained example, and also serves as a *soak test* for Gazette. 23 | Chunkers and summers run continuously, and each verifies expected guarantees of 24 | Gazette brokers and consumers: that all messages are delivered exactly one time, 25 | within a bounded SLA. If any of these guarantees are violated, the summer or chunker 26 | process will crash with an error. 27 | 28 | These guarantees are further tested by a suite of `crash tests`_ which crash and partition 29 | components of the application, like Etcd, gazette brokers, and consumer processes. 30 | 31 | .. _`crash tests`: https://github.com/gazette/core/blob/master/kustomize/test/bases/crash-tester 32 | 33 | Deploy the Example to Kubernetes 34 | -------------------------------- 35 | 36 | Deploy the example to the ``stream-sum`` namespace of a test Kubernetes cluster, 37 | such as Minikube or Docker for Desktop: 38 | 39 | .. code-block:: console 40 | 41 | $ kubectl apply -k ./kustomize/test/deploy-stream-sum/ 42 | namespace/stream-sum created 43 | serviceaccount/minio created 44 | configmap/etcd-scripts-24t872gm88 created 45 | configmap/example-journal-specs created 46 | configmap/gazette-zonemap created 47 | configmap/generate-shards-stream-sum-9k96chk9cg created 48 | configmap/minio-create-bucket-mm6f469cbt created 49 | configmap/postgres-init created 50 | secret/gazette-aws-credentials created 51 | secret/minio-credentials-fgdm8fkm5m created 52 | service/consumer-stream-sum created 53 | service/etcd created 54 | service/gazette created 55 | service/minio created 56 | deployment.apps/consumer-stream-sum created 57 | deployment.apps/gazette created 58 | deployment.apps/minio created 59 | statefulset.apps/etcd created 60 | job.batch/apply-journal-specs created 61 | job.batch/apply-shard-specs-stream-sum created 62 | job.batch/chunker-stream-sum created 63 | job.batch/minio-create-bucket created 64 | -------------------------------------------------------------------------------- /docs/examples.rst: -------------------------------------------------------------------------------- 1 | Consumer Examples 2 | ================== 3 | 4 | `Finding Cycles in streaming Citi-Bike Rides`_. 5 | We've been asked to help with an anomaly detection task: we want to find cases 6 | where a Citi Bike completes a graph cycle starting and ending at a station **T**, 7 | without re-visiting **T** in between. 8 | 9 | We'll also offer a "history" API which serves the most recent rides of 10 | a given bike ID. 11 | 12 | `Serving a Real-time Language Model`_. 13 | We must offer a real-time language model API which accepts new documents to 14 | add to the corpus, continuously integrates each one, and offers fast query access of 15 | current model states. 16 | 17 | `Summing over Multiplexed File Chunks`_. 18 | We're presented with a stream of multiplexed file chunks, and must compute 19 | and publish the full SHA-sum of each file upon it's completion. 20 | 21 | This example also serves as a *soak test* for Gazette and is used to verify 22 | correctness in automated Jepsen-style fault injection tests. 23 | 24 | `Playing Ping-Pong`_. 25 | Let's play ping pong. *At scale!* This example is a minimal application 26 | which provides opinionated scaffolding for starting a new Gazette consumer 27 | project. It demonstrates: 28 | 29 | - Building & end-to-end testing with RocksDB support. 30 | - Hermetic, Docker-based builds. 31 | - Protobuf code generation. 32 | - Packaging release-ready images. 33 | - Manifests for deploying and testing on Kubernetes. 34 | 35 | .. _`Finding Cycles in streaming Citi-Bike Rides`: examples-bike-share.html 36 | .. _`Serving a Real-time Language Model`: examples-language-model.html 37 | .. _`Summing over Multiplexed File Chunks`: examples-stream-sum.html 38 | .. _`Playing Ping-Pong`: https://github.com/gazette/ping-pong 39 | 40 | .. toctree:: 41 | :maxdepth: 2 42 | :hidden: 43 | 44 | Cycles in Citi-Bike Data 45 | Real-time Language Model 46 | Summing Multiplexed Chunks 47 | 48 | -------------------------------------------------------------------------------- /docs/index.rst: -------------------------------------------------------------------------------- 1 | .. include:: overview-intro.rst 2 | -------------------------------------------------------------------------------- /docs/overview-build-and-test.rst: -------------------------------------------------------------------------------- 1 | Build and Test 2 | =============== 3 | 4 | Most binaries and packages are "pure" Go and can be directly go-installed and go-tested: 5 | 6 | .. code-block:: console 7 | 8 | $ go install go.gazette.dev/core/cmd/gazette 9 | $ go install go.gazette.dev/core/cmd/gazctl 10 | $ go test go.gazette.dev/core/broker/... 11 | $ go test go.gazette.dev/core/consumer 12 | 13 | Certain packages used by consumer applications, like ``go.gazette.dev/core/consumer/store-rocksdb``, 14 | require CGO to build and also require appropriate development libraries for RocksDB. 15 | On Debian and Ubuntu, the ``librocksdb-dev`` and ``libsqlite3-dev`` packages are sufficient, 16 | paired with the ``libsqlite3`` Go build tag. 17 | 18 | 19 | Continuous Integration 20 | ----------------------- 21 | 22 | Gazette uses a :githubsource:`Make-based build system` which pulls down and stages 23 | development dependencies into a ``.build`` sub-directory of the repository root. 24 | 25 | Run CI tests as: 26 | 27 | .. code-block:: console 28 | 29 | $ make go-test-ci 30 | 31 | Continuous integration builds of Gazette run tests 15 times, with race detection enabled. 32 | 33 | A ``go-test-fast`` target is also available, wich runs tests once without race detection. 34 | 35 | Build release binaries for Gazette broker and examples using 36 | ``go-build`` or ``go-build-arm64`` targets. 37 | 38 | 39 | Building the Docs 40 | ------------------ 41 | 42 | To build these docs locally, you'll need a few more dependencies. To start with, you'll need 43 | ``python`` and ``pip``. Note that on some systems, these may be called ``python3`` and ``pip3``. 44 | Next you'll need to install the following python packages using ``pip install --user ``. 45 | 46 | * ``sphinx`` 47 | * ``sphinxcontrib-programoutput`` 48 | * ``sphinx_rtd_theme`` 49 | 50 | Once you have all those installed, you can change directory into ``docs/`` and run ``make html``. 51 | This will write the output to ``docs/_build``, and then you can open any of the html files in your 52 | browser. 53 | 54 | -------------------------------------------------------------------------------- /docs/overview-slides.rst: -------------------------------------------------------------------------------- 1 | Slides (`Direct Link`_) 2 | ======================== 3 | 4 | .. raw:: html 5 | 6 | 7 | 8 | .. _Direct Link: https://docs.google.com/presentation/d/e/2PACX-1vRq8pwusGbcv1KaoedwfvyKydmO-IBvziXaKQhwFpwCSYt5P7Yn4n5_gWD7XBW2feAlvhZ8-YP4h1uF/pub?start=false&loop=false&delayms=3000 9 | -------------------------------------------------------------------------------- /docs/reference-api.rst: -------------------------------------------------------------------------------- 1 | API Reference 2 | ================= 3 | 4 | API docs 5 | --------- 6 | 7 | Gazette API documentation is primarily hosted as godocs_. Packages of note include: 8 | 9 | - client_ for interacting with Gazette brokers. 10 | - consumer_ for core consumer Application interfaces. 11 | - runconsumer_ for opinionated initialization of consumer applications. 12 | 13 | .. _godocs: https://godoc.org/go.gazette.dev/core 14 | .. _client: https://godoc.org/go.gazette.dev/core/broker/client 15 | .. _consumer: https://godoc.org/go.gazette.dev/core/consumer 16 | .. _runconsumer: https://godoc.org/go.gazette.dev/core/mainboilerplate/runconsumer 17 | 18 | Broker Protocol Buffers & gRPC Service 19 | --------------------------------------- 20 | 21 | Browse `broker/protocol.proto`_ on Sourcegraph. 22 | 23 | .. _broker/protocol.proto: https://sourcegraph.com/github.com/gazette/core/-/blob/broker/protocol/protocol.proto 24 | 25 | Consumer Protocol Buffers & gRPC Service 26 | ----------------------------------------- 27 | 28 | Browse `consumer/protocol.proto`_ on Sourcegraph. 29 | 30 | .. _consumer/protocol.proto: https://sourcegraph.com/github.com/gazette/core/-/blob/consumer/protocol/protocol.proto 31 | -------------------------------------------------------------------------------- /docs/reference-gazctl.rst: -------------------------------------------------------------------------------- 1 | ``gazctl`` Command 2 | ============================= 3 | 4 | gazctl 5 | --------------------------- 6 | .. literalinclude:: _static/cmd-gazctl.txt 7 | 8 | gazctl attach-uuids 9 | --------------------------- 10 | .. literalinclude:: _static/cmd-gazctl-attach-uuids.txt 11 | 12 | gazctl journals append 13 | --------------------------- 14 | .. literalinclude:: _static/cmd-gazctl-journals-append.txt 15 | 16 | gazctl journals apply 17 | --------------------------- 18 | .. literalinclude:: _static/cmd-gazctl-journals-apply.txt 19 | 20 | gazctl journals edit 21 | --------------------------- 22 | .. literalinclude:: _static/cmd-gazctl-journals-edit.txt 23 | 24 | gazctl journals fragments 25 | --------------------------- 26 | .. literalinclude:: _static/cmd-gazctl-journals-fragments.txt 27 | 28 | gazctl journals list 29 | --------------------------- 30 | .. literalinclude:: _static/cmd-gazctl-journals-list.txt 31 | 32 | gazctl journals prune 33 | --------------------------- 34 | .. literalinclude:: _static/cmd-gazctl-journals-prune.txt 35 | 36 | gazctl journals read 37 | --------------------------- 38 | .. literalinclude:: _static/cmd-gazctl-journals-read.txt 39 | 40 | gazctl journals reset-head 41 | --------------------------- 42 | .. literalinclude:: _static/cmd-gazctl-journals-reset-head.txt 43 | 44 | gazctl journals suspend 45 | --------------------------- 46 | .. literalinclude:: _static/cmd-gazctl-journals-suspend.txt 47 | 48 | gazctl print-config 49 | --------------------------- 50 | .. literalinclude:: _static/cmd-gazctl-print-config.txt 51 | 52 | gazctl shards apply 53 | --------------------------- 54 | .. literalinclude:: _static/cmd-gazctl-shards-apply.txt 55 | 56 | gazctl shards edit 57 | --------------------------- 58 | .. literalinclude:: _static/cmd-gazctl-shards-edit.txt 59 | 60 | gazctl shards list 61 | --------------------------- 62 | .. literalinclude:: _static/cmd-gazctl-shards-list.txt 63 | 64 | gazctl shards prune 65 | --------------------------- 66 | .. literalinclude:: _static/cmd-gazctl-shards-prune.txt 67 | 68 | -------------------------------------------------------------------------------- /docs/reference-gazette.rst: -------------------------------------------------------------------------------- 1 | ``gazette`` Command 2 | =================== 3 | 4 | gazette serve 5 | --------------------------- 6 | .. literalinclude:: _static/cmd-gazette-serve.txt 7 | 8 | gazette print-config 9 | --------------------------- 10 | .. literalinclude:: _static/cmd-gazette-print-config.txt 11 | -------------------------------------------------------------------------------- /docs/requirements.txt: -------------------------------------------------------------------------------- 1 | alabaster==0.7.12 2 | Babel==2.9.1 3 | certifi==2024.7.4 4 | chardet==3.0.4 5 | docutils==0.15.2 6 | idna==3.7 7 | imagesize==1.1.0 8 | Jinja2==3.1.6 9 | MarkupSafe==1.1.1 10 | packaging==19.2 11 | Pygments==2.15.0 12 | pyparsing==2.4.5 13 | pytz==2019.3 14 | requests==2.32.0 15 | six==1.13.0 16 | snowballstemmer==2.0.0 17 | Sphinx==1.8.5 18 | sphinx-rtd-theme==0.4.3 19 | sphinxcontrib-applehelp==1.0.1 20 | sphinxcontrib-devhelp==1.0.1 21 | sphinxcontrib-htmlhelp==1.0.2 22 | sphinxcontrib-jsmath==1.0.1 23 | sphinxcontrib-programoutput==0.15 24 | sphinxcontrib-qthelp==1.0.2 25 | sphinxcontrib-serializinghtml==1.1.3 26 | sphinxcontrib-websupport==1.1.2 27 | urllib3==1.26.19 28 | -------------------------------------------------------------------------------- /docs/table-of-contents.rst: -------------------------------------------------------------------------------- 1 | Gazette Documentation 2 | ====================== 3 | 4 | .. toctree:: 5 | :maxdepth: 2 6 | :caption: Overview 7 | 8 | Introduction 9 | Slides 10 | Build & Test 11 | 12 | .. toctree:: 13 | :maxdepth: 2 14 | :caption: Brokers 15 | 16 | Concepts 17 | JournalSpecs 18 | Tutorial 19 | 20 | .. toctree:: 21 | :maxdepth: 2 22 | :caption: Consumers 23 | 24 | Concepts 25 | ShardSpecs 26 | Examples 27 | 28 | .. toctree:: 29 | :maxdepth: 2 30 | :caption: Reference 31 | 32 | reference-gazette 33 | reference-gazctl 34 | reference-api 35 | 36 | .. toctree:: 37 | :maxdepth: 2 38 | :caption: Architecture 39 | 40 | architecture-design-goals-and-nongoals 41 | architecture-exactly-once 42 | architecture-transactional-appends 43 | architecture-operational-considerations 44 | changelog 45 | -------------------------------------------------------------------------------- /etcdtest/etcd_default.go: -------------------------------------------------------------------------------- 1 | //go:build !linux 2 | // +build !linux 3 | 4 | package etcdtest 5 | 6 | import "syscall" 7 | 8 | var getSysProcAttr = func() *syscall.SysProcAttr { 9 | res := new(syscall.SysProcAttr) 10 | return res 11 | } 12 | -------------------------------------------------------------------------------- /etcdtest/etcd_linux.go: -------------------------------------------------------------------------------- 1 | //go:build linux 2 | // +build linux 3 | 4 | package etcdtest 5 | 6 | import "syscall" 7 | 8 | var getSysProcAttr = func() *syscall.SysProcAttr { 9 | // If this process dies (e.x, due to an uncaught panic from a 10 | // test timeout), deliver a SIGTERM to the `etcd` process). 11 | // This ensures a wrapping `go test` doesn't hang forever awaiting 12 | // the `etcd` child to exit. 13 | res := new(syscall.SysProcAttr) 14 | res.Pdeathsig = syscall.SIGTERM 15 | return res 16 | } 17 | -------------------------------------------------------------------------------- /examples/bike-share/bike-share/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "go.gazette.dev/core/examples/bike-share" 5 | "go.gazette.dev/core/mainboilerplate/runconsumer" 6 | ) 7 | 8 | func main() { runconsumer.Main(new(bike_share.Application)) } 9 | -------------------------------------------------------------------------------- /examples/doc.go: -------------------------------------------------------------------------------- 1 | // Package examples parents packages which implement example Gazette consumer applications. 2 | package examples 3 | -------------------------------------------------------------------------------- /examples/generate-zonemap-gke.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # This script expects stdin to be the JSON output of Kubernetes nodes, eg: 4 | # kubectl get nodes -o json 5 | 6 | # enumerateCases outputs shell switch statement cases for each node pool 7 | # of the current kubectl context, like: 8 | # 9 | # my-pool-name-1f57135e-*) echo "us-central1-c" ;; 10 | # my-pool-name-427cf4f2-*) echo "us-central1-a" ;; 11 | # 12 | function enumerateCases { 13 | FILTER=$(mktemp) 14 | 15 | cat > $FILTER </dev/null && break 12 | sleep 1s 13 | done 14 | done 15 | 16 | # Re-joining after failure? 17 | if [ -f /var/run/etcd/member_id ]; then 18 | echo "Re-joining etcd member" 19 | member_id=$(cat /var/run/etcd/member_id) 20 | 21 | # Re-join member. 22 | exec etcd \ 23 | --advertise-client-urls http://$(hostname).${ETCD_SERVICE_NAME}:2379 \ 24 | --data-dir /var/run/etcd/default.etcd \ 25 | --listen-client-urls http://0.0.0.0:2379 \ 26 | --listen-peer-urls http://0.0.0.0:2380 \ 27 | --logger zap \ 28 | --name $(hostname) 29 | fi 30 | 31 | # This member is being bootstrapped. 32 | persist_member_hash & 33 | 34 | # Adding a seed member? 35 | if [ $(member_index) -lt ${MIN_REPLICAS} ]; then 36 | exec etcd \ 37 | --advertise-client-urls http://$(hostname).${ETCD_SERVICE_NAME}:2379 \ 38 | --data-dir /var/run/etcd/default.etcd \ 39 | --initial-advertise-peer-urls http://$(hostname).${ETCD_SERVICE_NAME}:2380 \ 40 | --initial-cluster $(seed_peers) \ 41 | --initial-cluster-state new \ 42 | --initial-cluster-token etcd-cluster-1 \ 43 | --listen-client-urls http://0.0.0.0:2379 \ 44 | --listen-peer-urls http://0.0.0.0:2380 \ 45 | --logger zap \ 46 | --name $(hostname) 47 | fi 48 | 49 | # We're adding a new member to an existing cluster. 50 | # Requires other members are healthy and available. 51 | 52 | echo "Adding new member" 53 | etcdctl ${AUTH_FLAGS} member add $(hostname) \ 54 | --peer-urls http://$(hostname).${ETCD_SERVICE_NAME}:2380 \ 55 | --command-timeout=300s \ 56 | --debug | 57 | grep "^ETCD_" >/var/run/etcd/new_member_envs 58 | 59 | if [ $? -ne 0 ]; then 60 | echo "Failed to add member." 61 | exit 1 62 | fi 63 | 64 | cat /var/run/etcd/new_member_envs 65 | . /var/run/etcd/new_member_envs 66 | 67 | exec etcd \ 68 | --advertise-client-urls http://$(hostname).${ETCD_SERVICE_NAME}:2379 \ 69 | --data-dir /var/run/etcd/default.etcd \ 70 | --initial-advertise-peer-urls http://$(hostname).${ETCD_SERVICE_NAME}:2380 \ 71 | --initial-cluster ${ETCD_INITIAL_CLUSTER} \ 72 | --initial-cluster-state ${ETCD_INITIAL_CLUSTER_STATE} \ 73 | --listen-client-urls http://0.0.0.0:2379 \ 74 | --listen-peer-urls http://0.0.0.0:2380 \ 75 | --logger zap \ 76 | --name $(hostname) 77 | -------------------------------------------------------------------------------- /kustomize/bases/etcd/etcd-lib.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh -ex 2 | 3 | readonly ETCD_SERVICE_NAME="${ETCD_SERVICE_NAME?ETCD_SERVICE_NAME is required}" 4 | readonly MIN_REPLICAS="${MIN_REPLICAS?MIN_REPLICAS is required}" 5 | readonly AUTH_FLAGS="${AUTH_FLAGS?AUTH_FLAGS is required}" 6 | 7 | # Compose endpoint list of the MIN_REPLICAS seed cluster. 8 | seed_endpoints() { 9 | lst="" 10 | for i in $(seq 0 $((MIN_REPLICAS - 1))); do 11 | lst="${lst}${lst:+,}http://${ETCD_SERVICE_NAME}-${i}.${ETCD_SERVICE_NAME}:2379" 12 | done 13 | echo "${lst}" 14 | } 15 | 16 | # Compose peer list of the MIN_REPLICAS seed cluster. 17 | seed_peers() { 18 | lst="" 19 | for i in $(seq 0 $((MIN_REPLICAS - 1))); do 20 | lst="${lst}${lst:+,}${ETCD_SERVICE_NAME}-${i}=http://${ETCD_SERVICE_NAME}-${i}.${ETCD_SERVICE_NAME}:2380" 21 | done 22 | echo "${lst}" 23 | } 24 | 25 | # Select the member hash ID of this host from amoung the current member list. 26 | member_hash() { 27 | etcdctl ${AUTH_FLAGS} member list | grep "http://$(hostname).${ETCD_SERVICE_NAME}:2380" | cut -d',' -f1 28 | } 29 | 30 | # Persist member hash ID into the persistent volume for future member restart. 31 | persist_member_hash() { 32 | while ! etcdctl ${AUTH_FLAGS} member list; do sleep 1; done 33 | member_hash >/var/run/etcd/member_id 34 | } 35 | 36 | member_index() { 37 | readonly h=$(hostname) 38 | echo ${h##*[^0-9]} 39 | } 40 | -------------------------------------------------------------------------------- /kustomize/bases/etcd/etcd-shutdown.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh -ex 2 | 3 | . /opt/etcd/etcd-lib.sh 4 | export ETCDCTL_ENDPOINTS="$(seed_endpoints)" 5 | 6 | # Remove this member from the cluster peer set. 7 | if [ $(member_index) -ge ${MIN_REPLICAS} ]; then 8 | echo "Removing $(hostname) from etcd cluster." 9 | if etcdctl ${AUTH_FLAGS} member remove $(member_hash); then 10 | # Remove everything otherwise the cluster will no longer scale-up 11 | rm -rf /var/run/etcd/* 12 | fi 13 | fi 14 | -------------------------------------------------------------------------------- /kustomize/bases/etcd/kustomization.yaml: -------------------------------------------------------------------------------- 1 | # Etcd deploys Etcd as a StatefulSet backed by PersistentVolumeClaims for durability. 2 | # 3 | # MIN_REPLICAS defines the "seed" size of the cluster, and the StatefulSet 4 | # expects to always run with at least that many replicas. More replicas may 5 | # be added by scaling the StatefulSet, and each new replica will dynamically 6 | # join (or leave) the cluster on scale-up (or down). It's recommended to 7 | # scale by one member at a time. 8 | # 9 | # TODO(johnny): Etcd v3.4.1, used by this manifest, has a known scale-up bug: 10 | # https://github.com/etcd-io/etcd/issues/11186 11 | resources: 12 | - statefulset.yaml 13 | - service.yaml 14 | 15 | vars: 16 | - name: ETCD_SERVICE_NAME 17 | objref: 18 | kind: StatefulSet 19 | name: etcd 20 | apiVersion: apps/v1 21 | fieldref: 22 | fieldpath: spec.serviceName 23 | 24 | configMapGenerator: 25 | - name: etcd-scripts 26 | files: 27 | - etcd-bootstrap.sh 28 | - etcd-lib.sh 29 | - etcd-shutdown.sh 30 | 31 | -------------------------------------------------------------------------------- /kustomize/bases/etcd/service.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: v1 2 | kind: Service 3 | metadata: 4 | name: etcd 5 | labels: 6 | app.kubernetes.io/name: etcd 7 | spec: 8 | ports: 9 | - port: 2380 10 | name: etcd-server 11 | - port: 2379 12 | name: etcd-client 13 | clusterIP: None 14 | selector: 15 | app.kubernetes.io/name: etcd 16 | publishNotReadyAddresses: true -------------------------------------------------------------------------------- /kustomize/bases/etcd/statefulset.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: apps/v1 2 | kind: StatefulSet 3 | metadata: 4 | name: etcd 5 | labels: 6 | app.kubernetes.io/name: etcd 7 | spec: 8 | replicas: 3 9 | serviceName: etcd 10 | selector: 11 | matchLabels: 12 | app.kubernetes.io/name: etcd 13 | template: 14 | metadata: 15 | name: etcd 16 | labels: 17 | app.kubernetes.io/name: etcd 18 | spec: 19 | containers: 20 | - name: etcd 21 | image: quay.io/coreos/etcd:v3.4.1 22 | imagePullPolicy: IfNotPresent 23 | ports: 24 | - containerPort: 2380 25 | name: peer 26 | - containerPort: 2379 27 | name: client 28 | env: 29 | # Optional Etcd authentication flags. 30 | - name: AUTH_FLAGS 31 | value: "" 32 | # Enable auto-compaction to prune away old key version. 33 | - name: ETCD_AUTO_COMPACTION_RETENTION 34 | value: "1" 35 | # DNS service name of this StatefulSet, reflected from its serviceName. 36 | - name: ETCD_SERVICE_NAME 37 | value: $(ETCD_SERVICE_NAME) 38 | # Smallest number of replicas we may run with. This set is always 39 | # expected to be running, and seeds the cluster. 40 | - name: MIN_REPLICAS 41 | value: "3" 42 | 43 | volumeMounts: 44 | - name: datadir 45 | mountPath: /var/run/etcd 46 | - name: etcd-scripts 47 | mountPath: /opt/etcd 48 | command: 49 | - /bin/sh 50 | - "-e" 51 | - /opt/etcd/etcd-bootstrap.sh 52 | lifecycle: 53 | preStop: 54 | exec: 55 | command: 56 | - /bin/sh 57 | - "-e" 58 | - /opt/etcd/etcd-shutdown.sh 59 | volumes: 60 | - name: etcd-scripts 61 | configMap: 62 | name: etcd-scripts 63 | 64 | volumeClaimTemplates: 65 | - metadata: 66 | name: datadir 67 | spec: 68 | accessModes: 69 | - ReadWriteOnce 70 | resources: 71 | requests: 72 | storage: "256Mi" 73 | -------------------------------------------------------------------------------- /kustomize/bases/example-bike-share/apply-shard-specs.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: batch/v1 2 | kind: Job 3 | metadata: 4 | name: apply-shard-specs 5 | spec: 6 | template: 7 | spec: 8 | restartPolicy: OnFailure 9 | containers: 10 | - name: apply-specs 11 | image: gazette/broker 12 | imagePullPolicy: IfNotPresent 13 | command: 14 | - gazctl 15 | - shards 16 | - apply 17 | - "--specs=/config/shard_specs.yaml" 18 | env: 19 | - name: BROKER_ADDRESS 20 | value: "http://gazette:8080" 21 | - name: CONSUMER_ADDRESS 22 | value: "http://consumer-bike-share:8080" 23 | volumeMounts: 24 | - name: shard-specs 25 | mountPath: /config 26 | volumes: 27 | - name: shard-specs 28 | configMap: 29 | name: shard-specs 30 | -------------------------------------------------------------------------------- /kustomize/bases/example-bike-share/kustomization.yaml: -------------------------------------------------------------------------------- 1 | # example-bike-share deploys the Gazette bike-share example, consisting of: 2 | # * A "bike-share" Deployment and Service, which finds graph cycles and serves bike history. 3 | # * A Job to create ShardSpecs for journals of the test environment. 4 | # * A Job to stream & load a portion of the dataset. 5 | bases: 6 | - ../consumer 7 | patchesStrategicMerge: 8 | - patches.yaml 9 | resources: 10 | - apply-shard-specs.yaml 11 | - stream-rides.yaml 12 | 13 | nameSuffix: -bike-share 14 | 15 | images: 16 | - name: consumer/image 17 | newName: gazette/examples 18 | 19 | configMapGenerator: 20 | - name: shard-specs 21 | files: 22 | - shard_specs.yaml 23 | - name: stream-rides 24 | files: 25 | - stream_rides.sh -------------------------------------------------------------------------------- /kustomize/bases/example-bike-share/patches.yaml: -------------------------------------------------------------------------------- 1 | # Set app name of consumer Deployment. 2 | # Add --postgres connection argument. 3 | apiVersion: apps/v1 4 | kind: Deployment 5 | metadata: 6 | name: consumer 7 | labels: 8 | app.kubernetes.io/name: bike-share 9 | spec: 10 | replicas: 2 11 | selector: 12 | matchLabels: 13 | app.kubernetes.io/name: bike-share 14 | template: 15 | metadata: 16 | labels: 17 | app.kubernetes.io/name: bike-share 18 | spec: 19 | containers: 20 | - name: consumer 21 | env: 22 | - name: CONSUMER_APP_ARGS 23 | value: "--postgres=postgres://test:test@postgres?sslmode=disable" 24 | --- 25 | # Set app name of consumer Service. 26 | apiVersion: v1 27 | kind: Service 28 | metadata: 29 | name: consumer 30 | labels: 31 | app.kubernetes.io/name: bike-share 32 | spec: 33 | selector: 34 | app.kubernetes.io/name: bike-share 35 | -------------------------------------------------------------------------------- /kustomize/bases/example-bike-share/shard_specs.yaml: -------------------------------------------------------------------------------- 1 | common: 2 | max_txn_duration: 1s 3 | shards: 4 | # First two specs use a remote Postgres DB. 5 | - id: cycles-part-000 6 | labels: 7 | - name: store 8 | value: postgres 9 | sources: 10 | - journal: examples/bike-share/rides/part-000 11 | 12 | - id: cycles-part-001 13 | labels: 14 | - name: store 15 | value: postgres 16 | sources: 17 | - journal: examples/bike-share/rides/part-001 18 | 19 | # Next two specs use an embedded SQLite store. It's weird to do this. 20 | # Generally all shards of an application would use the same store type. 21 | # This example just demonstrates it's possible to do otherwise. 22 | - id: cycles-part-002 23 | labels: 24 | - name: store 25 | value: sqlite 26 | sources: 27 | - journal: examples/bike-share/rides/part-002 28 | hint_prefix: /gazette/hints 29 | recovery_log_prefix: examples/bike-share/recovery-logs 30 | 31 | - id: cycles-part-003 32 | labels: 33 | - name: store 34 | value: sqlite 35 | sources: 36 | - journal: examples/bike-share/rides/part-003 37 | hint_prefix: /gazette/hints 38 | recovery_log_prefix: examples/bike-share/recovery-logs 39 | -------------------------------------------------------------------------------- /kustomize/bases/example-bike-share/stream-rides.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: batch/v1 2 | kind: Job 3 | metadata: 4 | name: stream-rides 5 | spec: 6 | template: 7 | spec: 8 | restartPolicy: OnFailure 9 | containers: 10 | - name: stream-rides 11 | image: ubuntu:18.04 12 | imagePullPolicy: IfNotPresent 13 | command: 14 | - /bin/bash 15 | - "-c" 16 | - | 17 | set -Eeux -o pipefail 18 | 19 | # Install dependencies for downloading & streaming the dataset. 20 | apt-get update -y \ 21 | && apt-get install --no-install-recommends -y \ 22 | pv \ 23 | curl \ 24 | unzip \ 25 | ca-certificates 26 | 27 | # TODO(johnny): Switch to a proper automatic build artifact rather than this one-off binary. 28 | curl -o /usr/local/bin/gazctl https://storage.googleapis.com/tribal-sign-234818.appspot.com/gazctl 29 | chmod +x /usr/local/bin/gazctl 30 | 31 | source /config/stream_rides.sh 32 | env: 33 | - name: BROKER_ADDRESS 34 | value: "http://gazette:8080" 35 | - name: BROKER_CACHE_SIZE 36 | value: "256" 37 | - name: DATASET 38 | value: "201909-citibike-tripdata.csv" 39 | - name: RATE 40 | value: "3000" 41 | volumeMounts: 42 | - name: stream-rides 43 | mountPath: /config 44 | volumes: 45 | - name: stream-rides 46 | configMap: 47 | name: stream-rides 48 | 49 | -------------------------------------------------------------------------------- /kustomize/bases/example-bike-share/stream_rides.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | readonly DATASET=${DATASET:-201909-citibike-tripdata.csv} 4 | readonly RATE=${RATE:-300} 5 | 6 | curl -o ${DATASET}.zip https://s3.amazonaws.com/tripdata/${DATASET}.zip 7 | 8 | # Run a pipeline that unpacks CSV records, prefixes each with a UUID, 9 | # partitions on Bike ID, rate-limited to $RATE, and appends each 10 | # record to a modulo-mapped journal. 11 | unzip -p ${DATASET}.zip \ 12 | | gazctl attach-uuids --skip-header \ 13 | | awk -F "," '{print $13}{print}' \ 14 | | pv --line-mode --quiet --rate-limit ${RATE} \ 15 | | gazctl journals append -l app.gazette.dev/message-type=bike_share.Ride --framing=lines --mapping=modulo 16 | -------------------------------------------------------------------------------- /kustomize/bases/example-stream-sum/apply-shard-specs.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: batch/v1 2 | kind: Job 3 | metadata: 4 | name: apply-shard-specs 5 | spec: 6 | template: 7 | spec: 8 | restartPolicy: OnFailure 9 | containers: 10 | - name: apply-specs 11 | image: gazette/broker 12 | imagePullPolicy: IfNotPresent 13 | command: 14 | - /bin/sh 15 | - "-ec" 16 | - | 17 | /config/generate-shards.sh | \ 18 | gazctl shards apply 19 | env: 20 | - name: BROKER_ADDRESS 21 | value: "http://gazette:8080" 22 | - name: CONSUMER_ADDRESS 23 | value: "http://consumer-stream-sum:8080" 24 | volumeMounts: 25 | - name: generate-shards 26 | mountPath: /config 27 | volumes: 28 | - name: generate-shards 29 | configMap: 30 | name: generate-shards 31 | items: 32 | - key: generate-shards.sh 33 | path: generate-shards.sh 34 | mode: 0777 35 | -------------------------------------------------------------------------------- /kustomize/bases/example-stream-sum/chunker.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: batch/v1 2 | kind: Job 3 | metadata: 4 | name: chunker 5 | labels: 6 | app.kubernetes.io/name: chunker 7 | spec: 8 | parallelism: 3 9 | template: 10 | metadata: 11 | labels: 12 | app.kubernetes.io/name: chunker 13 | spec: 14 | containers: 15 | - name: chunker 16 | image: gazette/examples 17 | imagePullPolicy: IfNotPresent 18 | command: 19 | # Run within a shell to allow for interpretation of values. 20 | - "/bin/sh" 21 | - "-ec" 22 | - | 23 | exec chunker run \ 24 | --chunker.zone=$(/home/gazette/.config/zonemap/node-zone.sh $NODE_NAME) \ 25 | --chunker.streams=${CHUNKER_STREAMS} \ 26 | --chunker.chunks=${CHUNKER_CHUNKS} \ 27 | --log.format=json 28 | env: 29 | # Broker service endpoint to use. 30 | # By default, assume a "gazette" service running in our same namespace. 31 | - name: BROKER_ADDRESS 32 | value: "http://gazette:8080" 33 | # Size of the journal route cache to use. This reduces network hops by 34 | # directing RPCs directly to a responsible broker for a given journal. 35 | - name: BROKER_CACHE_SIZE 36 | value: "256" 37 | - name: CHUNKER_STREAMS 38 | value: "100" 39 | - name: CHUNKER_CHUNKS 40 | value: "1000" 41 | # Node hostname, used to map to a failure zone. 42 | - name: NODE_NAME 43 | valueFrom: 44 | fieldRef: 45 | fieldPath: spec.nodeName 46 | 47 | volumeMounts: 48 | - name: zonemap 49 | readOnly: true 50 | mountPath: /home/gazette/.config/zonemap 51 | 52 | volumes: 53 | - name: zonemap 54 | configMap: 55 | name: gazette-zonemap 56 | items: 57 | - key: node-zone.sh 58 | path: node-zone.sh 59 | mode: 511 # Equivalent to 0777. 60 | 61 | restartPolicy: Never 62 | 63 | -------------------------------------------------------------------------------- /kustomize/bases/example-stream-sum/generate-shards.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash -e 2 | 3 | readonly STORES=(rocksdb sqlite) 4 | 5 | cat </dev/null 2>&1) 37 | return $? 38 | } 39 | 40 | # createBucket ($bucket, $policy, $purge) 41 | # Ensure bucket exists, purging if asked to 42 | createBucket() { 43 | BUCKET=$1 44 | POLICY=$2 45 | PURGE=$3 46 | 47 | # Purge the bucket, if set & exists 48 | # Since PURGE is user input, check explicitly for `true` 49 | if [ $PURGE = true ]; then 50 | if checkBucketExists $BUCKET; then 51 | echo "Purging bucket '$BUCKET'." 52 | set +e # don't exit if this fails 53 | /usr/bin/mc rm -r --force myminio/$BUCKET 54 | set -e # reset `e` as active 55 | else 56 | echo "Bucket '$BUCKET' does not exist, skipping purge." 57 | fi 58 | fi 59 | 60 | # Create the bucket if it does not exist 61 | if ! checkBucketExists $BUCKET; then 62 | echo "Creating bucket '$BUCKET'" 63 | /usr/bin/mc mb myminio/$BUCKET 64 | else 65 | echo "Bucket '$BUCKET' already exists." 66 | fi 67 | 68 | # At this point, the bucket should exist, skip checking for existence 69 | # Set policy on the bucket 70 | echo "Setting policy of bucket '$BUCKET' to '$POLICY'." 71 | /usr/bin/mc policy set $POLICY myminio/$BUCKET 72 | 73 | if checkBucketExists $BUCKET; then 74 | echo "Successfully created bucket '$BUCKET'" 75 | fi 76 | } 77 | 78 | # Try connecting to Minio instance 79 | scheme=http 80 | connectToMinio $scheme 81 | 82 | # Create the bucket 83 | createBucket examples public false 84 | -------------------------------------------------------------------------------- /kustomize/bases/minio/create-bucket.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: batch/v1 2 | kind: Job 3 | metadata: 4 | name: minio-create-bucket 5 | spec: 6 | template: 7 | spec: 8 | restartPolicy: OnFailure 9 | containers: 10 | - name: minio-mc 11 | image: "minio/mc:RELEASE.2019-08-07T23-14-43Z" 12 | imagePullPolicy: IfNotPresent 13 | command: ["/bin/sh", "/opt/minio/scripts/create-bucket.sh"] 14 | env: 15 | - name: MINIO_ENDPOINT 16 | value: $(MINIO_ENDPOINT) 17 | - name: MINIO_PORT 18 | value: "9000" 19 | volumeMounts: 20 | - name: create-bucket 21 | mountPath: /opt/minio/scripts 22 | - name: credentials 23 | mountPath: /opt/minio/credentials 24 | resources: 25 | requests: 26 | cpu: 250m 27 | memory: 256Mi 28 | 29 | volumes: 30 | - name: create-bucket 31 | configMap: 32 | name: minio-create-bucket 33 | - name: credentials 34 | secret: 35 | secretName: minio-credentials 36 | -------------------------------------------------------------------------------- /kustomize/bases/minio/deployment.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: apps/v1 2 | kind: Deployment 3 | metadata: 4 | name: minio 5 | labels: 6 | app.kubernetes.io/name: minio 7 | spec: 8 | selector: 9 | matchLabels: 10 | app.kubernetes.io/name: minio 11 | template: 12 | metadata: 13 | labels: 14 | app.kubernetes.io/name: minio 15 | spec: 16 | serviceAccountName: minio 17 | containers: 18 | - name: minio 19 | image: "minio/minio:RELEASE.2019-08-07T01-59-21Z" 20 | imagePullPolicy: IfNotPresent 21 | command: [ "/bin/sh", 22 | "-ce", 23 | "/usr/bin/docker-entrypoint.sh minio -C /root/.minio/ server /export" ] 24 | volumeMounts: 25 | - name: minio-config-dir 26 | mountPath: /root/.minio/ 27 | - name: export 28 | mountPath: /export 29 | ports: 30 | - name: service 31 | containerPort: 9000 32 | env: 33 | - name: MINIO_ACCESS_KEY 34 | valueFrom: 35 | secretKeyRef: 36 | name: minio-credentials 37 | key: accesskey 38 | - name: MINIO_SECRET_KEY 39 | valueFrom: 40 | secretKeyRef: 41 | name: minio-credentials 42 | key: secretkey 43 | - name: MINIO_BROWSER 44 | value: "on" 45 | livenessProbe: 46 | httpGet: 47 | path: /minio/health/live 48 | port: service 49 | scheme: HTTP 50 | initialDelaySeconds: 5 51 | periodSeconds: 30 52 | timeoutSeconds: 1 53 | successThreshold: 1 54 | failureThreshold: 3 55 | readinessProbe: 56 | httpGet: 57 | path: /minio/health/ready 58 | port: service 59 | initialDelaySeconds: 5 60 | periodSeconds: 15 61 | timeoutSeconds: 1 62 | successThreshold: 1 63 | failureThreshold: 3 64 | resources: 65 | requests: 66 | cpu: 250m 67 | memory: 256Mi 68 | 69 | volumes: 70 | - name: export 71 | emptyDir: {} 72 | - name: minio-config-dir 73 | emptyDir: {} 74 | -------------------------------------------------------------------------------- /kustomize/bases/minio/kustomization.yaml: -------------------------------------------------------------------------------- 1 | # Minio deploys an ephemeral instance of Minio with an "examples" bucket 2 | # and configured test credentials. 3 | resources: 4 | - create-bucket.yaml 5 | - deployment.yaml 6 | - service.yaml 7 | - serviceaccount.yaml 8 | 9 | vars: 10 | - name: MINIO_ENDPOINT 11 | objref: 12 | name: minio 13 | kind: Service 14 | apiVersion: v1 15 | 16 | secretGenerator: 17 | - name: minio-credentials 18 | literals: 19 | - accesskey=AKIAIOSFODNN7EXAMPLE 20 | - secretkey=wJalrXUtnFEMI/K7MDENG/bPxRfiCYEXAMPLEKEY 21 | 22 | configMapGenerator: 23 | - name: minio-create-bucket 24 | files: 25 | - create-bucket.sh 26 | 27 | -------------------------------------------------------------------------------- /kustomize/bases/minio/service.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: v1 2 | kind: Service 3 | metadata: 4 | name: minio 5 | labels: 6 | app.kubernetes.io/name: minio 7 | spec: 8 | type: ClusterIP 9 | ports: 10 | - name: service 11 | port: 9000 12 | protocol: TCP 13 | targetPort: 9000 14 | selector: 15 | app.kubernetes.io/name: minio 16 | -------------------------------------------------------------------------------- /kustomize/bases/minio/serviceaccount.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: v1 2 | kind: ServiceAccount 3 | metadata: 4 | name: minio 5 | -------------------------------------------------------------------------------- /kustomize/bases/postgres/kustomization.yaml: -------------------------------------------------------------------------------- 1 | # Postgres deploys Postgres as a StatefulSet with pod-ephemeral storage. 2 | # A ConfigMap "postgres-tables" must be defined outside of this manifest. 3 | resources: 4 | - statefulset.yaml 5 | - service.yaml 6 | -------------------------------------------------------------------------------- /kustomize/bases/postgres/service.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: v1 2 | kind: Service 3 | metadata: 4 | name: postgres 5 | labels: 6 | app.kubernetes.io/name: postgres 7 | spec: 8 | ports: 9 | - port: 5432 10 | name: postgres 11 | selector: 12 | app.kubernetes.io/name: postgres -------------------------------------------------------------------------------- /kustomize/bases/postgres/statefulset.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: apps/v1 2 | kind: StatefulSet 3 | metadata: 4 | name: postgres 5 | labels: 6 | app.kubernetes.io/name: postgres 7 | spec: 8 | replicas: 1 9 | serviceName: postgres 10 | selector: 11 | matchLabels: 12 | app.kubernetes.io/name: postgres 13 | template: 14 | metadata: 15 | name: postgres 16 | labels: 17 | app.kubernetes.io/name: postgres 18 | spec: 19 | containers: 20 | - name: postgres 21 | image: postgres:12 22 | imagePullPolicy: IfNotPresent 23 | ports: 24 | - containerPort: 5432 25 | name: tcp 26 | env: 27 | - name: POSTGRES_DB 28 | value: test 29 | - name: POSTGRES_USER 30 | value: test 31 | - name: POSTGRES_PASSWORD 32 | value: test 33 | - name: PGDATA 34 | value: /var/lib/postgresql/data/pgdata 35 | 36 | volumeMounts: 37 | - name: datadir 38 | mountPath: /var/lib/postgresql/data 39 | - name: postgres-init 40 | mountPath: /docker-entrypoint-initdb.d 41 | volumes: 42 | - name: postgres-init 43 | configMap: 44 | name: postgres-init 45 | 46 | volumeClaimTemplates: 47 | - metadata: 48 | name: datadir 49 | spec: 50 | accessModes: 51 | - ReadWriteOnce 52 | resources: 53 | requests: 54 | storage: "512Mi" 55 | -------------------------------------------------------------------------------- /kustomize/test/bases/crash-tester/kustomization.yaml: -------------------------------------------------------------------------------- 1 | # Crash-tester is deployed alongside the stream-sum example, turning 2 | # it into a soak test. It repeatedly runs a Go test-suite which crashes, 3 | # partitions, and otherwise faults components of the deployment and then 4 | # verifies correct recovery and processing SLAs. 5 | resources: 6 | - rbac.yaml 7 | - runner.yaml 8 | - urkel.yaml 9 | 10 | namespace: stream-sum -------------------------------------------------------------------------------- /kustomize/test/bases/crash-tester/rbac.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: v1 2 | kind: ServiceAccount 3 | metadata: 4 | name: crash-test-runner 5 | labels: 6 | app.kubernetes.io/name: crash-test 7 | --- 8 | apiVersion: rbac.authorization.k8s.io/v1 9 | kind: Role 10 | metadata: 11 | name: crash-test-runner 12 | labels: 13 | app.kubernetes.io/name: crash-test 14 | rules: 15 | - apiGroups: [""] 16 | resources: [pods] 17 | verbs: [get, list, watch, delete] 18 | 19 | - apiGroups: [""] 20 | resources: [pods/exec] 21 | verbs: [create] 22 | --- 23 | apiVersion: rbac.authorization.k8s.io/v1 24 | kind: RoleBinding 25 | metadata: 26 | name: crash-test-runner 27 | labels: 28 | app.kubernetes.io/name: crash-test 29 | subjects: 30 | - kind: ServiceAccount 31 | name: crash-test-runner 32 | apiGroup: "" 33 | roleRef: 34 | kind: Role 35 | name: crash-test-runner 36 | apiGroup: rbac.authorization.k8s.io 37 | -------------------------------------------------------------------------------- /kustomize/test/bases/crash-tester/runner.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: batch/v1 2 | kind: Job 3 | metadata: 4 | name: crash-test-runner 5 | labels: 6 | app.kubernetes.io/name: crash-test 7 | spec: 8 | # Fail-fast if the job fails. 9 | backoffLimit: 0 10 | template: 11 | metadata: 12 | labels: 13 | app.kubernetes.io/name: crash-test 14 | spec: 15 | restartPolicy: Never 16 | serviceAccountName: crash-test-runner 17 | containers: 18 | - name: runner 19 | image: gazette/examples 20 | command: 21 | - integration.test 22 | - "-test.v" 23 | - "-test.count=10" 24 | - "-test.failfast" 25 | -------------------------------------------------------------------------------- /kustomize/test/bases/crash-tester/urkel.yaml: -------------------------------------------------------------------------------- 1 | # Urkel is a fault-injection service which offers a gRPC API for enacting 2 | # various kinds of faults, such as network partitions, latency, disk stress, 3 | # etc. It runs on every Kubernetes node which runs Pods under test. 4 | apiVersion: apps/v1 5 | kind: DaemonSet 6 | metadata: 7 | name: urkel 8 | labels: 9 | app.kubernetes.io/name: urkel 10 | spec: 11 | selector: 12 | matchLabels: 13 | app.kubernetes.io/name: urkel 14 | template: 15 | metadata: 16 | labels: 17 | app.kubernetes.io/name: urkel 18 | spec: 19 | containers: 20 | - name: urkel 21 | image: dademurphy/urkel:v0.1.2 22 | imagePullPolicy: IfNotPresent 23 | securityContext: 24 | # Urkel functions by entering the network namespace of the 25 | # target container and configuring iptables, etc to impact 26 | # traffic flows. It require capabilities to setns() and mount(). 27 | privileged: true 28 | capabilities: 29 | add: 30 | - SYS_ADMIN 31 | ports: 32 | - name: http 33 | containerPort: 1666 34 | hostPort: 1666 35 | protocol: TCP 36 | volumeMounts: 37 | - name: netns 38 | mountPath: /var/run/netns 39 | readOnly: true 40 | mountPropagation: HostToContainer 41 | 42 | # Urkel inspects the process and network namespaces of the 43 | # host to map to specific network namespace into which faults 44 | # are injected. 45 | hostNetwork: true 46 | hostPID: true 47 | 48 | volumes: 49 | - name: netns 50 | hostPath: 51 | path: /var/run/netns 52 | type: Directory 53 | -------------------------------------------------------------------------------- /kustomize/test/bases/environment/apply-journal-specs.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: batch/v1 2 | kind: Job 3 | metadata: 4 | name: apply-journal-specs 5 | spec: 6 | template: 7 | spec: 8 | restartPolicy: OnFailure 9 | containers: 10 | - name: apply-specs 11 | image: gazette/broker 12 | imagePullPolicy: IfNotPresent 13 | command: 14 | - gazctl 15 | - journals 16 | - apply 17 | - "--specs=/config/examples.journalspace.yaml" 18 | env: 19 | # Broker service endpoint to use. By default, assume a "gazette" 20 | # service running in our same namespace. 21 | - name: BROKER_ADDRESS 22 | value: "http://gazette:8080" 23 | volumeMounts: 24 | - name: example-journal-specs 25 | mountPath: /config 26 | volumes: 27 | - name: example-journal-specs 28 | configMap: 29 | name: example-journal-specs 30 | -------------------------------------------------------------------------------- /kustomize/test/bases/environment/kustomization.yaml: -------------------------------------------------------------------------------- 1 | # environment sets up a context for testing Gazette. It consists of: 2 | # * A gazette-aws-credentials Secret with Minio default credentials. 3 | # * A gazette-zonemap ConfigMap which randomly selects "zone-A" or "zone-B". 4 | # * A job which applies examples.journalspace.yaml to brokers. 5 | resources: 6 | - apply-journal-specs.yaml 7 | 8 | configMapGenerator: 9 | # Configure a zonemap which randomly selects zone-A or zone-B. 10 | - name: gazette-zonemap 11 | files: 12 | - node-zone.sh=random_node_zonemap.sh 13 | # Configure a collection of journal fixtures used by examples & integration tests. 14 | - name: example-journal-specs 15 | files: 16 | - examples.journalspace.yaml 17 | # Configure tables to be created should postgres be included in the manifest. 18 | - name: postgres-init 19 | files: 20 | - postgres_tables.sql 21 | 22 | secretGenerator: 23 | # Configure AWS application default credentials (~/.aws/) using Minio keys. 24 | - name: gazette-aws-credentials 25 | files: 26 | - config=minio_config 27 | - credentials=minio_credentials 28 | 29 | generatorOptions: 30 | # Update ConfigMap and Secret in-place should they ever change. 31 | disableNameSuffixHash: true -------------------------------------------------------------------------------- /kustomize/test/bases/environment/minio_config: -------------------------------------------------------------------------------- 1 | # Note that ~/.aws/config uses "[profile my-name]", 2 | # where ~/.aws/credentials uses just "[my-name]". 3 | [profile minio] 4 | region=us-east-1 5 | -------------------------------------------------------------------------------- /kustomize/test/bases/environment/minio_credentials: -------------------------------------------------------------------------------- 1 | [minio] 2 | # These are the default example credentials which Minio starts with. 3 | aws_access_key_id=AKIAIOSFODNN7EXAMPLE 4 | aws_secret_access_key=wJalrXUtnFEMI/K7MDENG/bPxRfiCYEXAMPLEKEY 5 | -------------------------------------------------------------------------------- /kustomize/test/bases/environment/postgres_tables.sql: -------------------------------------------------------------------------------- 1 | -- Table gazette_checkpoints is expected by consumer.SQLStore for its use in 2 | -- persisting consumer transaction checkpoints. It's created just once per 3 | -- database, and is shared by all consumer applications and shards which use it. 4 | CREATE TABLE gazette_checkpoints 5 | ( 6 | shard_fqn TEXT PRIMARY KEY NOT NULL, 7 | fence INTEGER NOT NULL, 8 | checkpoint BYTEA NOT NULL 9 | ); 10 | 11 | -- Table rides captures a bike-share datum with its Gazette UUID. 12 | CREATE TABLE rides 13 | ( 14 | uuid UUID NOT NULL, 15 | trip_duration INTERVAL NOT NULL, 16 | bike_id INTEGER NOT NULL, 17 | user_type VARCHAR(16) NOT NULL, 18 | birth_year SMALLINT NOT NULL, 19 | gender SMALLINT NOT NULL, 20 | 21 | start_time TIMESTAMP NOT NULL, 22 | start_station_id INTEGER NOT NULL, 23 | start_station_name VARCHAR(64) NOT NULL, 24 | start_station_latitude REAL NOT NULL, 25 | start_station_longitude REAL NOT NULL, 26 | 27 | end_time TIMESTAMP NOT NULL, 28 | end_station_id INTEGER NOT NULL, 29 | end_station_name VARCHAR(64) NOT NULL, 30 | end_station_latitude REAL NOT NULL, 31 | end_station_longitude REAL NOT NULL, 32 | 33 | PRIMARY KEY (bike_id, start_time) 34 | ); 35 | -------------------------------------------------------------------------------- /kustomize/test/bases/environment/random_node_zonemap.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | shuf -n 1 << EOF 4 | zone-A 5 | zone-B 6 | EOF 7 | -------------------------------------------------------------------------------- /kustomize/test/deploy-bike-share/kustomization.yaml: -------------------------------------------------------------------------------- 1 | # Deploy-bike-share deploys the bike-share Gazette example and its 2 | # dependencies to the "bike-share" namespace. 3 | bases: 4 | - ../../bases/broker 5 | - ../../bases/etcd 6 | - ../../bases/example-bike-share 7 | - ../../bases/minio 8 | - ../../bases/postgres 9 | - ../bases/environment 10 | resources: 11 | - namespace.yaml 12 | 13 | namespace: bike-share 14 | 15 | commonLabels: 16 | app.kubernetes.io/instance: bike-share 17 | -------------------------------------------------------------------------------- /kustomize/test/deploy-bike-share/namespace.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: v1 2 | kind: Namespace 3 | metadata: 4 | name: bike-share 5 | -------------------------------------------------------------------------------- /kustomize/test/deploy-stream-sum-with-crash-tests/kustomization.yaml: -------------------------------------------------------------------------------- 1 | # deploy-stream-sum-with-crash-tests runs deploy-stream-sum with the 2 | # crash-tester mixin. 3 | bases: 4 | - ../deploy-stream-sum 5 | - ../bases/crash-tester 6 | 7 | patchesStrategicMerge: 8 | - patches.yaml 9 | -------------------------------------------------------------------------------- /kustomize/test/deploy-stream-sum-with-crash-tests/patches.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: apps/v1 2 | kind: Deployment 3 | metadata: 4 | name: gazette 5 | spec: 6 | # The test expects that four brokers are running, and each journal has a 7 | # replication factor of three. This allows us to: 8 | # * Partition one broker from Etcd without losing liveness. 9 | # * Crash any two brokers without losing liveness or durability. 10 | replicas: 4 11 | template: 12 | spec: 13 | containers: 14 | - name: gazette 15 | # Shorten the readiness check of the broker to 1s, from the default of 10s. 16 | # This more tightly bounds the amount of time that the gazette Service is 17 | # unavailable. 18 | readinessProbe: 19 | periodSeconds: 1 20 | --- 21 | apiVersion: batch/v1 22 | kind: Job 23 | metadata: 24 | name: chunker 25 | spec: 26 | template: 27 | spec: 28 | containers: 29 | - name: chunker 30 | env: 31 | # Configure chunker jobs to run forever, 32 | # but reduce the volume of chunks per stream. 33 | - name: CHUNKER_STREAMS 34 | value: "-1" 35 | - name: CHUNKER_CHUNKS 36 | value: "3" 37 | -------------------------------------------------------------------------------- /kustomize/test/deploy-stream-sum/kustomization.yaml: -------------------------------------------------------------------------------- 1 | # Deploy-stream-sum deploys the stream-sum Gazette example and its 2 | # dependencies to the "stream-sum" namespace. 3 | bases: 4 | - ../../bases/broker 5 | - ../../bases/etcd 6 | - ../../bases/example-stream-sum 7 | - ../../bases/minio 8 | - ../bases/environment 9 | resources: 10 | - namespace.yaml 11 | 12 | namespace: stream-sum 13 | 14 | commonLabels: 15 | app.kubernetes.io/instance: stream-sum 16 | -------------------------------------------------------------------------------- /kustomize/test/deploy-stream-sum/namespace.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: v1 2 | kind: Namespace 3 | metadata: 4 | name: stream-sum 5 | -------------------------------------------------------------------------------- /kustomize/test/deploy-word-count/kustomization.yaml: -------------------------------------------------------------------------------- 1 | # Deploy-word-count deploys the word-count Gazette example and its 2 | # dependencies to the "word-count" namespace. 3 | bases: 4 | - ../../bases/broker 5 | - ../../bases/etcd 6 | - ../../bases/example-word-count 7 | - ../../bases/minio 8 | - ../bases/environment 9 | resources: 10 | - namespace.yaml 11 | 12 | namespace: word-count 13 | 14 | commonLabels: 15 | app.kubernetes.io/instance: word-count 16 | -------------------------------------------------------------------------------- /kustomize/test/deploy-word-count/namespace.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: v1 2 | kind: Namespace 3 | metadata: 4 | name: word-count 5 | -------------------------------------------------------------------------------- /kustomize/test/run-with-local-registry/always-pull.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: apps/v1 2 | kind: Deployment 3 | metadata: 4 | name: gazette 5 | spec: 6 | template: 7 | spec: 8 | containers: 9 | - name: gazette 10 | imagePullPolicy: Always 11 | --- 12 | apiVersion: apps/v1 13 | kind: Deployment 14 | metadata: 15 | name: consumer 16 | spec: 17 | template: 18 | spec: 19 | containers: 20 | - name: consumer 21 | imagePullPolicy: Always 22 | -------------------------------------------------------------------------------- /kustomize/test/run-with-local-registry/kustomization.yaml: -------------------------------------------------------------------------------- 1 | # run-with-local-registry is a kustomize helper which patches a base, 2 | # which must be set below in "bases:", to use a local registry for the 3 | # gazette and example images and to always pull from that registry. 4 | bases: 5 | - ../deploy-stream-sum-with-crash-tests 6 | 7 | patchesStrategicMerge: 8 | - always-pull.yaml 9 | 10 | images: 11 | - name: gazette/broker 12 | newName: localhost:32000/broker 13 | newTag: latest 14 | 15 | - name: gazette/examples 16 | newName: localhost:32000/examples 17 | newTag: latest 18 | 19 | -------------------------------------------------------------------------------- /mainboilerplate/cmdreg.go: -------------------------------------------------------------------------------- 1 | package mainboilerplate 2 | 3 | import "github.com/jessevdk/go-flags" 4 | 5 | // AddCommandFunc are used to register sub-commands with a parent 6 | type AddCommandFunc func(*flags.Command) error 7 | 8 | // CommandRegistry is a simple tool for building a tree of github.com/jessevdk/go-flags.AddCommand functions 9 | // that you can use to register sub-commands under a github.com/jessevdk/go-flags.Command 10 | type CommandRegistry map[string][]AddCommandFunc 11 | 12 | // NewCommandRegistry creates a new registry 13 | func NewCommandRegistry() CommandRegistry { 14 | return make(CommandRegistry) 15 | } 16 | 17 | // AddCommand takes a parentName and then an github.com/jessevdk/go-flags.AddCommand specification and stores it in the registry 18 | // You can specify a tree of commands by separating parentName with dots. 19 | // Example for adding command level1 and then level1 level2 20 | // AddCommand("level1",....) 21 | // AddCommand("level1.level2",....) 22 | func (cr CommandRegistry) AddCommand(parentName string, command string, shortDescription string, longDescription string, data interface{}) { 23 | cr[parentName] = append(cr[parentName], func(cmd *flags.Command) error { 24 | _, err := cmd.AddCommand(command, shortDescription, longDescription, data) 25 | return err 26 | }) 27 | } 28 | 29 | // AddCommands recursively walks through the tree of registered sub-commands under rootName and adds them under rootCmd. 30 | // If recursive is true it will recurse down the tree to add sub-commands of sub-commands 31 | func (cr CommandRegistry) AddCommands(rootName string, rootCmd *flags.Command, recursive bool) error { 32 | 33 | // Register any command 34 | for _, addCommandFunc := range cr[rootName] { 35 | if err := addCommandFunc(rootCmd); err != nil { 36 | return err 37 | } 38 | } 39 | 40 | // See if there are any sub-commands that now need registered under this one 41 | if recursive { 42 | for _, cmd := range rootCmd.Commands() { 43 | // Register every sub-command. Sub-Commands are separated with a . 44 | cmdName := cmd.Name 45 | if rootName != "" { 46 | cmdName = rootName + "." + cmdName 47 | } 48 | // Recursively register any sub-commands under this command 49 | if err := cr.AddCommands(cmdName, cmd, recursive); err != nil { 50 | return err 51 | } 52 | } 53 | } 54 | 55 | return nil 56 | 57 | } 58 | -------------------------------------------------------------------------------- /mainboilerplate/etcd.go: -------------------------------------------------------------------------------- 1 | package mainboilerplate 2 | 3 | import ( 4 | "context" 5 | "crypto/tls" 6 | "time" 7 | 8 | log "github.com/sirupsen/logrus" 9 | clientv3 "go.etcd.io/etcd/client/v3" 10 | "go.gazette.dev/core/broker/protocol" 11 | "go.gazette.dev/core/server" 12 | "google.golang.org/grpc" 13 | ) 14 | 15 | // EtcdConfig configures the application Etcd session. 16 | type EtcdConfig struct { 17 | Address protocol.Endpoint `long:"address" env:"ADDRESS" default:"http://localhost:2379" description:"Etcd service address endpoint"` 18 | CertFile string `long:"cert-file" env:"CERT_FILE" default:"" description:"Path to the client TLS certificate"` 19 | CertKeyFile string `long:"cert-key-file" env:"CERT_KEY_FILE" default:"" description:"Path to the client TLS private key"` 20 | TrustedCAFile string `long:"trusted-ca-file" env:"TRUSTED_CA_FILE" default:"" description:"Path to the trusted CA for client verification of server certificates"` 21 | LeaseTTL time.Duration `long:"lease" env:"LEASE_TTL" default:"20s" description:"Time-to-live of Etcd lease"` 22 | } 23 | 24 | // MustDial builds an Etcd client connection. 25 | func (c *EtcdConfig) MustDial() *clientv3.Client { 26 | var addr = c.Address.URL() 27 | var tlsConfig *tls.Config 28 | 29 | switch addr.Scheme { 30 | case "https": 31 | var err error 32 | tlsConfig, err = server.BuildTLSConfig(c.CertFile, c.CertKeyFile, c.TrustedCAFile) 33 | Must(err, "failed to build TLS config") 34 | case "unix": 35 | // The Etcd client requires hostname is stripped from unix:// URLs. 36 | addr.Host = "" 37 | } 38 | 39 | // Use a blocking dial to build a trial connection to Etcd. If we're actively 40 | // partitioned or mis-configured this avoids a K8s CrashLoopBackoff, and 41 | // there's nothing actionable to do anyway aside from wait (or be SIGTERM'd). 42 | var timer = time.AfterFunc(time.Second, func() { 43 | log.WithField("addr", addr.String()).Warn("dialing Etcd is taking a while (is network okay?)") 44 | }) 45 | trialEtcd, err := clientv3.New(clientv3.Config{ 46 | Endpoints: []string{addr.String()}, 47 | DialOptions: []grpc.DialOption{grpc.WithBlock()}, 48 | TLS: tlsConfig, 49 | }) 50 | Must(err, "failed to build trial Etcd client") 51 | 52 | _ = trialEtcd.Close() 53 | timer.Stop() 54 | 55 | // Build our actual |etcd| connection, with much tighter timeout bounds. 56 | etcd, err := clientv3.New(clientv3.Config{ 57 | Endpoints: []string{addr.String()}, 58 | // Automatically and periodically sync the set of Etcd servers. 59 | // If a network split occurs, this allows for attempting different 60 | // members until a connectable one is found on our "side" of the network 61 | // partition. 62 | AutoSyncInterval: time.Minute, 63 | // Use aggressive timeouts to quickly cycle through member endpoints, 64 | // prior to our lease TTL expiring. 65 | DialTimeout: c.LeaseTTL / 20, 66 | DialKeepAliveTime: c.LeaseTTL / 4, 67 | DialKeepAliveTimeout: c.LeaseTTL / 4, 68 | // Require a reasonably recent server cluster. 69 | RejectOldCluster: true, 70 | TLS: tlsConfig, 71 | }) 72 | Must(err, "failed to build Etcd client") 73 | 74 | Must(etcd.Sync(context.Background()), "initial Etcd endpoint sync failed") 75 | return etcd 76 | } 77 | -------------------------------------------------------------------------------- /mainboilerplate/logging.go: -------------------------------------------------------------------------------- 1 | package mainboilerplate 2 | 3 | import ( 4 | log "github.com/sirupsen/logrus" 5 | ) 6 | 7 | // LogConfig configures handling of application log events. 8 | type LogConfig struct { 9 | Level string `long:"level" env:"LEVEL" default:"warn" choice:"trace" choice:"debug" choice:"info" choice:"warn" choice:"error" choice:"fatal" description:"Logging level"` 10 | Format string `long:"format" env:"FORMAT" default:"text" choice:"json" choice:"text" choice:"color" description:"Logging output format"` 11 | } 12 | 13 | // InitLog configures the logger. 14 | func InitLog(cfg LogConfig) { 15 | if cfg.Format == "json" { 16 | log.SetFormatter(&log.JSONFormatter{}) 17 | } else if cfg.Format == "text" { 18 | log.SetFormatter(&log.TextFormatter{}) 19 | } else if cfg.Format == "color" { 20 | log.SetFormatter(&log.TextFormatter{ForceColors: true}) 21 | } 22 | 23 | if lvl, err := log.ParseLevel(cfg.Level); err != nil { 24 | log.WithField("err", err).Fatal("unrecognized log level") 25 | } else { 26 | log.SetLevel(lvl) 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /message/csv_framing.go: -------------------------------------------------------------------------------- 1 | package message 2 | 3 | import ( 4 | "bufio" 5 | "encoding/csv" 6 | "errors" 7 | "fmt" 8 | 9 | "github.com/google/uuid" 10 | pb "go.gazette.dev/core/broker/protocol" 11 | "go.gazette.dev/core/labels" 12 | ) 13 | 14 | // CSVFramable is the interface of a Frameable required by a CSV Framing. 15 | type CSVFrameable interface { 16 | // MarshalCSV returns CSV records describing of the message. 17 | MarshalCSV() ([]string, error) 18 | // UnmarshalCSV applies the records to unmarshal the message 19 | // from its CSV description. It must copy the []string records if it 20 | // wishes to retain them after returning. 21 | UnmarshalCSV([]string) error 22 | } 23 | 24 | // CSVRecord is a minimal implementation of CSVFrameable and Message. 25 | // It requires that the first field is a string-encoded UUID. 26 | type CSVRecord []string 27 | 28 | // MarshalCSV returns the CSVRecord directly. 29 | func (r CSVRecord) MarshalCSV() ([]string, error) { return r, nil } 30 | 31 | // UnmarshalCSV copies the []string to this CSVRecord, 32 | // and verifies the first column parses as a UUID. 33 | func (r *CSVRecord) UnmarshalCSV(fields []string) error { 34 | if len(fields) == 0 { 35 | return errors.New("fields are empty") 36 | } 37 | if _, err := uuid.Parse(fields[0]); err != nil { 38 | return err 39 | } 40 | *r = append((*r)[:0], fields...) 41 | return nil 42 | } 43 | 44 | func (r CSVRecord) GetUUID() UUID { return uuid.MustParse(r[0]) } 45 | func (r CSVRecord) SetUUID(uuid UUID) { r[0] = uuid.String() } 46 | func (r CSVRecord) NewAcknowledgement(pb.Journal) Message { return make(CSVRecord, len(r)) } 47 | 48 | type csvFraming struct{} 49 | 50 | func (csvFraming) ContentType() string { return labels.ContentType_CSV } 51 | 52 | func (csvFraming) Marshal(f Frameable, bw *bufio.Writer) error { 53 | var cf, ok = f.(CSVFrameable) 54 | if !ok { 55 | return fmt.Errorf("%#v is not a CSVFrameable", f) 56 | } else if records, err := cf.MarshalCSV(); err != nil { 57 | return err 58 | } else { 59 | return csv.NewWriter(bw).Write(records) // Marshals directly to |bw|. 60 | } 61 | } 62 | 63 | func (csvFraming) NewUnmarshalFunc(r *bufio.Reader) UnmarshalFunc { 64 | var cr = csv.NewReader(r) 65 | cr.ReuseRecord = true 66 | 67 | return func(f Frameable) error { 68 | var cf, ok = f.(CSVFrameable) 69 | if !ok { 70 | return fmt.Errorf("%#v is not a CSVFrameable", f) 71 | } 72 | 73 | var records, err = cr.Read() 74 | if err != nil { 75 | return err 76 | } 77 | return cf.UnmarshalCSV(records) 78 | } 79 | } 80 | 81 | func init() { RegisterFraming(new(csvFraming)) } 82 | -------------------------------------------------------------------------------- /message/csv_framing_test.go: -------------------------------------------------------------------------------- 1 | package message 2 | 3 | import ( 4 | "bufio" 5 | "bytes" 6 | "io" 7 | "testing" 8 | 9 | "github.com/google/uuid" 10 | "github.com/stretchr/testify/require" 11 | "go.gazette.dev/core/labels" 12 | ) 13 | 14 | func TestCSVFramingMarshalWithFixtures(t *testing.T) { 15 | var f, _ = FramingByContentType(labels.ContentType_CSV) 16 | var buf bytes.Buffer 17 | var bw = bufio.NewWriter(&buf) 18 | var uuid = uuid.New() 19 | 20 | // Expect encoding/csv gracefully handles quoting. 21 | require.NoError(t, f.Marshal(&CSVRecord{uuid.String(), "123", `qu"ote`}, bw)) 22 | _ = bw.Flush() 23 | require.Equal(t, uuid.String()+`,123,"qu""ote"`+"\n", buf.String()) 24 | 25 | require.NoError(t, f.Marshal(&CSVRecord{uuid.String(), "567,891"}, bw)) 26 | _ = bw.Flush() 27 | require.Equal(t, uuid.String()+`,123,"qu""ote"`+"\n"+uuid.String()+`,"567,891"`+"\n", buf.String()) 28 | } 29 | 30 | func TestCSVFramingDecodeWithFixture(t *testing.T) { 31 | var f, _ = FramingByContentType(labels.ContentType_CSV) 32 | var uuid = uuid.New() 33 | var fixture = []byte(uuid.String() + `,bar,baz` + "\n") 34 | 35 | var msg CSVRecord 36 | var unmarshal = f.NewUnmarshalFunc(testReader(fixture)) 37 | 38 | // Read single message. 39 | require.NoError(t, unmarshal(&msg)) 40 | require.Equal(t, CSVRecord{uuid.String(), "bar", "baz"}, msg) 41 | 42 | // EOF read on message boundary is returned as EOF. 43 | require.Equal(t, io.EOF, unmarshal(&msg)) 44 | } 45 | 46 | func TestCSVFramingErrorsIfFieldsChange(t *testing.T) { 47 | var f, _ = FramingByContentType(labels.ContentType_CSV) 48 | var uuid = uuid.New() 49 | var fixture = []byte(uuid.String() + ",foo,bar\n" + uuid.String() + ",baz\n") 50 | 51 | var msg CSVRecord 52 | var unmarshal = f.NewUnmarshalFunc(testReader(fixture)) 53 | 54 | require.NoError(t, unmarshal(&msg)) 55 | require.Equal(t, CSVRecord{uuid.String(), "foo", "bar"}, msg) 56 | 57 | require.EqualError(t, unmarshal(&msg), "record on line 2: wrong number of fields") 58 | } 59 | 60 | func TestCSVFramingDecodeError(t *testing.T) { 61 | var f, _ = FramingByContentType(labels.ContentType_CSV) 62 | var fixture = []byte("not-a-uuid, bar, baz\n") 63 | 64 | var msg CSVRecord 65 | var unmarshal = f.NewUnmarshalFunc(testReader(fixture)) 66 | 67 | require.EqualError(t, unmarshal(&msg), "invalid UUID length: 10") 68 | } 69 | -------------------------------------------------------------------------------- /message/json_framing.go: -------------------------------------------------------------------------------- 1 | package message 2 | 3 | import ( 4 | "bufio" 5 | "encoding/json" 6 | 7 | "go.gazette.dev/core/labels" 8 | ) 9 | 10 | type jsonFraming struct{} 11 | 12 | // JSONMarshalerTo should be implemented (along with json.Unmarshaler) 13 | // by the message being Marshaled if it needs to specify its JSON encoding method. 14 | // If this interface is not implemented jsonFraming will default to encoding/json 15 | // returns the number of bytes written and any error that occurs 16 | type JSONMarshalerTo interface { 17 | MarshalJSONTo(*bufio.Writer) (int, error) 18 | } 19 | 20 | // ContentType returns labels.ContentType_JSONLines. 21 | func (*jsonFraming) ContentType() string { return labels.ContentType_JSONLines } 22 | 23 | // Marshal implements Framing. 24 | func (*jsonFraming) Marshal(msg Frameable, bw *bufio.Writer) error { 25 | if jf, ok := msg.(JSONMarshalerTo); ok { 26 | _, err := jf.MarshalJSONTo(bw) 27 | return err 28 | } 29 | return json.NewEncoder(bw).Encode(msg) 30 | } 31 | 32 | // NewUnmarshalFunc returns an UnmarshalFunc which decodes JSON messages from the Reader. 33 | func (*jsonFraming) NewUnmarshalFunc(r *bufio.Reader) UnmarshalFunc { 34 | // We cannot use json.NewDecoder, as it buffers internally beyond the 35 | // precise boundary of a JSON message. 36 | return func(f Frameable) (err error) { 37 | var l []byte 38 | if l, err = UnpackLine(r); err != nil { 39 | return 40 | } 41 | if jf, ok := f.(json.Unmarshaler); ok { 42 | return jf.UnmarshalJSON(l) 43 | } 44 | return json.Unmarshal(l, f) 45 | 46 | } 47 | } 48 | 49 | func init() { RegisterFraming(new(jsonFraming)) } 50 | -------------------------------------------------------------------------------- /message/uuid_test.go: -------------------------------------------------------------------------------- 1 | package message 2 | 3 | import ( 4 | "testing" 5 | "time" 6 | 7 | "github.com/google/uuid" 8 | "github.com/stretchr/testify/require" 9 | ) 10 | 11 | func TestUUIDProducerUniqueness(t *testing.T) { 12 | // Sanity check that the distribution of NewProducerID is highly random. 13 | var set = make(map[ProducerID]struct{}, 1000) 14 | for i := 0; i != 1000; i++ { 15 | set[NewProducerID()] = struct{}{} 16 | } 17 | require.Len(t, set, 1000) 18 | } 19 | 20 | func TestUUIDClock(t *testing.T) { 21 | var clock Clock 22 | 23 | // Each Tick produces a new value. 24 | require.Equal(t, clock.Tick(), Clock(1)) 25 | require.Equal(t, clock.Tick(), Clock(2)) 26 | 27 | clock.Update(time.Unix(12, 300)) 28 | 29 | // Clock representation is precise to 100ns. 30 | require.Equal(t, clock, NewClock(time.Unix(12, 399))) 31 | require.Equal(t, clock.AsTime(), time.Unix(12, 300)) 32 | require.True(t, clock < NewClock(time.Unix(12, 400))) 33 | require.True(t, clock > NewClock(time.Unix(12, 299))) 34 | 35 | // Within a 100ns interval, Clock uses remaining bits to expand the 36 | // distinguishable sequence. 37 | require.Equal(t, clock, NewClock(time.Unix(12, 300))) 38 | clock.Tick() 39 | require.True(t, clock > NewClock(time.Unix(12, 399))) 40 | 41 | // Just 4 bits are available. If they're overflowed, Tick will spill 42 | // over to update the timestamp. 43 | for i := 0; i != 14; i++ { 44 | clock.Tick() 45 | require.True(t, clock < NewClock(time.Unix(12, 400))) 46 | } 47 | clock.Tick() // 16th tick. 48 | require.Equal(t, clock, NewClock(time.Unix(12, 400))) 49 | require.Equal(t, clock.AsTime(), time.Unix(12, 400)) 50 | 51 | // Update must never decrease the clock value. 52 | clock.Update(time.Unix(11, 100)) 53 | require.Equal(t, clock, NewClock(time.Unix(12, 400))) 54 | 55 | // That includes sequence bits. 56 | clock.Tick() 57 | clock.Update(time.Unix(12, 400)) 58 | require.True(t, clock > NewClock(time.Unix(12, 400))) 59 | 60 | // Sequence bits are reset if the clock timestamp is updated. 61 | clock.Update(time.Unix(12, 500)) 62 | require.Equal(t, clock, NewClock(time.Unix(12, 500))) 63 | require.Equal(t, clock.AsTime(), time.Unix(12, 500)) 64 | } 65 | 66 | func TestUUIDBuilding(t *testing.T) { 67 | var producer = NewProducerID() 68 | 69 | // Craft an interesting Clock fixture which uses the full bit-range 70 | // and includes clock sequence increments. 71 | var clock Clock 72 | const expectSecs, expectNanos = 1567304621, 981273734 73 | clock.Update(time.Unix(expectSecs, expectNanos)) 74 | clock.Tick() 75 | clock.Tick() 76 | 77 | const expectFlags = 682 // 0b1010101010 78 | 79 | var id = BuildUUID(producer, clock, expectFlags) 80 | require.Equal(t, clock, GetClock(id)) 81 | require.Equal(t, producer, GetProducerID(id)) 82 | require.Equal(t, Flags(expectFlags), GetFlags(id)) 83 | 84 | // Verify compatibility with github.com/google/uuid package. 85 | require.Equal(t, uuid.Version(1), id.Version()) 86 | require.Equal(t, uuid.RFC4122, id.Variant()) 87 | require.Equal(t, producer[:], id.NodeID()) 88 | require.Equal(t, uuid.Time(clock>>4), id.Time()) 89 | 90 | // Expect package extracts our time-point, rounded to 100ns precision. 91 | var sec, nsec = id.Time().UnixTime() 92 | require.Equal(t, int64(expectSecs), sec) 93 | require.Equal(t, int64((expectNanos/100)*100), nsec) 94 | } 95 | -------------------------------------------------------------------------------- /task/group.go: -------------------------------------------------------------------------------- 1 | package task 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/pkg/errors" 7 | log "github.com/sirupsen/logrus" 8 | "golang.org/x/sync/errgroup" 9 | ) 10 | 11 | // Group is a group of tasks which should each be executed concurrently, 12 | // and which should be collectively blocked on until all are complete. 13 | // Tasks should be preemptable, and the first task to return a non-nil 14 | // error cancels the entire Group. While Group is used to invoke and 15 | // wait on multiple goroutines, it is not itself thread-safe. 16 | type Group struct { 17 | // Context of the Group, which is cancelled by: 18 | // * Any function of the Group returning non-nil error, or 19 | // * An explicit call to Cancel, or 20 | // * A cancellation of the parent Context of the Group. 21 | // 22 | // Tasks queued to the Group should monitor Context and return 23 | // upon its cancellation. 24 | ctx context.Context 25 | // Cancels Context. 26 | cancelFn context.CancelFunc 27 | 28 | tasks []task 29 | eg *errgroup.Group 30 | started bool 31 | } 32 | 33 | // task composes a runnable and its description. 34 | type task struct { 35 | desc string 36 | fn func() error 37 | } 38 | 39 | // NewGroup returns a new, empty Group with the given Context. 40 | func NewGroup(ctx context.Context) *Group { 41 | ctx, cancel := context.WithCancel(ctx) 42 | eg, ctx := errgroup.WithContext(ctx) 43 | return &Group{ctx: ctx, eg: eg, cancelFn: cancel} 44 | } 45 | 46 | // Context returns the Group Context. 47 | func (g *Group) Context() context.Context { return g.ctx } 48 | 49 | // Cancel the Group Context. 50 | func (g *Group) Cancel() { g.cancelFn() } 51 | 52 | // Queue a function for execution with the Group. 53 | // Cannot be called after GoRun is invoked or Queue panics. 54 | func (g *Group) Queue(desc string, fn func() error) { 55 | if g.started { 56 | panic("Queue called after GoRun") 57 | } 58 | g.tasks = append(g.tasks, task{desc: desc, fn: fn}) 59 | } 60 | 61 | // GoRun all queued functions. GoRun may be called only once: 62 | // the second invocation will panic. 63 | func (g *Group) GoRun() { 64 | if g.started { 65 | panic("GoRun already called") 66 | } 67 | g.started = true 68 | 69 | for i := range g.tasks { 70 | var t = g.tasks[i] 71 | g.eg.Go(func() error { 72 | if err := t.fn(); err != nil { 73 | log.WithFields(log.Fields{"task": t.desc, "err": err}).Info("task exited with error") 74 | return errors.WithMessage(err, t.desc) 75 | } 76 | log.WithField("task", t.desc).Info("task completed") 77 | return nil 78 | }) 79 | } 80 | } 81 | 82 | // Wait for started functions, returning only after all complete. 83 | // The first encountered non-nil error is returned. 84 | // GoRun must have been called or Wait panics. 85 | func (g *Group) Wait() error { 86 | if !g.started { 87 | panic("Wait called before GoRun") 88 | } 89 | return g.eg.Wait() 90 | } 91 | -------------------------------------------------------------------------------- /test/integration/crash_test.go: -------------------------------------------------------------------------------- 1 | // +build integration 2 | 3 | package integration 4 | 5 | import ( 6 | "testing" 7 | "time" 8 | 9 | "github.com/jgraettinger/urkel" 10 | ) 11 | 12 | func TestCrashOneEtcdMember(t *testing.T) { 13 | var pod = urkel.FetchPods(t, testNamespace, etcdPodSelector)[0] 14 | 15 | // Etcd cluster, brokers, and summers recover seamlessly from an Etcd crash. 16 | var fs = urkel.NewFaultSet(t) 17 | defer fs.RemoveAll() 18 | 19 | fs.Crash(pod) 20 | sleepAndRequireNoChunkerFails(t, 30*time.Second) 21 | } 22 | 23 | func TestCrashTwoBrokers(t *testing.T) { 24 | var pods = urkel.FetchPods(t, testNamespace, brokerPodSelector)[:2] 25 | 26 | // Up to two brokers can fail without risking append durability. 27 | // The crash will cause a stall for < 20s while their Etcd leases expire. 28 | var fs = urkel.NewFaultSet(t) 29 | defer fs.RemoveAll() 30 | 31 | fs.Crash(pods...) 32 | sleepAndRequireNoChunkerFails(t, 30*time.Second) 33 | } 34 | 35 | func TestCrashOneSummer(t *testing.T) { 36 | var pod = urkel.FetchPods(t, testNamespace, summerPodSelector)[0] 37 | 38 | // A summer can crash without violating exactly-once semantics, and without 39 | // a significant impact to live-ness. The crash will a stall for < 20s while 40 | // its Etcd leases expire. 41 | var fs = urkel.NewFaultSet(t) 42 | defer fs.RemoveAll() 43 | 44 | fs.Crash(pod) 45 | sleepAndRequireNoChunkerFails(t, 30*time.Second) 46 | } 47 | 48 | func TestDeleteAllBrokers(t *testing.T) { 49 | var pods = urkel.FetchPods(t, testNamespace, brokerPodSelector) 50 | 51 | // Brokers will seamlessly hand off journals and exit. Summers and chunkers 52 | // transition to new owners without incident. 53 | var fs = urkel.NewFaultSet(t) 54 | defer fs.RemoveAll() 55 | 56 | fs.Delete(pods...) 57 | sleepAndRequireNoChunkerFails(t, 30*time.Second) 58 | } 59 | 60 | func TestDeleteAllSummers(t *testing.T) { 61 | var pods = urkel.FetchPods(t, testNamespace, summerPodSelector) 62 | 63 | // Summers will seamlessly hand off shard processing and exit. 64 | var fs = urkel.NewFaultSet(t) 65 | defer fs.RemoveAll() 66 | 67 | fs.Delete(pods...) 68 | sleepAndRequireNoChunkerFails(t, 30*time.Second) 69 | } 70 | --------------------------------------------------------------------------------