diff --git a/.env b/.env index 76ba694..15e239b 100644 --- a/.env +++ b/.env @@ -38,8 +38,8 @@ AWS_ALLOW_HTTP="true" # minio storage engine # create a bucket "tansu" at: http://localhost:9001/buckets # or: mc mb local/tansu -STORAGE_ENGINE="minio=s3://tansu/" +# STORAGE_ENGINE="minio=s3://tansu/" # PostgreSQL storage engine # STORAGE_ENGINE="pg=postgres://postgres:postgres@db" -# STORAGE_ENGINE="pg=postgres://postgres:postgres@localhost" +STORAGE_ENGINE="pg=postgres://postgres:postgres@localhost" diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index bbb5ab8..723ef24 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -17,8 +17,13 @@ jobs: matrix: os: - ubuntu-latest + pg: + - postgres:16 steps: - uses: actions/checkout@v4 + - run: docker compose up --detach db + env: + POSTGRES_IMAGE: ${{matrix.pg}} - uses: actions-rust-lang/setup-rust-toolchain@v1 with: components: clippy, rustfmt diff --git a/compose.yaml b/compose.yaml index a142f81..6714c0b 100644 --- a/compose.yaml +++ b/compose.yaml @@ -1,7 +1,7 @@ --- services: db: - image: postgres:16.4 + image: ${POSTGRES_IMAGE:-postgres:16.4} environment: PGUSER: postgres POSTGRES_PASSWORD: postgres diff --git a/justfile b/justfile index 761a2e0..d265c64 100644 --- a/justfile +++ b/justfile @@ -9,7 +9,7 @@ test: cargo test --workspace --all-targets clippy: - cargo clippy --all-targets + cargo clippy --all-targets -- -D warnings fmt: cargo fmt --all diff --git a/tansu-kafka-sans-io/build.rs b/tansu-kafka-sans-io/build.rs index 13a6eed..b1e2f72 100644 --- a/tansu-kafka-sans-io/build.rs +++ b/tansu-kafka-sans-io/build.rs @@ -1127,7 +1127,7 @@ fn all(pattern: &str) -> Result> { .and_then(|mut paths| { paths.try_fold(Vec::new(), |mut acc, p| { p.map_err(Into::into) - // .inspect(|path| eprintln!("path: {path:?}")) + .inspect(|path| println!("cargo::rerun-if-changed={}", path.display())) .and_then(read_value) .and_then(|v| Message::try_from(&Wv::from(&v)).map_err(Into::into)) .map(|m| { diff --git a/tansu-kafka-sans-io/message/AddOffsetsToTxnRequest.json b/tansu-kafka-sans-io/message/AddOffsetsToTxnRequest.json index ade3fc7..157ae20 100644 --- a/tansu-kafka-sans-io/message/AddOffsetsToTxnRequest.json +++ b/tansu-kafka-sans-io/message/AddOffsetsToTxnRequest.json @@ -23,7 +23,9 @@ // Version 2 adds the support for new error code PRODUCER_FENCED. // // Version 3 enables flexible versions. - "validVersions": "0-3", + // + // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-4", "flexibleVersions": "3+", "fields": [ { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId", diff --git a/tansu-kafka-sans-io/message/AddOffsetsToTxnResponse.json b/tansu-kafka-sans-io/message/AddOffsetsToTxnResponse.json index 71fa655..6a713fe 100644 --- a/tansu-kafka-sans-io/message/AddOffsetsToTxnResponse.json +++ b/tansu-kafka-sans-io/message/AddOffsetsToTxnResponse.json @@ -22,7 +22,9 @@ // Version 2 adds the support for new error code PRODUCER_FENCED. // // Version 3 enables flexible versions. - "validVersions": "0-3", + // + // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-4", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", diff --git a/tansu-kafka-sans-io/message/AddPartitionsToTxnRequest.json b/tansu-kafka-sans-io/message/AddPartitionsToTxnRequest.json index 1b89c54..2270f7a 100644 --- a/tansu-kafka-sans-io/message/AddPartitionsToTxnRequest.json +++ b/tansu-kafka-sans-io/message/AddPartitionsToTxnRequest.json @@ -25,9 +25,11 @@ // Version 3 enables flexible versions. // // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions. + // + // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). // Versions 3 and below will be exclusively used by clients and versions 4 and above will be used by brokers. "latestVersionUnstable": false, - "validVersions": "0-4", + "validVersions": "0-5", "flexibleVersions": "3+", "fields": [ { "name": "Transactions", "type": "[]AddPartitionsToTxnTransaction", "versions": "4+", diff --git a/tansu-kafka-sans-io/message/AddPartitionsToTxnResponse.json b/tansu-kafka-sans-io/message/AddPartitionsToTxnResponse.json index 326b4ac..6c4eefd 100644 --- a/tansu-kafka-sans-io/message/AddPartitionsToTxnResponse.json +++ b/tansu-kafka-sans-io/message/AddPartitionsToTxnResponse.json @@ -24,7 +24,9 @@ // Version 3 enables flexible versions. // // Version 4 adds support to batch multiple transactions and a top level error code. - "validVersions": "0-4", + // + // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-5", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", diff --git a/tansu-kafka-sans-io/message/ConsumerGroupDescribeRequest.json b/tansu-kafka-sans-io/message/ConsumerGroupDescribeRequest.json index 1d7842a..c1c65ca 100644 --- a/tansu-kafka-sans-io/message/ConsumerGroupDescribeRequest.json +++ b/tansu-kafka-sans-io/message/ConsumerGroupDescribeRequest.json @@ -19,10 +19,6 @@ "listeners": ["zkBroker", "broker"], "name": "ConsumerGroupDescribeRequest", "validVersions": "0", - // The ConsumerGroupDescribe API is added as part of KIP-848 and is still - // under development. Hence, the API is not exposed by default by brokers - // unless explicitly enabled. - "latestVersionUnstable": true, "flexibleVersions": "0+", "fields": [ { "name": "GroupIds", "type": "[]string", "versions": "0+", "entityType": "groupId", diff --git a/tansu-kafka-sans-io/message/ConsumerGroupDescribeResponse.json b/tansu-kafka-sans-io/message/ConsumerGroupDescribeResponse.json index b20c051..3c6ed4e 100644 --- a/tansu-kafka-sans-io/message/ConsumerGroupDescribeResponse.json +++ b/tansu-kafka-sans-io/message/ConsumerGroupDescribeResponse.json @@ -87,13 +87,7 @@ ]}, { "name": "Assignment", "versions": "0+", "fields": [ { "name": "TopicPartitions", "type": "[]TopicPartitions", "versions": "0+", - "about": "The assigned topic-partitions to the member." }, - { "name": "Error", "type": "int8", "versions": "0+", - "about": "The assigned error." }, - { "name": "MetadataVersion", "type": "int32", "versions": "0+", - "about": "The assignor metadata version." }, - { "name": "MetadataBytes", "type": "bytes", "versions": "0+", - "about": "The assignor metadata bytes." } + "about": "The assigned topic-partitions to the member." } ]} ] } diff --git a/tansu-kafka-sans-io/message/DescribeTopicPartitionsRequest.json b/tansu-kafka-sans-io/message/DescribeTopicPartitionsRequest.json new file mode 100644 index 0000000..63c5b5c --- /dev/null +++ b/tansu-kafka-sans-io/message/DescribeTopicPartitionsRequest.json @@ -0,0 +1,40 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 75, + "type": "request", + "listeners": ["broker"], + "name": "DescribeTopicPartitionsRequest", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "Topics", "type": "[]TopicRequest", "versions": "0+", + "about": "The topics to fetch details for.", + "fields": [ + { "name": "Name", "type": "string", "versions": "0+", + "about": "The topic name", "versions": "0+", "entityType": "topicName"} + ] + }, + { "name": "ResponsePartitionLimit", "type": "int32", "versions": "0+", "default": "2000", + "about": "The maximum number of partitions included in the response." }, + { "name": "Cursor", "type": "Cursor", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The first topic and partition index to fetch details for.", "fields": [ + { "name": "TopicName", "type": "string", "versions": "0+", + "about": "The name for the first topic to process", "versions": "0+", "entityType": "topicName"}, + { "name": "PartitionIndex", "type": "int32", "versions": "0+", "about": "The partition index to start with"} + ]} + ] +} diff --git a/tansu-kafka-sans-io/message/DescribeTopicPartitionsResponse.json b/tansu-kafka-sans-io/message/DescribeTopicPartitionsResponse.json new file mode 100644 index 0000000..e8eee7d --- /dev/null +++ b/tansu-kafka-sans-io/message/DescribeTopicPartitionsResponse.json @@ -0,0 +1,66 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 75, + "type": "response", + "name": "DescribeTopicPartitionsResponse", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Topics", "type": "[]DescribeTopicPartitionsResponseTopic", "versions": "0+", + "about": "Each topic in the response.", "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The topic error, or 0 if there was no error." }, + { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "entityType": "topicName", "nullableVersions": "0+", + "about": "The topic name." }, + { "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The topic id." }, + { "name": "IsInternal", "type": "bool", "versions": "0+", "default": "false", "ignorable": true, + "about": "True if the topic is internal." }, + { "name": "Partitions", "type": "[]DescribeTopicPartitionsResponsePartition", "versions": "0+", + "about": "Each partition in the topic.", "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The partition error, or 0 if there was no error." }, + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", + "about": "The ID of the leader broker." }, + { "name": "LeaderEpoch", "type": "int32", "versions": "0+", "default": "-1", "ignorable": true, + "about": "The leader epoch of this partition." }, + { "name": "ReplicaNodes", "type": "[]int32", "versions": "0+", "entityType": "brokerId", + "about": "The set of all nodes that host this partition." }, + { "name": "IsrNodes", "type": "[]int32", "versions": "0+", "entityType": "brokerId", + "about": "The set of nodes that are in sync with the leader for this partition." }, + { "name": "EligibleLeaderReplicas", "type": "[]int32", "default": "null", "entityType": "brokerId", + "versions": "0+", "nullableVersions": "0+", + "about": "The new eligible leader replicas otherwise." }, + { "name": "LastKnownElr", "type": "[]int32", "default": "null", "entityType": "brokerId", + "versions": "0+", "nullableVersions": "0+", + "about": "The last known ELR." }, + { "name": "OfflineReplicas", "type": "[]int32", "versions": "0+", "ignorable": true, "entityType": "brokerId", + "about": "The set of offline replicas of this partition." }]}, + { "name": "TopicAuthorizedOperations", "type": "int32", "versions": "0+", "default": "-2147483648", + "about": "32-bit bitfield to represent authorized operations for this topic." }] + }, + { "name": "NextCursor", "type": "Cursor", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The next topic and partition index to fetch details for.", "fields": [ + { "name": "TopicName", "type": "string", "versions": "0+", + "about": "The name for the first topic to process", "versions": "0+", "entityType": "topicName"}, + { "name": "PartitionIndex", "type": "int32", "versions": "0+", "about": "The partition index to start with"} + ]} + ] +} diff --git a/tansu-kafka-sans-io/message/EndTxnRequest.json b/tansu-kafka-sans-io/message/EndTxnRequest.json index f16ef76..bc66adc 100644 --- a/tansu-kafka-sans-io/message/EndTxnRequest.json +++ b/tansu-kafka-sans-io/message/EndTxnRequest.json @@ -23,7 +23,9 @@ // Version 2 adds the support for new error code PRODUCER_FENCED. // // Version 3 enables flexible versions. - "validVersions": "0-3", + // + // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-4", "flexibleVersions": "3+", "fields": [ { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId", diff --git a/tansu-kafka-sans-io/message/EndTxnResponse.json b/tansu-kafka-sans-io/message/EndTxnResponse.json index 3071953..08ac6cd 100644 --- a/tansu-kafka-sans-io/message/EndTxnResponse.json +++ b/tansu-kafka-sans-io/message/EndTxnResponse.json @@ -22,7 +22,9 @@ // Version 2 adds the support for new error code PRODUCER_FENCED. // // Version 3 enables flexible versions. - "validVersions": "0-3", + // + // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-4", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", diff --git a/tansu-kafka-sans-io/message/FindCoordinatorRequest.json b/tansu-kafka-sans-io/message/FindCoordinatorRequest.json index 49ae7e9..42b2f4c 100644 --- a/tansu-kafka-sans-io/message/FindCoordinatorRequest.json +++ b/tansu-kafka-sans-io/message/FindCoordinatorRequest.json @@ -25,7 +25,9 @@ // Version 3 is the first flexible version. // // Version 4 adds support for batching via CoordinatorKeys (KIP-699) - "validVersions": "0-4", + // + // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-5", "deprecatedVersions": "0", "flexibleVersions": "3+", "fields": [ diff --git a/tansu-kafka-sans-io/message/FindCoordinatorResponse.json b/tansu-kafka-sans-io/message/FindCoordinatorResponse.json index 9309c01..860d655 100644 --- a/tansu-kafka-sans-io/message/FindCoordinatorResponse.json +++ b/tansu-kafka-sans-io/message/FindCoordinatorResponse.json @@ -24,7 +24,9 @@ // Version 3 is the first flexible version. // // Version 4 adds support for batching via Coordinators (KIP-699) - "validVersions": "0-4", + // + // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-5", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, diff --git a/tansu-kafka-sans-io/message/InitProducerIdRequest.json b/tansu-kafka-sans-io/message/InitProducerIdRequest.json index 4e75352..39f546d 100644 --- a/tansu-kafka-sans-io/message/InitProducerIdRequest.json +++ b/tansu-kafka-sans-io/message/InitProducerIdRequest.json @@ -25,7 +25,9 @@ // Version 3 adds ProducerId and ProducerEpoch, allowing producers to try to resume after an INVALID_PRODUCER_EPOCH error // // Version 4 adds the support for new error code PRODUCER_FENCED. - "validVersions": "0-4", + // + // Verison 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-5", "flexibleVersions": "2+", "fields": [ { "name": "TransactionalId", "type": "string", "versions": "0+", "nullableVersions": "0+", "entityType": "transactionalId", diff --git a/tansu-kafka-sans-io/message/InitProducerIdResponse.json b/tansu-kafka-sans-io/message/InitProducerIdResponse.json index f56c2fe..c5dfec6 100644 --- a/tansu-kafka-sans-io/message/InitProducerIdResponse.json +++ b/tansu-kafka-sans-io/message/InitProducerIdResponse.json @@ -24,7 +24,9 @@ // Version 3 is the same as version 2. // // Version 4 adds the support for new error code PRODUCER_FENCED. - "validVersions": "0-4", + // + // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-5", "flexibleVersions": "2+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, diff --git a/tansu-kafka-sans-io/message/ListClientMetricsResourcesResponse.json b/tansu-kafka-sans-io/message/ListClientMetricsResourcesResponse.json index 6d3321c..281781c 100644 --- a/tansu-kafka-sans-io/message/ListClientMetricsResourcesResponse.json +++ b/tansu-kafka-sans-io/message/ListClientMetricsResourcesResponse.json @@ -22,9 +22,12 @@ "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, - { "name": "ErrorCode", "type": "int16", "versions": "0+" }, - { "name": "ClientMetricsResources", "type": "[]ClientMetricsResource", "versions": "0+", "fields": [ - { "name": "Name", "type": "string", "versions": "0+" } + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + { "name": "ClientMetricsResources", "type": "[]ClientMetricsResource", "versions": "0+", + "about": "Each client metrics resource in the response.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", + "about": "The resource name." } ]} ] } diff --git a/tansu-kafka-sans-io/message/ListTransactionsRequest.json b/tansu-kafka-sans-io/message/ListTransactionsRequest.json index 21f4552..2aeeaa6 100644 --- a/tansu-kafka-sans-io/message/ListTransactionsRequest.json +++ b/tansu-kafka-sans-io/message/ListTransactionsRequest.json @@ -18,7 +18,8 @@ "type": "request", "listeners": ["zkBroker", "broker"], "name": "ListTransactionsRequest", - "validVersions": "0", + // Version 1: adds DurationFilter to list transactions older than specified duration + "validVersions": "0-1", "flexibleVersions": "0+", "fields": [ { "name": "StateFilters", "type": "[]string", "versions": "0+", @@ -26,6 +27,9 @@ }, { "name": "ProducerIdFilters", "type": "[]int64", "versions": "0+", "entityType": "producerId", "about": "The producerIds to filter by: if empty, all transactions will be returned; if non-empty, only transactions which match one of the filtered producerIds will be returned" + }, + { "name": "DurationFilter", "type": "int64", "versions": "1+", "default": -1, + "about": "Duration (in millis) to filter by: if < 0, all transactions will be returned; otherwise, only transactions running longer than this duration will be returned" } ] } diff --git a/tansu-kafka-sans-io/message/ListTransactionsResponse.json b/tansu-kafka-sans-io/message/ListTransactionsResponse.json index 2f17873..e992480 100644 --- a/tansu-kafka-sans-io/message/ListTransactionsResponse.json +++ b/tansu-kafka-sans-io/message/ListTransactionsResponse.json @@ -17,7 +17,8 @@ "apiKey": 66, "type": "response", "name": "ListTransactionsResponse", - "validVersions": "0", + // Version 1 is the same as version 0 (KIP-994). + "validVersions": "0-1", "flexibleVersions": "0+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", diff --git a/tansu-kafka-sans-io/message/MetadataResponse.json b/tansu-kafka-sans-io/message/MetadataResponse.json index 714b28b..408cdc7 100644 --- a/tansu-kafka-sans-io/message/MetadataResponse.json +++ b/tansu-kafka-sans-io/message/MetadataResponse.json @@ -48,7 +48,7 @@ { "name": "ThrottleTimeMs", "type": "int32", "versions": "3+", "ignorable": true, "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, { "name": "Brokers", "type": "[]MetadataResponseBroker", "versions": "0+", - "about": "Each broker in the response.", "fields": [ + "about": "A list of brokers present in the cluster.", "fields": [ { "name": "NodeId", "type": "int32", "versions": "0+", "mapKey": true, "entityType": "brokerId", "about": "The broker ID." }, { "name": "Host", "type": "string", "versions": "0+", @@ -67,8 +67,9 @@ { "name": "ErrorCode", "type": "int16", "versions": "0+", "about": "The topic error, or 0 if there was no error." }, { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "entityType": "topicName", "nullableVersions": "12+", - "about": "The topic name." }, - { "name": "TopicId", "type": "uuid", "versions": "10+", "ignorable": true, "about": "The topic id." }, + "about": "The topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated." }, + { "name": "TopicId", "type": "uuid", "versions": "10+", "ignorable": true, + "about": "The topic id. Zero for non-existing topics queried by name. This is never zero when ErrorCode is zero. One of Name and TopicId is always populated." }, { "name": "IsInternal", "type": "bool", "versions": "1+", "default": "false", "ignorable": true, "about": "True if the topic is internal." }, { "name": "Partitions", "type": "[]MetadataResponsePartition", "versions": "0+", diff --git a/tansu-kafka-sans-io/message/ProduceRequest.json b/tansu-kafka-sans-io/message/ProduceRequest.json index 3451a84..ae01fe5 100644 --- a/tansu-kafka-sans-io/message/ProduceRequest.json +++ b/tansu-kafka-sans-io/message/ProduceRequest.json @@ -35,7 +35,9 @@ // Version 9 enables flexible versions. // // Version 10 is the same as version 9 (KIP-951). - "validVersions": "0-10", + // + // Version 11 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-11", "deprecatedVersions": "0-6", "flexibleVersions": "9+", "fields": [ diff --git a/tansu-kafka-sans-io/message/ProduceResponse.json b/tansu-kafka-sans-io/message/ProduceResponse.json index d294fb8..92c7a22 100644 --- a/tansu-kafka-sans-io/message/ProduceResponse.json +++ b/tansu-kafka-sans-io/message/ProduceResponse.json @@ -34,7 +34,9 @@ // Version 9 enables flexible versions. // // Version 10 adds 'CurrentLeader' and 'NodeEndpoints' as tagged fields (KIP-951) - "validVersions": "0-10", + // + // Version 11 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-11", "flexibleVersions": "9+", "fields": [ { "name": "Responses", "type": "[]TopicProduceResponse", "versions": "0+", diff --git a/tansu-kafka-sans-io/message/TxnOffsetCommitRequest.json b/tansu-kafka-sans-io/message/TxnOffsetCommitRequest.json index 0e7b187..3cb63aa 100644 --- a/tansu-kafka-sans-io/message/TxnOffsetCommitRequest.json +++ b/tansu-kafka-sans-io/message/TxnOffsetCommitRequest.json @@ -23,7 +23,9 @@ // Version 2 adds the committed leader epoch. // // Version 3 adds the member.id, group.instance.id and generation.id. - "validVersions": "0-3", + // + // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-4", "flexibleVersions": "3+", "fields": [ { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId", diff --git a/tansu-kafka-sans-io/message/TxnOffsetCommitResponse.json b/tansu-kafka-sans-io/message/TxnOffsetCommitResponse.json index 96b03a0..1a04cef 100644 --- a/tansu-kafka-sans-io/message/TxnOffsetCommitResponse.json +++ b/tansu-kafka-sans-io/message/TxnOffsetCommitResponse.json @@ -22,7 +22,9 @@ // Version 2 is the same as version 1. // // Version 3 adds illegal generation, fenced instance id, and unknown member id errors. - "validVersions": "0-3", + // + // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-4", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", diff --git a/tansu-kafka-sans-io/src/lib.rs b/tansu-kafka-sans-io/src/lib.rs index 93690fd..26e60de 100644 --- a/tansu-kafka-sans-io/src/lib.rs +++ b/tansu-kafka-sans-io/src/lib.rs @@ -93,6 +93,7 @@ pub enum Error { EnvVar(VarError), FromUtf8(string::FromUtf8Error), InvalidAckValue(i16), + InvalidCoordinatorType(i8), InvalidIsolationLevel(i8), Io(io::Error), Message(String), @@ -981,9 +982,12 @@ impl Display for ErrorCode { } #[non_exhaustive] -#[derive(Clone, Copy, Eq, Hash, Debug, Ord, PartialEq, PartialOrd)] +#[derive( + Clone, Copy, Default, Deserialize, Eq, Hash, Debug, Ord, PartialEq, PartialOrd, Serialize, +)] pub enum ErrorCode { UnknownServerError, + #[default] None, OffsetOutOfRange, CorruptMessage, @@ -1106,8 +1110,9 @@ pub enum ErrorCode { InvalidRegistration, } -#[derive(Clone, Copy, Debug, Deserialize, Eq, Ord, PartialEq, PartialOrd, Serialize)] +#[derive(Clone, Copy, Debug, Default, Deserialize, Eq, Ord, PartialEq, PartialOrd, Serialize)] pub enum IsolationLevel { + #[default] ReadUncommitted, ReadCommitted, } @@ -1144,6 +1149,36 @@ impl TryFrom for Ack { } } +#[derive(Clone, Debug, Default, Deserialize, Eq, Hash, Ord, PartialEq, PartialOrd, Serialize)] +pub enum TimestampType { + #[default] + CreateTime, + LogAppendTime, +} + +impl TimestampType { + const TIMESTAMP_TYPE_BITMASK: i16 = 8; +} + +impl From for TimestampType { + fn from(value: i16) -> Self { + if value & Self::TIMESTAMP_TYPE_BITMASK == Self::TIMESTAMP_TYPE_BITMASK { + Self::LogAppendTime + } else { + Self::CreateTime + } + } +} + +impl From for i16 { + fn from(value: TimestampType) -> Self { + match value { + TimestampType::CreateTime => 0, + TimestampType::LogAppendTime => TimestampType::TIMESTAMP_TYPE_BITMASK, + } + } +} + #[derive(Clone, Debug, Default, Deserialize, Eq, Hash, Ord, PartialEq, PartialOrd, Serialize)] pub enum Compression { #[default] @@ -1223,6 +1258,88 @@ impl Compression { } } +#[derive(Clone, Debug, Default, Deserialize, Eq, Hash, Ord, PartialEq, PartialOrd, Serialize)] +pub struct BatchAttribute { + pub compression: Compression, + pub timestamp: TimestampType, + pub transaction: bool, + pub control: bool, + pub delete_horizon: bool, +} + +impl BatchAttribute { + const TRANSACTION_BITMASK: i16 = 16; + const CONTROL_BITMASK: i16 = 32; + const DELETE_HORIZON_BITMASK: i16 = 64; + + pub fn compression(self, compression: Compression) -> Self { + Self { + compression, + ..self + } + } + + pub fn timestamp(self, timestamp: TimestampType) -> Self { + Self { timestamp, ..self } + } + + pub fn transaction(self, transaction: bool) -> Self { + Self { + transaction, + ..self + } + } + + pub fn control(self, control: bool) -> Self { + Self { control, ..self } + } + + pub fn delete_horizon(self, delete_horizon: bool) -> Self { + Self { + delete_horizon, + ..self + } + } +} + +impl From for i16 { + fn from(value: BatchAttribute) -> Self { + let mut attributes = i16::from(value.compression); + attributes |= i16::from(value.timestamp); + + if value.transaction { + attributes |= BatchAttribute::TRANSACTION_BITMASK; + } + + if value.control { + attributes |= BatchAttribute::CONTROL_BITMASK; + } + + if value.delete_horizon { + attributes |= BatchAttribute::DELETE_HORIZON_BITMASK; + } + + attributes + } +} + +impl TryFrom for BatchAttribute { + type Error = Error; + + fn try_from(value: i16) -> Result { + Compression::try_from(value).map(|compression| { + Self::default() + .compression(compression) + .timestamp(TimestampType::from(value)) + .transaction(value & Self::TRANSACTION_BITMASK == Self::TRANSACTION_BITMASK) + .control(value & Self::CONTROL_BITMASK == Self::CONTROL_BITMASK) + .delete_horizon( + value & Self::DELETE_HORIZON_BITMASK == Self::DELETE_HORIZON_BITMASK, + ) + }) + } +} + pub enum EndpointType { Unknown, Broker, @@ -1249,6 +1366,25 @@ impl From for i8 { } } +pub enum CoordinatorType { + Group, + Transaction, + Share, +} + +impl TryFrom for CoordinatorType { + type Error = Error; + + fn try_from(value: i8) -> Result { + match value { + 0 => Ok(Self::Group), + 1 => Ok(Self::Transaction), + 2 => Ok(Self::Share), + otherwise => Err(Error::InvalidCoordinatorType(otherwise)), + } + } +} + #[derive(Clone, Copy, Debug, Eq, Hash, Ord, PartialEq, PartialOrd)] pub enum ConfigResource { Group, @@ -1272,6 +1408,16 @@ impl From for ConfigResource { } } +impl From for i8 { + fn from(value: CoordinatorType) -> Self { + match value { + CoordinatorType::Group => 0, + CoordinatorType::Transaction => 1, + CoordinatorType::Share => 2, + } + } +} + impl From for i8 { fn from(value: ConfigResource) -> Self { match value { @@ -1405,4 +1551,44 @@ pub fn to_timestamp(system_time: SystemTime) -> Result { .and_then(|since_epoch| i64::try_from(since_epoch).map_err(Into::into)) } +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn batch_attribute() { + assert_eq!(0, i16::from(BatchAttribute::default())); + assert_eq!( + 0, + i16::from(BatchAttribute::default().compression(Compression::None)) + ); + assert_eq!( + 1, + i16::from(BatchAttribute::default().compression(Compression::Gzip)) + ); + assert_eq!( + 2, + i16::from(BatchAttribute::default().compression(Compression::Snappy)) + ); + assert_eq!( + 3, + i16::from(BatchAttribute::default().compression(Compression::Lz4)) + ); + assert_eq!( + 4, + i16::from(BatchAttribute::default().compression(Compression::Zstd)) + ); + assert_eq!( + 8, + i16::from(BatchAttribute::default().timestamp(TimestampType::LogAppendTime)) + ); + assert_eq!(16, i16::from(BatchAttribute::default().transaction(true))); + assert_eq!(32, i16::from(BatchAttribute::default().control(true))); + assert_eq!( + 64, + i16::from(BatchAttribute::default().delete_horizon(true)) + ); + } +} + include!(concat!(env!("OUT_DIR"), "/generate.rs")); diff --git a/tansu-kafka-sans-io/src/record/deflated.rs b/tansu-kafka-sans-io/src/record/deflated.rs index 0e4e9dd..14a6dc8 100644 --- a/tansu-kafka-sans-io/src/record/deflated.rs +++ b/tansu-kafka-sans-io/src/record/deflated.rs @@ -66,6 +66,19 @@ pub struct Batch { pub record_data: Bytes, } +impl Batch { + const TRANSACTIONAL_BITMASK: i16 = 0b1_0000i16; + const CONTROL_BITMASK: i16 = 0b10_0000i16; + + pub fn is_transactional(&self) -> bool { + self.attributes & Self::TRANSACTIONAL_BITMASK == Self::TRANSACTIONAL_BITMASK + } + + pub fn is_control(&self) -> bool { + self.attributes & Self::CONTROL_BITMASK == Self::CONTROL_BITMASK + } +} + #[derive(Clone, Debug, Default, Eq, Hash, Ord, PartialEq, PartialOrd, Serialize)] pub struct CrcData { pub attributes: i16, @@ -723,4 +736,71 @@ mod tests { Ok(()) } + + #[test] + pub fn is_transactional() -> Result<()> { + let _guard = init_tracing()?; + + let batch = Batch { + base_offset: 0, + batch_length: 68, + partition_leader_epoch: 0, + magic: 2, + crc: 3650210183, + attributes: 16, + last_offset_delta: 0, + base_timestamp: 1729509915759, + max_timestamp: 1729509915759, + producer_id: 5, + producer_epoch: 0, + base_sequence: 0, + record_count: 1, + record_data: Bytes::from_static(b"$\0\0\0\x08\0\0\0\0\x10test0-ok\0"), + }; + + assert!(batch.is_transactional()); + + Ok(()) + } + + #[test] + pub fn is_transactional_control() -> Result<()> { + use crate::record::inflated; + + let _guard = init_tracing()?; + + let deflated = Batch { + base_offset: 1, + batch_length: 66, + partition_leader_epoch: 0, + magic: 2, + crc: 820655041, + attributes: 48, + last_offset_delta: 0, + base_timestamp: 1729509916024, + max_timestamp: 1729509916024, + producer_id: 5, + producer_epoch: 0, + base_sequence: -1, + record_count: 1, + record_data: Bytes::from_static(b" \0\0\0\x08\0\0\0\x01\x0c\0\0\0\0\0\0\0"), + }; + + assert!(deflated.is_transactional()); + assert!(deflated.is_control()); + + let inflated = inflated::Batch::try_from(deflated)?; + + assert_eq!(1, inflated.records.len()); + assert_eq!( + Some(Bytes::from_static(b"\0\0\0\x01")), + inflated.records[0].key + ); + assert_eq!( + Some(Bytes::from_static(b"\0\0\0\0\0\0")), + inflated.records[0].value + ); + + Ok(()) + } } diff --git a/tansu-kafka-sans-io/tests/add_partitions_to_txn.rs b/tansu-kafka-sans-io/tests/add_partitions_to_txn.rs index 18c4c5a..0aa056c 100644 --- a/tansu-kafka-sans-io/tests/add_partitions_to_txn.rs +++ b/tansu-kafka-sans-io/tests/add_partitions_to_txn.rs @@ -26,7 +26,7 @@ fn check_message_meta() { let message = meta.get("AddPartitionsToTxnRequest").unwrap(); assert_eq!(24, message.api_key); assert_eq!(MessageKind::Request, message.message_kind); - assert_eq!(VersionRange { start: 0, end: 4 }, message.version.valid); + assert_eq!(VersionRange { start: 0, end: 5 }, message.version.valid); assert_eq!( VersionRange { start: 3, diff --git a/tansu-kafka-sans-io/tests/decode.rs b/tansu-kafka-sans-io/tests/decode.rs index 0fb4d91..f308e3f 100644 --- a/tansu-kafka-sans-io/tests/decode.rs +++ b/tansu-kafka-sans-io/tests/decode.rs @@ -14,7 +14,6 @@ // along with this program. If not, see . use bytes::Bytes; -//use pretty_assertions::assert_eq; use serde::Deserialize; use std::{fs::File, io::Cursor, sync::Arc, thread}; use tansu_kafka_sans_io::{ @@ -27,6 +26,7 @@ use tansu_kafka_sans_io::{ EpochEndOffset, FetchableTopicResponse, LeaderIdAndEpoch, PartitionData, SnapshotId, }, join_group_response::JoinGroupResponseMember, + list_transactions_response::TransactionState, metadata_request::MetadataRequestTopic, metadata_response::{MetadataResponseBroker, MetadataResponsePartition, MetadataResponseTopic}, offset_fetch_response::{OffsetFetchResponsePartition, OffsetFetchResponseTopic}, @@ -3664,6 +3664,78 @@ fn list_partition_reassignments_request_v0_000() -> Result<()> { Ok(()) } +#[ignore] +#[test] +fn list_transactions_request_v1_000() -> Result<()> { + let _guard = init_tracing()?; + + let v = vec![ + 0, 0, 0, 35, 0, 66, 0, 1, 0, 0, 0, 4, 0, 13, 97, 100, 109, 105, 110, 99, 108, 105, 101, + 110, 116, 45, 49, 0, 1, 1, 255, 255, 255, 255, 255, 255, 255, 255, 0, + ]; + + let mut c = Cursor::new(v); + let mut deserializer = Decoder::request(&mut c); + + assert_eq!( + Frame { + size: 35, + header: Header::Request { + api_key: 66, + api_version: 1, + correlation_id: 4, + client_id: Some("adminclient-1".into()) + }, + body: Body::ListTransactionsRequest { + state_filters: Some([].into()), + producer_id_filters: Some([].into()), + duration_filter: Some(-1), + } + }, + Frame::deserialize(&mut deserializer)? + ); + + Ok(()) +} + +#[test] +fn list_transactions_response_v1_000() -> Result<()> { + let _guard = init_tracing()?; + + let api_key = 66; + let api_version = 1; + + let v = vec![ + 0, 0, 0, 70, 0, 0, 0, 4, 0, 0, 0, 0, 0, 0, 0, 1, 2, 32, 108, 105, 98, 114, 100, 107, 97, + 102, 107, 97, 95, 116, 114, 97, 110, 115, 97, 99, 116, 105, 111, 110, 115, 95, 101, 120, + 97, 109, 112, 108, 101, 0, 0, 0, 0, 0, 0, 0, 0, 15, 67, 111, 109, 112, 108, 101, 116, 101, + 67, 111, 109, 109, 105, 116, 0, 0, + ]; + + assert_eq!( + Frame { + size: 70, + header: Header::Response { correlation_id: 4 }, + body: Body::ListTransactionsResponse { + throttle_time_ms: 0, + error_code: 0, + unknown_state_filters: Some([].into()), + transaction_states: Some( + [TransactionState { + transactional_id: "librdkafka_transactions_example".into(), + producer_id: 0, + transaction_state: "CompleteCommit".into() + }] + .into() + ) + } + }, + Frame::response_from_bytes(&v, api_key, api_version)? + ); + + Ok(()) +} + #[test] fn metadata_request_v1_000() -> Result<()> { use tansu_kafka_sans_io::metadata_request::MetadataRequestTopic; diff --git a/tansu-kafka-sans-io/tests/find_coordinator.rs b/tansu-kafka-sans-io/tests/find_coordinator.rs index 753d161..cbbbd56 100644 --- a/tansu-kafka-sans-io/tests/find_coordinator.rs +++ b/tansu-kafka-sans-io/tests/find_coordinator.rs @@ -27,7 +27,7 @@ fn check_message_meta() { assert_eq!(10, message.api_key); assert_eq!(MessageKind::Request, message.message_kind); - assert_eq!(VersionRange { start: 0, end: 4 }, message.version.valid); + assert_eq!(VersionRange { start: 0, end: 5 }, message.version.valid); assert_eq!( Some(VersionRange { start: 0, end: 3 }), diff --git a/tansu-server/src/broker.rs b/tansu-server/src/broker.rs index f5da0a8..b65aabb 100644 --- a/tansu-server/src/broker.rs +++ b/tansu-server/src/broker.rs @@ -28,11 +28,9 @@ pub mod list_partition_reassignments; pub mod metadata; pub mod produce; pub mod telemetry; +pub mod txn; -use crate::{ - coordinator::group::{Coordinator, OffsetCommit}, - Error, Result, -}; +use crate::{coordinator::group::Coordinator, Error, Result}; use api_versions::ApiVersionsRequest; use create_topic::CreateTopic; use delete_records::DeleteRecordsRequest; @@ -47,14 +45,17 @@ use list_partition_reassignments::ListPartitionReassignmentsRequest; use metadata::MetadataRequest; use produce::ProduceRequest; use std::io::ErrorKind; -use tansu_kafka_sans_io::{broker_registration_request::Listener, Body, Frame, Header}; +use tansu_kafka_sans_io::{ + broker_registration_request::Listener, Body, Frame, Header, IsolationLevel, +}; use tansu_storage::{BrokerRegistationRequest, Storage}; use telemetry::GetTelemetrySubscriptionsRequest; use tokio::{ io::{AsyncReadExt, AsyncWriteExt}, net::{TcpListener, TcpStream}, }; -use tracing::{debug, error, info}; +use tracing::{debug, error, info, span, Instrument, Level}; +use txn::{add_offsets::AddOffsets, add_partitions::AddPartitions}; use url::Url; use uuid::Uuid; @@ -146,17 +147,25 @@ where let mut broker = self.clone(); _ = tokio::spawn(async move { - match broker.stream_handler(stream).await { - Err(ref error @ Error::Io(ref io)) if io.kind() == ErrorKind::UnexpectedEof => { - info!(?error); - } + let span = span!(Level::DEBUG, "peer", addr = %addr); - Err(error) => { - error!(?error); - } + async move { + match broker.stream_handler(stream).await { + Err(ref error @ Error::Io(ref io)) + if io.kind() == ErrorKind::UnexpectedEof => + { + info!(?error); + } - Ok(_) => {} + Err(error) => { + error!(?error); + } + + Ok(_) => {} + } } + .instrument(span) + .await }); } } @@ -220,10 +229,23 @@ where .. } => { debug!(?api_key, ?api_version, ?correlation_id); - let body = self - .response_for(client_id.as_deref(), body, correlation_id) - .await - .inspect_err(|err| error!(?err))?; + + let span = span!( + Level::DEBUG, + "request", + api_key, + api_version, + correlation_id + ); + + let body = async move { + self.response_for(client_id.as_deref(), body, correlation_id) + .await + .inspect_err(|err| error!(?err)) + } + .instrument(span) + .await?; + debug!(?body, ?correlation_id); Frame::response( Header::Response { correlation_id }, @@ -415,13 +437,20 @@ where ?producer_epoch, ); - let init_producer_id = InitProducerIdRequest; - Ok(init_producer_id.response( - transactional_id.as_deref(), - transaction_timeout_ms, - producer_id, - producer_epoch, - )) + InitProducerIdRequest::with_storage(self.storage.clone()) + .response( + transactional_id.as_deref(), + transaction_timeout_ms, + producer_id, + producer_epoch, + ) + .await + .map(|response| Body::InitProducerIdResponse { + throttle_time_ms: 0, + error_code: response.error.into(), + producer_id: response.id, + producer_epoch: response.epoch, + }) } Body::JoinGroupRequest { @@ -479,6 +508,11 @@ where } => { debug!(?replica_id, ?isolation_level, ?topics); + let isolation_level = isolation_level + .map_or(Ok(IsolationLevel::ReadUncommitted), |isolation_level| { + IsolationLevel::try_from(isolation_level) + })?; + ListOffsetsRequest::with_storage(self.storage.clone()) .response(replica_id, isolation_level, topics.as_deref()) .await @@ -516,7 +550,7 @@ where ?topics ); - let detail = OffsetCommit { + let detail = crate::coordinator::group::OffsetCommit { group_id: group_id.as_str(), generation_id_or_member_epoch, member_id: member_id.as_deref(), @@ -555,6 +589,11 @@ where ProduceRequest::with_storage(self.storage.clone()) .response(transactional_id, acks, timeout_ms, topic_data) .await + .map(|response| Body::ProduceResponse { + responses: response.responses, + throttle_time_ms: response.throttle_time_ms, + node_endpoints: response.node_endpoints, + }) } Body::SyncGroupRequest { @@ -579,7 +618,89 @@ where .await } - _ => unimplemented!(), + Body::AddOffsetsToTxnRequest { + transactional_id, + producer_id, + producer_epoch, + group_id, + } => { + debug!(?transactional_id, ?producer_id, ?producer_epoch, ?group_id); + + AddOffsets::with_storage(self.storage.clone()) + .response( + transactional_id.as_str(), + producer_id, + producer_epoch, + group_id.as_str(), + ) + .await + } + + add_partitions @ Body::AddPartitionsToTxnRequest { .. } => { + AddPartitions::with_storage(self.storage.clone()) + .response(add_partitions.try_into()?) + .await + } + + Body::TxnOffsetCommitRequest { + transactional_id, + group_id, + producer_id, + producer_epoch, + generation_id, + member_id, + group_instance_id, + topics, + } => { + debug!( + ?transactional_id, + ?group_id, + ?producer_id, + ?producer_epoch, + ?generation_id, + ?member_id, + ?group_instance_id, + ?topics, + ); + + txn::offset_commit::OffsetCommit::with_storage(self.storage.clone()) + .response( + transactional_id.as_str(), + group_id.as_str(), + producer_id, + producer_epoch, + generation_id, + member_id, + group_instance_id, + topics, + ) + .await + } + + Body::EndTxnRequest { + transactional_id, + producer_id, + producer_epoch, + committed, + } => self + .storage + .txn_end( + transactional_id.as_str(), + producer_id, + producer_epoch, + committed, + ) + .await + .map(|error_code| Body::EndTxnResponse { + throttle_time_ms: 0, + error_code: i16::from(error_code), + }) + .map_err(Into::into), + + request => { + error!(?request); + unimplemented!("{request:?}") + } } } } diff --git a/tansu-server/src/broker/api_versions.rs b/tansu-server/src/broker/api_versions.rs index 71b1538..f735b97 100644 --- a/tansu-server/src/broker/api_versions.rs +++ b/tansu-server/src/broker/api_versions.rs @@ -13,18 +13,24 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . +use std::sync::LazyLock; + use tansu_kafka_sans_io::{api_versions_response::ApiVersion, Body, ErrorCode, RootMessageMeta}; #[derive(Clone, Copy, Debug, Default, Eq, Hash, Ord, PartialEq, PartialOrd)] pub struct ApiVersionsRequest; -fn is_sasl(api_key: &i16) -> bool { - [17].contains(api_key) -} +const TELEMETRY: [i16; 3] = [71, 72, 74]; +const SASL: [i16; 1] = [17]; +const DESCRIBE_TOPIC_PARTITIONS: [i16; 1] = [75]; -fn is_telemetry(api_key: &i16) -> bool { - [71, 72, 74].contains(api_key) -} +static UNSUPPORTED: LazyLock> = LazyLock::new(|| { + let mut unsupported = vec![]; + unsupported.extend_from_slice(&TELEMETRY); + unsupported.extend_from_slice(&SASL); + unsupported.extend_from_slice(&DESCRIBE_TOPIC_PARTITIONS); + unsupported +}); impl ApiVersionsRequest { pub fn response( @@ -45,7 +51,7 @@ impl ApiVersionsRequest { RootMessageMeta::messages() .requests() .iter() - .filter(|(api_key, _)| !(is_telemetry(api_key) || is_sasl(api_key))) + .filter(|(api_key, _)| !UNSUPPORTED.contains(api_key)) .map(|(_, meta)| ApiVersion { api_key: meta.api_key, min_version: meta.version.valid.start, diff --git a/tansu-server/src/broker/create_topic.rs b/tansu-server/src/broker/create_topic.rs index fc35e4d..fb2a699 100644 --- a/tansu-server/src/broker/create_topic.rs +++ b/tansu-server/src/broker/create_topic.rs @@ -34,7 +34,7 @@ where } async fn create_topic( - &self, + &mut self, mut topic: CreatableTopic, validate_only: bool, ) -> CreatableTopicResult { @@ -97,7 +97,7 @@ where } pub async fn response( - &self, + &mut self, creatable: Option>, validate_only: bool, ) -> Result> { @@ -130,7 +130,7 @@ mod tests { let storage = DynoStore::new(cluster, node, InMemory::new()); - let create_topic = CreateTopic::with_storage(storage); + let mut create_topic = CreateTopic::with_storage(storage); let name = "pqr"; let num_partitions = 5; @@ -169,7 +169,7 @@ mod tests { let storage = DynoStore::new(cluster, node, InMemory::new()); - let create_topic = CreateTopic::with_storage(storage); + let mut create_topic = CreateTopic::with_storage(storage); let name = "pqr"; let num_partitions = -1; @@ -208,7 +208,7 @@ mod tests { let storage = DynoStore::new(cluster, node, InMemory::new()); - let create_topic = CreateTopic::with_storage(storage); + let mut create_topic = CreateTopic::with_storage(storage); let name = "pqr"; let num_partitions = 5; diff --git a/tansu-server/src/broker/delete_records.rs b/tansu-server/src/broker/delete_records.rs index d0f545c..6b9e421 100644 --- a/tansu-server/src/broker/delete_records.rs +++ b/tansu-server/src/broker/delete_records.rs @@ -31,7 +31,7 @@ where Self { storage } } - pub async fn request(&self, topics: &[DeleteRecordsTopic]) -> Result { + pub async fn request(&mut self, topics: &[DeleteRecordsTopic]) -> Result { let topics = self .storage .delete_records(topics) diff --git a/tansu-server/src/broker/delete_topics.rs b/tansu-server/src/broker/delete_topics.rs index c57b7ef..63549eb 100644 --- a/tansu-server/src/broker/delete_topics.rs +++ b/tansu-server/src/broker/delete_topics.rs @@ -34,7 +34,7 @@ where } pub async fn response( - &self, + &mut self, topics: Option>, topic_names: Option>, ) -> Result> { @@ -88,7 +88,7 @@ mod tests { let cluster = "abc"; let node = 12321; - let storage = DynoStore::new(cluster, node, InMemory::new()); + let mut storage = DynoStore::new(cluster, node, InMemory::new()); let topic = "pqr"; @@ -105,7 +105,7 @@ mod tests { let cluster = "abc"; let node = 12321; - let storage = DynoStore::new(cluster, node, InMemory::new()); + let mut storage = DynoStore::new(cluster, node, InMemory::new()); let topic = Uuid::new_v4(); diff --git a/tansu-server/src/broker/describe_cluster.rs b/tansu-server/src/broker/describe_cluster.rs index 09664f9..c354eef 100644 --- a/tansu-server/src/broker/describe_cluster.rs +++ b/tansu-server/src/broker/describe_cluster.rs @@ -16,6 +16,7 @@ use crate::Result; use tansu_kafka_sans_io::{Body, ErrorCode}; use tansu_storage::Storage; +use tracing::debug; #[derive(Clone, Debug, Default, Eq, Hash, Ord, PartialEq, PartialOrd)] pub struct DescribeClusterRequest { @@ -28,13 +29,14 @@ where S: Storage, { pub async fn response( - &self, + &mut self, include_cluster_authorized_operations: bool, endpoint_type: Option, ) -> Result { - let _ = include_cluster_authorized_operations; + debug!(?include_cluster_authorized_operations, ?endpoint_type); let brokers = self.storage.brokers().await?; + debug!(?brokers); Ok(Body::DescribeClusterResponse { throttle_time_ms: 0, diff --git a/tansu-server/src/broker/describe_configs.rs b/tansu-server/src/broker/describe_configs.rs index 08667cb..e1a19bb 100644 --- a/tansu-server/src/broker/describe_configs.rs +++ b/tansu-server/src/broker/describe_configs.rs @@ -34,7 +34,7 @@ where } pub async fn response( - &self, + &mut self, resources: Option<&[DescribeConfigsResource]>, include_synonyms: Option, include_documentation: Option, diff --git a/tansu-server/src/broker/fetch.rs b/tansu-server/src/broker/fetch.rs index c2c26db..85fc22c 100644 --- a/tansu-server/src/broker/fetch.rs +++ b/tansu-server/src/broker/fetch.rs @@ -48,7 +48,7 @@ where max_wait_ms: Duration, min_bytes: u32, max_bytes: &mut u32, - isolation: Option, + isolation: IsolationLevel, topic: &str, fetch_partition: &FetchPartition, ) -> Result { @@ -74,7 +74,7 @@ where let mut fetched = self .storage - .fetch(&tp, offset, min_bytes, *max_bytes) + .fetch(&tp, offset, min_bytes, *max_bytes, isolation) .await .inspect(|r| debug!(?tp, ?offset, ?r)) .inspect_err(|error| error!(?tp, ?error)) @@ -158,7 +158,7 @@ where max_wait_ms: Duration, min_bytes: u32, max_bytes: &mut u32, - isolation: Option, + isolation: IsolationLevel, fetch: &FetchTopic, _is_first: bool, ) -> Result { @@ -204,7 +204,7 @@ where max_wait: Duration, min_bytes: u32, max_bytes: &mut u32, - isolation: Option, + isolation: IsolationLevel, topics: &[FetchTopic], ) -> Result> { debug!(?max_wait, ?min_bytes, ?isolation, ?topics); @@ -272,9 +272,10 @@ where debug!(?max_wait_ms, ?min_bytes, ?max_bytes, ?topics); let responses = Some(if let Some(topics) = topics { - let isolation_level = isolation_level.map_or(Ok(None), |isolation| { - IsolationLevel::try_from(isolation).map(Some) - })?; + let isolation_level = isolation_level + .map_or(Ok(IsolationLevel::ReadUncommitted), |isolation| { + IsolationLevel::try_from(isolation) + })?; let max_wait_ms = u64::try_from(max_wait_ms).map(Duration::from_millis)?; diff --git a/tansu-server/src/broker/init_producer_id.rs b/tansu-server/src/broker/init_producer_id.rs index 6f7ff66..dd7ce55 100644 --- a/tansu-server/src/broker/init_producer_id.rs +++ b/tansu-server/src/broker/init_producer_id.rs @@ -13,29 +13,138 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use tansu_kafka_sans_io::{Body, ErrorCode}; +use crate::Result; +use tansu_storage::{ProducerIdResponse, Storage}; +use tracing::debug; #[derive(Clone, Copy, Debug, Default, Eq, Hash, Ord, PartialEq, PartialOrd)] -pub struct InitProducerIdRequest; +pub struct InitProducerIdRequest { + storage: S, +} + +impl InitProducerIdRequest +where + S: Storage, +{ + pub fn with_storage(storage: S) -> Self { + Self { storage } + } -impl InitProducerIdRequest { - pub fn response( - &self, - transactional_id: Option<&str>, + pub async fn response( + &mut self, + transaction_id: Option<&str>, transaction_timeout_ms: i32, producer_id: Option, producer_epoch: Option, - ) -> Body { - let _ = transactional_id; - let _ = transaction_timeout_ms; - let _ = producer_id; - let _ = producer_epoch; - - Body::InitProducerIdResponse { - throttle_time_ms: 0, - error_code: ErrorCode::None.into(), - producer_id: 1, - producer_epoch: 0, - } + ) -> Result { + debug!( + ?transaction_id, + ?transaction_timeout_ms, + ?producer_id, + ?producer_epoch + ); + + self.storage + .init_producer( + transaction_id, + transaction_timeout_ms, + producer_id, + producer_epoch, + ) + .await + .map_err(Into::into) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::Error; + use object_store::memory::InMemory; + use tansu_kafka_sans_io::ErrorCode; + use tansu_storage::dynostore::DynoStore; + use tracing::subscriber::DefaultGuard; + + #[cfg(miri)] + fn init_tracing() -> Result<()> { + Ok(()) + } + + #[cfg(not(miri))] + fn init_tracing() -> Result { + use std::{fs::File, sync::Arc, thread}; + + use tracing::Level; + use tracing_subscriber::fmt::format::FmtSpan; + + Ok(tracing::subscriber::set_default( + tracing_subscriber::fmt() + .with_level(true) + .with_line_number(true) + .with_thread_names(false) + .with_max_level(Level::DEBUG) + .with_span_events(FmtSpan::ACTIVE) + .with_writer( + thread::current() + .name() + .ok_or(Error::Custom(String::from("unnamed thread"))) + .and_then(|name| { + File::create(format!("../logs/{}/{name}.log", env!("CARGO_PKG_NAME"))) + .map_err(Into::into) + }) + .map(Arc::new)?, + ) + .finish(), + )) + } + + #[tokio::test] + async fn no_txn_init_producer_id() -> Result<()> { + let _guard = init_tracing()?; + + let cluster = "abc"; + let node = 12321; + + let transaction_id = None; + let transaction_timeout_ms = 0; + let producer_id = Some(-1); + let producer_epoch = Some(-1); + + let mut request = + InitProducerIdRequest::with_storage(DynoStore::new(cluster, node, InMemory::new())); + + assert_eq!( + ProducerIdResponse { + error: ErrorCode::None, + id: 1, + epoch: 0 + }, + request + .response( + transaction_id, + transaction_timeout_ms, + producer_id, + producer_epoch, + ) + .await? + ); + + assert_eq!( + ProducerIdResponse { + error: ErrorCode::None, + id: 2, + epoch: 0 + }, + request + .response( + transaction_id, + transaction_timeout_ms, + producer_id, + producer_epoch, + ) + .await? + ); + + Ok(()) } } diff --git a/tansu-server/src/broker/list_offsets.rs b/tansu-server/src/broker/list_offsets.rs index 7a56f39..f509e2d 100644 --- a/tansu-server/src/broker/list_offsets.rs +++ b/tansu-server/src/broker/list_offsets.rs @@ -18,7 +18,7 @@ use std::{collections::BTreeSet, ops::Deref}; use tansu_kafka_sans_io::{ list_offsets_request::ListOffsetsTopic, list_offsets_response::{ListOffsetsPartitionResponse, ListOffsetsTopicResponse}, - Body, + Body, IsolationLevel, }; use tansu_storage::{ListOffsetRequest, Storage, Topition}; use tracing::{debug, error}; @@ -39,9 +39,9 @@ where } pub async fn response( - &self, + &mut self, replica_id: i32, - isolation_level: Option, + isolation_level: IsolationLevel, topics: Option<&[ListOffsetsTopic]>, ) -> Result { debug!(?replica_id, ?isolation_level, ?topics); @@ -64,7 +64,7 @@ where Some( self.storage - .list_offsets(offsets.deref()) + .list_offsets(isolation_level, offsets.deref()) .await .inspect(|r| debug!(?r, ?offsets)) .inspect_err(|err| error!(?err, ?offsets)) diff --git a/tansu-server/src/broker/list_partition_reassignments.rs b/tansu-server/src/broker/list_partition_reassignments.rs index ec4d17c..ac00376 100644 --- a/tansu-server/src/broker/list_partition_reassignments.rs +++ b/tansu-server/src/broker/list_partition_reassignments.rs @@ -37,7 +37,7 @@ where } pub async fn response( - &self, + &mut self, topics: Option<&[ListPartitionReassignmentsTopics]>, ) -> Result { let topics = topics.map(|topics| { diff --git a/tansu-server/src/broker/metadata.rs b/tansu-server/src/broker/metadata.rs index 8cb4659..4674620 100644 --- a/tansu-server/src/broker/metadata.rs +++ b/tansu-server/src/broker/metadata.rs @@ -32,7 +32,7 @@ where Self { storage } } - pub async fn response(&self, topics: Option>) -> Result { + pub async fn response(&mut self, topics: Option>) -> Result { let throttle_time_ms = Some(0); let topics = topics.map(|topics| topics.iter().map(TopicId::from).collect::>()); diff --git a/tansu-server/src/broker/produce.rs b/tansu-server/src/broker/produce.rs index 7423bbd..a192826 100644 --- a/tansu-server/src/broker/produce.rs +++ b/tansu-server/src/broker/produce.rs @@ -13,11 +13,11 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use crate::Result; +use crate::{Error, Result}; use tansu_kafka_sans_io::{ produce_request::{PartitionProduceData, TopicProduceData}, - produce_response::{PartitionProduceResponse, TopicProduceResponse}, - Body, ErrorCode, + produce_response::{NodeEndpoint, PartitionProduceResponse, TopicProduceResponse}, + ErrorCode, }; use tansu_storage::{Storage, Topition}; use tracing::{debug, error}; @@ -27,6 +27,13 @@ pub struct ProduceRequest { storage: S, } +#[derive(Clone, Debug, Default, Eq, Hash, Ord, PartialEq, PartialOrd)] +pub struct ProduceResponse { + pub responses: Option>, + pub throttle_time_ms: Option, + pub node_endpoints: Option>, +} + impl ProduceRequest where S: Storage, @@ -49,7 +56,8 @@ where } async fn partition( - &self, + &mut self, + transaction_id: Option<&str>, name: &str, partition: PartitionProduceData, ) -> PartitionProduceResponse { @@ -59,13 +67,14 @@ where let tp = Topition::new(name, partition.index); - if let Ok(base_offset) = self + match self .storage - .produce(&tp, batch) + .produce(transaction_id, &tp, batch) .await + .map_err(Into::into) .inspect_err(|err| error!(?err)) { - PartitionProduceResponse { + Ok(base_offset) => PartitionProduceResponse { index: partition.index, error_code: ErrorCode::None.into(), base_offset, @@ -74,9 +83,14 @@ where record_errors: Some([].into()), error_message: None, current_leader: None, + }, + + Err(Error::Storage(tansu_storage::Error::Api(error_code))) => { + debug!(?self, ?error_code); + self.error(partition.index, error_code) } - } else { - self.error(partition.index, ErrorCode::UnknownServerError) + + Err(_) => self.error(partition.index, ErrorCode::UnknownServerError), } } @@ -84,12 +98,16 @@ where } } - async fn topic(&self, topic: TopicProduceData) -> TopicProduceResponse { + async fn topic( + &mut self, + transaction_id: Option<&str>, + topic: TopicProduceData, + ) -> TopicProduceResponse { let mut partitions = vec![]; if let Some(partition_data) = topic.partition_data { for partition in partition_data { - partitions.push(self.partition(&topic.name, partition).await) + partitions.push(self.partition(transaction_id, &topic.name, partition).await) } } @@ -100,12 +118,12 @@ where } pub async fn response( - &self, - _transactional_id: Option, + &mut self, + transaction_id: Option, _acks: i16, _timeout_ms: i32, topic_data: Option>, - ) -> Result { + ) -> Result { let mut responses = Vec::with_capacity(topic_data.as_ref().map_or(0, |topic_data| topic_data.len())); @@ -113,14 +131,485 @@ where for topic in topics { debug!(?topic); - responses.push(self.topic(topic).await) + responses.push(self.topic(transaction_id.as_deref(), topic).await) } } - Ok(Body::ProduceResponse { + Ok(ProduceResponse { responses: Some(responses), throttle_time_ms: Some(0), node_endpoints: None, }) } } + +#[cfg(test)] +mod tests { + use super::*; + use crate::{broker::init_producer_id::InitProducerIdRequest, Error}; + use bytes::Bytes; + use object_store::memory::InMemory; + use tansu_kafka_sans_io::{ + record::{ + deflated::{self, Frame}, + inflated, Record, + }, + ErrorCode, + }; + use tansu_storage::dynostore::DynoStore; + use tracing::subscriber::DefaultGuard; + + #[cfg(miri)] + fn init_tracing() -> Result<()> { + Ok(()) + } + + #[cfg(not(miri))] + fn init_tracing() -> Result { + use std::{fs::File, sync::Arc, thread}; + + use tracing::Level; + use tracing_subscriber::fmt::format::FmtSpan; + + Ok(tracing::subscriber::set_default( + tracing_subscriber::fmt() + .with_level(true) + .with_line_number(true) + .with_thread_names(false) + .with_max_level(Level::DEBUG) + .with_span_events(FmtSpan::ACTIVE) + .with_writer( + thread::current() + .name() + .ok_or(Error::Custom(String::from("unnamed thread"))) + .and_then(|name| { + File::create(format!("../logs/{}/{name}.log", env!("CARGO_PKG_NAME"))) + .map_err(Into::into) + }) + .map(Arc::new)?, + ) + .finish(), + )) + } + + fn topic_data( + topic: &str, + index: i32, + builder: inflated::Builder, + ) -> Result>> { + builder + .build() + .and_then(deflated::Batch::try_from) + .map(|deflated| { + let partition_data = PartitionProduceData { + index, + records: Some(Frame { + batches: vec![deflated], + }), + }; + + Some(vec![TopicProduceData { + name: topic.into(), + partition_data: Some(vec![partition_data]), + }]) + }) + .map_err(Into::into) + } + + #[tokio::test] + async fn non_txn_idempotent_unknown_producer_id() -> Result<()> { + let _guard = init_tracing()?; + + let cluster = "abc"; + let node = 12321; + let topic = "pqr"; + let index = 0; + + let storage = DynoStore::new(cluster, node, InMemory::new()); + + let transactional_id = None; + let acks = 0; + let timeout_ms = 0; + + assert_eq!( + ProduceResponse { + responses: Some(vec![TopicProduceResponse { + name: topic.into(), + partition_responses: Some(vec![PartitionProduceResponse { + index, + error_code: ErrorCode::UnknownProducerId.into(), + base_offset: -1, + log_append_time_ms: Some(-1), + log_start_offset: Some(0), + record_errors: Some(vec![]), + error_message: None, + current_leader: None, + }],), + }]), + throttle_time_ms: Some(0), + node_endpoints: None + }, + ProduceRequest::with_storage(storage) + .response( + transactional_id, + acks, + timeout_ms, + topic_data( + topic, + index, + inflated::Batch::builder() + .record(Record::builder().value(Bytes::from_static(b"lorem").into())) + .producer_id(54345) + )? + ) + .await? + ); + + Ok(()) + } + + #[tokio::test] + async fn non_txn_idempotent() -> Result<()> { + let _guard = init_tracing()?; + + let cluster = "abc"; + let node = 12321; + let topic = "pqr"; + let index = 0; + + let storage = DynoStore::new(cluster, node, InMemory::new()); + + let producer = InitProducerIdRequest::with_storage(storage.clone()) + .response(None, 0, Some(-1), Some(-1)) + .await?; + + let mut request = ProduceRequest::with_storage(storage.clone()); + + let transactional_id = None; + let acks = 0; + let timeout_ms = 0; + + assert_eq!( + ProduceResponse { + responses: Some(vec![TopicProduceResponse { + name: topic.into(), + partition_responses: Some(vec![PartitionProduceResponse { + index, + error_code: ErrorCode::None.into(), + base_offset: 0, + log_append_time_ms: Some(-1), + log_start_offset: Some(0), + record_errors: Some(vec![]), + error_message: None, + current_leader: None, + }],), + }]), + throttle_time_ms: Some(0), + node_endpoints: None + }, + request + .response( + transactional_id.clone(), + acks, + timeout_ms, + topic_data( + topic, + index, + inflated::Batch::builder() + .record( + Record::builder().value( + Bytes::from_static(b"Lorem ipsum dolor sit amet").into() + ) + ) + .producer_id(producer.id) + )? + ) + .await? + ); + + assert_eq!( + ProduceResponse { + responses: Some(vec![TopicProduceResponse { + name: topic.into(), + partition_responses: Some(vec![PartitionProduceResponse { + index, + error_code: ErrorCode::None.into(), + base_offset: 1, + log_append_time_ms: Some(-1), + log_start_offset: Some(0), + record_errors: Some(vec![]), + error_message: None, + current_leader: None, + }],), + }]), + throttle_time_ms: Some(0), + node_endpoints: None + }, + request + .response( + transactional_id.clone(), + acks, + timeout_ms, + topic_data( + topic, + index, + inflated::Batch::builder() + .record( + Record::builder().value( + Bytes::from_static(b"consectetur adipiscing elit").into() + ) + ) + .record( + Record::builder() + .value(Bytes::from_static(b"sed do eiusmod tempor").into()) + ) + .base_sequence(1) + .last_offset_delta(1) + .producer_id(producer.id) + )? + ) + .await? + ); + + assert_eq!( + ProduceResponse { + responses: Some(vec![TopicProduceResponse { + name: topic.into(), + partition_responses: Some(vec![PartitionProduceResponse { + index, + error_code: ErrorCode::None.into(), + base_offset: 3, + log_append_time_ms: Some(-1), + log_start_offset: Some(0), + record_errors: Some(vec![]), + error_message: None, + current_leader: None, + }],), + }]), + throttle_time_ms: Some(0), + node_endpoints: None + }, + request + .response( + transactional_id.clone(), + acks, + timeout_ms, + topic_data( + topic, + index, + inflated::Batch::builder() + .record( + Record::builder() + .value(Bytes::from_static(b"incididunt ut labore").into()) + ) + .base_sequence(3) + .producer_id(producer.id) + )? + ) + .await? + ); + + Ok(()) + } + + #[tokio::test] + async fn non_txn_idempotent_duplicate_sequence() -> Result<()> { + let _guard = init_tracing()?; + + let cluster = "abc"; + let node = 12321; + let topic = "pqr"; + let index = 0; + + let storage = DynoStore::new(cluster, node, InMemory::new()); + + let producer = InitProducerIdRequest::with_storage(storage.clone()) + .response(None, 0, Some(-1), Some(-1)) + .await?; + + let mut request = ProduceRequest::with_storage(storage.clone()); + + let transactional_id = None; + let acks = 0; + let timeout_ms = 0; + + assert_eq!( + ProduceResponse { + responses: Some(vec![TopicProduceResponse { + name: topic.into(), + partition_responses: Some(vec![PartitionProduceResponse { + index, + error_code: ErrorCode::None.into(), + base_offset: 0, + log_append_time_ms: Some(-1), + log_start_offset: Some(0), + record_errors: Some(vec![]), + error_message: None, + current_leader: None, + }],), + }]), + throttle_time_ms: Some(0), + node_endpoints: None + }, + request + .response( + transactional_id.clone(), + acks, + timeout_ms, + topic_data( + topic, + index, + inflated::Batch::builder() + .record( + Record::builder().value( + Bytes::from_static(b"Lorem ipsum dolor sit amet").into() + ) + ) + .producer_id(producer.id) + )? + ) + .await? + ); + + assert_eq!( + ProduceResponse { + responses: Some(vec![TopicProduceResponse { + name: topic.into(), + partition_responses: Some(vec![PartitionProduceResponse { + index, + error_code: ErrorCode::DuplicateSequenceNumber.into(), + base_offset: -1, + log_append_time_ms: Some(-1), + log_start_offset: Some(0), + record_errors: Some(vec![]), + error_message: None, + current_leader: None, + }],), + }]), + throttle_time_ms: Some(0), + node_endpoints: None + }, + request + .response( + transactional_id, + acks, + timeout_ms, + topic_data( + topic, + index, + inflated::Batch::builder() + .record( + Record::builder().value( + Bytes::from_static(b"Lorem ipsum dolor sit amet").into() + ) + ) + .producer_id(producer.id) + )? + ) + .await? + ); + + Ok(()) + } + + #[tokio::test] + async fn non_txn_idempotent_sequence_out_of_order() -> Result<()> { + let _guard = init_tracing()?; + + let cluster = "abc"; + let node = 12321; + let topic = "pqr"; + let index = 0; + + let storage = DynoStore::new(cluster, node, InMemory::new()); + + let producer = InitProducerIdRequest::with_storage(storage.clone()) + .response(None, 0, Some(-1), Some(-1)) + .await?; + + let mut request = ProduceRequest::with_storage(storage.clone()); + + let transactional_id = None; + let acks = 0; + let timeout_ms = 0; + + assert_eq!( + ProduceResponse { + responses: Some(vec![TopicProduceResponse { + name: topic.into(), + partition_responses: Some(vec![PartitionProduceResponse { + index, + error_code: ErrorCode::None.into(), + base_offset: 0, + log_append_time_ms: Some(-1), + log_start_offset: Some(0), + record_errors: Some(vec![]), + error_message: None, + current_leader: None, + }],), + }]), + throttle_time_ms: Some(0), + node_endpoints: None + }, + request + .response( + transactional_id.clone(), + acks, + timeout_ms, + topic_data( + topic, + index, + inflated::Batch::builder() + .record( + Record::builder().value( + Bytes::from_static(b"Lorem ipsum dolor sit amet").into() + ) + ) + .producer_id(producer.id) + )? + ) + .await? + ); + + assert_eq!( + ProduceResponse { + responses: Some(vec![TopicProduceResponse { + name: topic.into(), + partition_responses: Some(vec![PartitionProduceResponse { + index, + error_code: ErrorCode::OutOfOrderSequenceNumber.into(), + base_offset: -1, + log_append_time_ms: Some(-1), + log_start_offset: Some(0), + record_errors: Some(vec![]), + error_message: None, + current_leader: None, + }],), + }]), + throttle_time_ms: Some(0), + node_endpoints: None + }, + request + .response( + transactional_id, + acks, + timeout_ms, + topic_data( + topic, + index, + inflated::Batch::builder() + .record( + Record::builder().value( + Bytes::from_static(b"Lorem ipsum dolor sit amet").into() + ) + ) + .base_sequence(2) + .producer_id(producer.id) + )? + ) + .await? + ); + + Ok(()) + } +} diff --git a/tansu-server/src/broker/txn.rs b/tansu-server/src/broker/txn.rs new file mode 100644 index 0000000..fbb56a9 --- /dev/null +++ b/tansu-server/src/broker/txn.rs @@ -0,0 +1,18 @@ +// Copyright ⓒ 2024 Peter Morgan +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +pub mod add_offsets; +pub mod add_partitions; +pub mod offset_commit; diff --git a/tansu-server/src/broker/txn/add_offsets.rs b/tansu-server/src/broker/txn/add_offsets.rs new file mode 100644 index 0000000..9a410ef --- /dev/null +++ b/tansu-server/src/broker/txn/add_offsets.rs @@ -0,0 +1,53 @@ +// Copyright ⓒ 2024 Peter Morgan +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use tansu_kafka_sans_io::Body; +use tansu_storage::Storage; +use tracing::debug; + +use crate::Result; + +#[derive(Clone, Copy, Debug, Default, Eq, Hash, Ord, PartialEq, PartialOrd)] +pub struct AddOffsets { + storage: S, +} + +impl AddOffsets +where + S: Storage, +{ + pub fn with_storage(storage: S) -> Self { + Self { storage } + } + + pub async fn response( + &mut self, + transaction_id: &str, + producer_id: i64, + producer_epoch: i16, + group_id: &str, + ) -> Result { + debug!(?transaction_id, ?producer_id, ?producer_epoch, ?group_id); + + self.storage + .txn_add_offsets(transaction_id, producer_id, producer_epoch, group_id) + .await + .map_err(Into::into) + .map(|error_code| Body::AddOffsetsToTxnResponse { + throttle_time_ms: 0, + error_code: error_code.into(), + }) + } +} diff --git a/tansu-server/src/broker/txn/add_partitions.rs b/tansu-server/src/broker/txn/add_partitions.rs new file mode 100644 index 0000000..16d0acb --- /dev/null +++ b/tansu-server/src/broker/txn/add_partitions.rs @@ -0,0 +1,57 @@ +// Copyright ⓒ 2024 Peter Morgan +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use tansu_kafka_sans_io::{Body, ErrorCode}; +use tansu_storage::{Storage, TxnAddPartitionsRequest, TxnAddPartitionsResponse}; +use tracing::debug; + +use crate::Result; + +#[derive(Clone, Copy, Debug, Default, Eq, Hash, Ord, PartialEq, PartialOrd)] +pub struct AddPartitions { + storage: S, +} + +impl AddPartitions +where + S: Storage, +{ + pub fn with_storage(storage: S) -> Self { + Self { storage } + } + + pub async fn response(&mut self, partitions: TxnAddPartitionsRequest) -> Result { + debug!(?partitions); + match self.storage.txn_add_partitions(partitions).await? { + TxnAddPartitionsResponse::VersionZeroToThree(results_by_topic_v_3_and_below) => { + Ok(Body::AddPartitionsToTxnResponse { + throttle_time_ms: 0, + error_code: Some(ErrorCode::None.into()), + results_by_transaction: Some([].into()), + results_by_topic_v_3_and_below: Some(results_by_topic_v_3_and_below), + }) + } + + TxnAddPartitionsResponse::VersionFourPlus(results_by_transaction) => { + Ok(Body::AddPartitionsToTxnResponse { + throttle_time_ms: 0, + error_code: Some(ErrorCode::None.into()), + results_by_transaction: Some(results_by_transaction), + results_by_topic_v_3_and_below: Some([].into()), + }) + } + } + } +} diff --git a/tansu-server/src/broker/txn/offset_commit.rs b/tansu-server/src/broker/txn/offset_commit.rs new file mode 100644 index 0000000..2c0b532 --- /dev/null +++ b/tansu-server/src/broker/txn/offset_commit.rs @@ -0,0 +1,65 @@ +// Copyright ⓒ 2024 Peter Morgan +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use tansu_kafka_sans_io::{txn_offset_commit_request::TxnOffsetCommitRequestTopic, Body}; +use tansu_storage::{Storage, TxnOffsetCommitRequest}; + +use crate::Result; + +#[derive(Clone, Copy, Debug, Default, Eq, Hash, Ord, PartialEq, PartialOrd)] +pub struct OffsetCommit { + storage: S, +} + +impl OffsetCommit +where + S: Storage, +{ + pub fn with_storage(storage: S) -> Self { + Self { storage } + } + + #[allow(clippy::too_many_arguments)] + pub async fn response( + &mut self, + transactional_id: &str, + group_id: &str, + producer_id: i64, + producer_epoch: i16, + generation_id: Option, + member_id: Option, + group_instance_id: Option, + topics: Option>, + ) -> Result { + let responses = self + .storage + .txn_offset_commit(TxnOffsetCommitRequest { + transaction_id: transactional_id.to_owned(), + group_id: group_id.to_owned(), + producer_id, + producer_epoch, + generation_id, + member_id, + group_instance_id, + topics: topics.unwrap_or_default(), + }) + .await?; + + Ok(Body::TxnOffsetCommitResponse { + throttle_time_ms: 0, + topics: Some(responses), + }) + } +} diff --git a/tansu-server/src/coordinator/group/administrator.rs b/tansu-server/src/coordinator/group/administrator.rs index 617cf08..48a630a 100644 --- a/tansu-server/src/coordinator/group/administrator.rs +++ b/tansu-server/src/coordinator/group/administrator.rs @@ -41,7 +41,7 @@ use tansu_storage::{ Version, }; use tokio::time::{sleep, Duration}; -use tracing::{debug, info}; +use tracing::{debug, error, info}; use uuid::Uuid; use crate::{Error, Result}; @@ -1488,6 +1488,7 @@ where topics: Some(topics), } }) + .inspect_err(|err| error!(?err)) .map_err(Into::into) } else { Ok(Body::OffsetCommitResponse { diff --git a/tansu-server/src/main.rs b/tansu-server/src/main.rs index 47d4f61..4abb8b9 100644 --- a/tansu-server/src/main.rs +++ b/tansu-server/src/main.rs @@ -105,8 +105,8 @@ async fn main() -> Result<()> { tracing_subscriber::fmt::layer() .with_level(true) .with_line_number(true) - .with_thread_ids(true) - .with_span_events(FmtSpan::ACTIVE), + .with_thread_ids(false) + .with_span_events(FmtSpan::NONE), ) .with(EnvFilter::from_default_env()) .init(); diff --git a/tansu-server/tests/common/mod.rs b/tansu-server/tests/common/mod.rs new file mode 100644 index 0000000..7fae1ae --- /dev/null +++ b/tansu-server/tests/common/mod.rs @@ -0,0 +1,250 @@ +// Copyright ⓒ 2024 Peter Morgan +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use bytes::Bytes; +use rand::{distributions::Alphanumeric, prelude::*, thread_rng}; +use tansu_kafka_sans_io::{ + join_group_request::JoinGroupRequestProtocol, join_group_response::JoinGroupResponseMember, + offset_fetch_request::OffsetFetchRequestTopic, offset_fetch_response::OffsetFetchResponseTopic, + sync_group_request::SyncGroupRequestAssignment, Body, ErrorCode, +}; +use tansu_server::{ + coordinator::group::{administrator::Controller, Coordinator}, + Error, Result, +}; +use tansu_storage::{pg::Postgres, StorageContainer}; + +pub(crate) fn storage_container(cluster: impl Into, node: i32) -> Result { + Postgres::builder("postgres://postgres:postgres@localhost") + .map(|builder| builder.cluster(cluster)) + .map(|builder| builder.node(node)) + .map(|builder| builder.build()) + .map(StorageContainer::Postgres) + .map_err(Into::into) +} + +pub(crate) fn alphanumeric_string(length: usize) -> String { + thread_rng() + .sample_iter(&Alphanumeric) + .take(length) + .map(char::from) + .collect() +} + +#[derive(Clone, Debug, Default, Eq, Hash, Ord, PartialEq, PartialOrd)] +pub(crate) struct JoinGroupResponse { + pub error_code: ErrorCode, + pub generation_id: i32, + pub protocol_type: String, + pub protocol_name: String, + pub leader: String, + pub skip_assignment: bool, + pub member_id: String, + pub members: Vec, +} + +impl TryFrom for JoinGroupResponse { + type Error = Error; + + fn try_from(value: Body) -> Result { + match value { + Body::JoinGroupResponse { + throttle_time_ms: Some(0), + error_code, + generation_id, + protocol_type: Some(protocol_type), + protocol_name: Some(protocol_name), + leader, + skip_assignment: Some(skip_assignment), + members: Some(members), + member_id, + } => ErrorCode::try_from(error_code) + .map(|error_code| JoinGroupResponse { + error_code, + generation_id, + protocol_type, + protocol_name, + leader, + skip_assignment, + member_id, + members, + }) + .map_err(Into::into), + otherwise => panic!("{otherwise:?}"), + } + } +} + +#[allow(clippy::too_many_arguments)] +pub(crate) async fn join_group( + controller: &mut Controller, + client_id: Option<&str>, + group_id: &str, + session_timeout_ms: i32, + rebalance_timeout_ms: Option, + member_id: &str, + group_instance_id: Option<&str>, + protocol_type: &str, + protocols: Option<&[JoinGroupRequestProtocol]>, + reason: Option<&str>, +) -> Result { + controller + .join( + client_id, + group_id, + session_timeout_ms, + rebalance_timeout_ms, + member_id, + group_instance_id, + protocol_type, + protocols, + reason, + ) + .await + .map_err(Into::into) + .and_then(TryInto::try_into) +} + +#[derive(Clone, Debug, Default, Eq, Hash, Ord, PartialEq, PartialOrd)] +pub(crate) struct SyncGroupResponse { + pub error_code: ErrorCode, + pub protocol_type: String, + pub protocol_name: String, + pub assignment: Bytes, +} + +impl TryFrom for SyncGroupResponse { + type Error = Error; + + fn try_from(value: Body) -> std::result::Result { + match value { + Body::SyncGroupResponse { + throttle_time_ms: Some(0), + error_code, + protocol_type: Some(protocol_type), + protocol_name: Some(protocol_name), + assignment, + } => ErrorCode::try_from(error_code) + .map(|error_code| SyncGroupResponse { + error_code, + protocol_type, + protocol_name, + assignment, + }) + .map_err(Into::into), + + otherwise => panic!("{otherwise:?}"), + } + } +} + +#[allow(clippy::too_many_arguments)] +pub(crate) async fn sync_group( + controller: &mut Controller, + group_id: &str, + generation_id: i32, + member_id: &str, + group_instance_id: Option<&str>, + protocol_type: &str, + protocol_name: &str, + assignments: &[SyncGroupRequestAssignment], +) -> Result { + controller + .sync( + group_id, + generation_id, + member_id, + group_instance_id, + Some(protocol_type), + Some(protocol_name), + Some(assignments), + ) + .await + .map_err(Into::into) + .and_then(TryInto::try_into) +} + +#[derive(Clone, Debug, Default, Eq, Hash, Ord, PartialEq, PartialOrd)] +pub(crate) struct HeartbeatResponse { + pub error_code: ErrorCode, +} + +impl TryFrom for HeartbeatResponse { + type Error = Error; + + fn try_from(value: Body) -> std::result::Result { + match value { + Body::HeartbeatResponse { + throttle_time_ms: Some(0), + error_code, + } => ErrorCode::try_from(error_code) + .map(|error_code| HeartbeatResponse { error_code }) + .map_err(Into::into), + + otherwise => panic!("{otherwise:?}"), + } + } +} + +pub(crate) async fn heartbeat( + controller: &mut Controller, + group_id: &str, + generation_id: i32, + member_id: &str, + group_instance_id: Option<&str>, +) -> Result { + controller + .heartbeat(group_id, generation_id, member_id, group_instance_id) + .await + .map_err(Into::into) + .and_then(TryInto::try_into) +} + +#[derive(Clone, Debug, Default, Eq, Hash, Ord, PartialEq, PartialOrd)] +pub(crate) struct OffsetFetchResponse { + pub topics: Vec, + pub error_code: ErrorCode, +} + +impl TryFrom for OffsetFetchResponse { + type Error = Error; + + fn try_from(value: Body) -> Result { + match value { + Body::OffsetFetchResponse { + throttle_time_ms: Some(0), + topics: Some(topics), + error_code: Some(error_code), + groups: None, + } => ErrorCode::try_from(error_code) + .map(|error_code| OffsetFetchResponse { topics, error_code }) + .map_err(Into::into), + + otherwise => panic!("{otherwise:?}"), + } + } +} + +pub(crate) async fn offset_fetch( + controller: &mut Controller, + group_id: &str, + topics: &[OffsetFetchRequestTopic], +) -> Result { + controller + .offset_fetch(Some(group_id), Some(topics), None, Some(false)) + .await + .map_err(Into::into) + .and_then(TryInto::try_into) +} diff --git a/tansu-server/tests/pg_txn.rs b/tansu-server/tests/pg_txn.rs new file mode 100644 index 0000000..6d9644b --- /dev/null +++ b/tansu-server/tests/pg_txn.rs @@ -0,0 +1,530 @@ +// Copyright ⓒ 2024 Peter Morgan +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use bytes::Bytes; +use rand::{prelude::*, thread_rng}; +use tansu_kafka_sans_io::{ + add_partitions_to_txn_request::AddPartitionsToTxnTopic, + broker_registration_request::Listener, + create_topics_request::CreatableTopic, + join_group_request::JoinGroupRequestProtocol, + join_group_response::JoinGroupResponseMember, + offset_fetch_request::OffsetFetchRequestTopic, + offset_fetch_response::{OffsetFetchResponsePartition, OffsetFetchResponseTopic}, + record::{inflated, Record}, + sync_group_request::SyncGroupRequestAssignment, + txn_offset_commit_request::{TxnOffsetCommitRequestPartition, TxnOffsetCommitRequestTopic}, + txn_offset_commit_response::{TxnOffsetCommitResponsePartition, TxnOffsetCommitResponseTopic}, + BatchAttribute, ErrorCode, IsolationLevel, +}; +use tansu_server::{coordinator::group::administrator::Controller, Error, Result}; +use tansu_storage::{ + BrokerRegistationRequest, ListOffsetRequest, Storage, Topition, TxnAddPartitionsRequest, + TxnOffsetCommitRequest, +}; +use tracing::{debug, subscriber::DefaultGuard}; +use uuid::Uuid; + +mod common; + +fn init_tracing() -> Result { + use std::{fs::File, sync::Arc, thread}; + + use tracing::Level; + + Ok(tracing::subscriber::set_default( + tracing_subscriber::fmt() + .with_level(true) + .with_line_number(true) + .with_max_level(Level::DEBUG) + .with_writer( + thread::current() + .name() + .ok_or(Error::Message(String::from("unnamed thread"))) + .and_then(|name| { + File::create(format!("../logs/{}/{name}.log", env!("CARGO_PKG_NAME"))) + .map_err(Into::into) + }) + .map(Arc::new)?, + ) + .finish(), + )) +} + +#[tokio::test] +async fn simple_txn_commit() -> Result<()> { + let _guard = init_tracing()?; + + let mut rng = thread_rng(); + + let cluster_id = Uuid::now_v7(); + let broker_id = rng.gen_range(0..i32::MAX); + let incarnation_id = Uuid::now_v7(); + + debug!(?cluster_id, ?broker_id, ?incarnation_id); + + let mut sc = common::storage_container(cluster_id, broker_id)?; + + let port = rng.gen_range(1024..u16::MAX); + let security_protocol = rng.gen_range(0..i16::MAX); + + let broker_registration = BrokerRegistationRequest { + broker_id, + cluster_id: cluster_id.into(), + incarnation_id, + listeners: vec![Listener { + name: "broker".into(), + host: "test.local".into(), + port, + security_protocol, + }], + features: vec![], + rack: None, + }; + + sc.register_broker(broker_registration).await?; + + let input_topic_name: String = common::alphanumeric_string(15); + debug!(?input_topic_name); + + let num_partitions = 6; + let replication_factor = 0; + let assignments = Some([].into()); + let configs = Some([].into()); + + // create input topic + // + let input_topic_id = sc + .create_topic( + CreatableTopic { + name: input_topic_name.clone(), + num_partitions, + replication_factor, + assignments: assignments.clone(), + configs: configs.clone(), + }, + false, + ) + .await?; + debug!(?input_topic_id); + + let input_partition_index = rng.gen_range(0..num_partitions); + let input_topition = Topition::new(input_topic_name.clone(), input_partition_index); + let records = 6; + + // populate the input topic with some records + // + for n in 0..records { + let value = format!("Lorem ipsum dolor sit amet: {n}"); + + let batch = inflated::Batch::builder() + .record(Record::builder().value(Bytes::copy_from_slice(value.as_bytes()).into())) + .build() + .and_then(TryInto::try_into) + .inspect(|deflated| debug!(?deflated))?; + + _ = sc + .produce(None, &input_topition, batch) + .await + .inspect(|offset| debug!(?offset))?; + } + + let output_topic_name: String = common::alphanumeric_string(15); + debug!(?output_topic_name); + + // create output topic + // + let output_topic_id = sc + .create_topic( + CreatableTopic { + name: output_topic_name.clone(), + num_partitions, + replication_factor, + assignments, + configs, + }, + false, + ) + .await?; + debug!(?output_topic_id); + + // consumer group controller + // + let mut controller = Controller::with_storage(sc.clone())?; + + let session_timeout_ms = 45_000; + let rebalance_timeout_ms = Some(300_000); + let group_instance_id = None; + let reason = None; + + let group_id: String = common::alphanumeric_string(15); + debug!(?group_id); + + const CLIENT_ID: &str = "console-consumer"; + const RANGE: &str = "range"; + const COOPERATIVE_STICKY: &str = "cooperative-sticky"; + + const PROTOCOL_TYPE: &str = "consumer"; + + let first_member_range_meta = Bytes::from_static(b"first_member_range_meta_01"); + let first_member_sticky_meta = Bytes::from_static(b"first_member_sticky_meta_01"); + + let protocols = [ + JoinGroupRequestProtocol { + name: RANGE.into(), + metadata: first_member_range_meta.clone(), + }, + JoinGroupRequestProtocol { + name: COOPERATIVE_STICKY.into(), + metadata: first_member_sticky_meta, + }, + ]; + + // join group without a member id + // + let member_id_required = common::join_group( + &mut controller, + Some(CLIENT_ID), + group_id.as_str(), + session_timeout_ms, + rebalance_timeout_ms, + "", + group_instance_id, + PROTOCOL_TYPE, + Some(&protocols[..]), + reason, + ) + .await?; + + // join rejected as member id is required + // + assert_eq!(ErrorCode::MemberIdRequired, member_id_required.error_code); + assert_eq!("consumer", member_id_required.protocol_type); + assert_eq!("", member_id_required.protocol_name); + assert!(member_id_required.leader.is_empty()); + assert!(member_id_required.member_id.starts_with(CLIENT_ID)); + assert_eq!(0, member_id_required.members.len()); + + // join with the supplied member id + // + let join_response = common::join_group( + &mut controller, + Some(CLIENT_ID), + group_id.as_str(), + session_timeout_ms, + rebalance_timeout_ms, + member_id_required.member_id.as_str(), + group_instance_id, + PROTOCOL_TYPE, + Some(&protocols[..]), + reason, + ) + .await?; + + // join accepted as leader + // + assert_eq!(ErrorCode::None, join_response.error_code); + assert_eq!(0, join_response.generation_id); + assert_eq!(PROTOCOL_TYPE, join_response.protocol_type); + assert_eq!(RANGE, join_response.protocol_name); + assert_eq!(member_id_required.member_id.as_str(), join_response.leader); + assert_eq!( + vec![JoinGroupResponseMember { + member_id: member_id_required.member_id.clone(), + group_instance_id: None, + metadata: first_member_range_meta.clone(), + }], + join_response.members + ); + + let member_id = member_id_required.member_id.clone(); + debug!(?member_id); + + let first_member_assignment_01 = Bytes::from_static(b"assignment_01"); + + let assignments = [SyncGroupRequestAssignment { + member_id: member_id.clone(), + assignment: first_member_assignment_01.clone(), + }]; + + // sync to form the group + // + let sync_response = common::sync_group( + &mut controller, + group_id.as_str(), + join_response.generation_id, + member_id.as_str(), + group_instance_id, + PROTOCOL_TYPE, + RANGE, + &assignments, + ) + .await?; + assert_eq!(ErrorCode::None, sync_response.error_code); + assert_eq!(PROTOCOL_TYPE, sync_response.protocol_type); + assert_eq!(RANGE, sync_response.protocol_name); + assert_eq!(first_member_assignment_01, sync_response.assignment); + + // heartbeat establishing leadership of current generation + // + assert_eq!( + common::HeartbeatResponse { + error_code: ErrorCode::None, + }, + common::heartbeat( + &mut controller, + group_id.as_str(), + join_response.generation_id, + &member_id, + group_instance_id + ) + .await? + ); + + let transaction_id: String = common::alphanumeric_string(10); + debug!(?transaction_id); + + let transaction_timeout_ms = 10_000; + + // initialise producer with a transaction + // + let producer = sc + .init_producer( + Some(transaction_id.as_str()), + transaction_timeout_ms, + Some(-1), + Some(-1), + ) + .await?; + debug!(?producer); + + // add all output topic partitions to the transaction + // + let txn_add_partitions = TxnAddPartitionsRequest::VersionZeroToThree { + transaction_id: transaction_id.clone(), + producer_id: producer.id, + producer_epoch: producer.epoch, + topics: vec![AddPartitionsToTxnTopic { + name: output_topic_name.clone(), + partitions: Some((0..num_partitions).collect()), + }], + }; + + let txn_add_partitions_response = sc.txn_add_partitions(txn_add_partitions).await?; + debug!(?txn_add_partitions_response); + assert_eq!(1, txn_add_partitions_response.zero_to_three().len()); + assert_eq!( + output_topic_name, + txn_add_partitions_response.zero_to_three()[0].name + ); + + // adding offsets + // + assert_eq!( + ErrorCode::None, + sc.txn_add_offsets( + transaction_id.as_str(), + producer.id, + producer.epoch, + group_id.as_str(), + ) + .await? + ); + + const COMMITTED_OFFSET: i64 = 6543456; + + // commit an offset for the input topic to the consumer group + // + assert_eq!( + vec![TxnOffsetCommitResponseTopic { + name: input_topic_name.clone(), + partitions: Some( + [TxnOffsetCommitResponsePartition { + partition_index: input_partition_index, + error_code: ErrorCode::None.into(), + }] + .into(), + ), + }], + sc.txn_offset_commit(TxnOffsetCommitRequest { + transaction_id: transaction_id.clone(), + group_id: group_id.clone(), + producer_id: producer.id, + producer_epoch: producer.epoch, + generation_id: Some(join_response.generation_id), + member_id: Some(member_id), + group_instance_id: group_instance_id + .map(|group_instance_id| group_instance_id.to_owned()), + topics: [TxnOffsetCommitRequestTopic { + name: input_topic_name.clone(), + partitions: Some( + [TxnOffsetCommitRequestPartition { + partition_index: input_partition_index, + committed_offset: COMMITTED_OFFSET, + committed_leader_epoch: Some(-1), + committed_metadata: None, + }] + .into(), + ), + }] + .into(), + }) + .await? + ); + + // verify that the committed offset is not visible as the transaction remains in progress + // + assert_eq!( + common::OffsetFetchResponse { + topics: [OffsetFetchResponseTopic { + name: input_topic_name.clone(), + partitions: Some( + [OffsetFetchResponsePartition { + partition_index: input_partition_index, + committed_offset: -1, + committed_leader_epoch: None, + metadata: None, + error_code: 0 + }] + .into() + ) + }] + .into(), + error_code: ErrorCode::None + }, + common::offset_fetch( + &mut controller, + group_id.as_str(), + &[OffsetFetchRequestTopic { + name: input_topic_name.clone(), + partition_indexes: Some([input_partition_index].into()), + }], + ) + .await? + ); + + let output_partition_index = rng.gen_range(0..num_partitions); + let output_topition = Topition::new(output_topic_name.clone(), output_partition_index); + + // verify the high watermark of the output topic is 0 + // prior to producing records + // + let list_offsets_before_produce = sc + .list_offsets( + IsolationLevel::ReadUncommitted, + &[(output_topition.clone(), ListOffsetRequest::Latest)], + ) + .await?; + assert_eq!(1, list_offsets_before_produce.len()); + assert_eq!(output_topic_name, list_offsets_before_produce[0].0.topic()); + assert_eq!( + output_partition_index, + list_offsets_before_produce[0].0.partition() + ); + assert_eq!(ErrorCode::None, list_offsets_before_produce[0].1.error_code); + assert_eq!(Some(0), list_offsets_before_produce[0].1.offset); + + // produce records to output topic while in a transaction + // + for n in 0..records { + let value = format!("Consectetur adipiscing elit: {n}"); + let batch = inflated::Batch::builder() + .record(Record::builder().value(Bytes::copy_from_slice(value.as_bytes()).into())) + .attributes(BatchAttribute::default().transaction(true).into()) + .producer_id(producer.id) + .producer_epoch(producer.epoch) + .build() + .and_then(TryInto::try_into) + .inspect(|deflated| debug!(?deflated))?; + + _ = sc + .produce(Some(transaction_id.as_str()), &output_topition, batch) + .await + .inspect(|offset| debug!(?offset))?; + } + + // read uncommitted latest offset has updated + // + let list_offsets_after_produce = sc + .list_offsets( + IsolationLevel::ReadUncommitted, + &[(output_topition.clone(), ListOffsetRequest::Latest)], + ) + .await?; + assert_eq!(1, list_offsets_after_produce.len()); + assert_eq!(output_topic_name, list_offsets_after_produce[0].0.topic()); + assert_eq!( + output_partition_index, + list_offsets_after_produce[0].0.partition() + ); + assert_eq!(ErrorCode::None, list_offsets_after_produce[0].1.error_code); + assert_eq!(Some(records - 1), list_offsets_after_produce[0].1.offset); + + // read committed offset is at 0 + // + let list_offsets_after_produce = sc + .list_offsets( + IsolationLevel::ReadCommitted, + &[(output_topition.clone(), ListOffsetRequest::Latest)], + ) + .await?; + assert_eq!(1, list_offsets_after_produce.len()); + assert_eq!(output_topic_name, list_offsets_after_produce[0].0.topic()); + assert_eq!( + output_partition_index, + list_offsets_after_produce[0].0.partition() + ); + assert_eq!(ErrorCode::None, list_offsets_after_produce[0].1.error_code); + assert_eq!(Some(0), list_offsets_after_produce[0].1.offset); + + // commit the transaction + // + assert_eq!( + ErrorCode::None, + sc.txn_end(transaction_id.as_str(), producer.id, producer.epoch, true) + .await? + ); + + assert_eq!( + common::OffsetFetchResponse { + topics: [OffsetFetchResponseTopic { + name: input_topic_name.clone(), + partitions: Some( + [OffsetFetchResponsePartition { + partition_index: input_partition_index, + committed_offset: COMMITTED_OFFSET, + committed_leader_epoch: None, + metadata: None, + error_code: 0 + }] + .into() + ) + }] + .into(), + error_code: ErrorCode::None + }, + common::offset_fetch( + &mut controller, + group_id.as_str(), + &[OffsetFetchRequestTopic { + name: input_topic_name.clone(), + partition_indexes: Some([input_partition_index].into()), + }], + ) + .await? + ); + + Ok(()) +} diff --git a/tansu-storage/src/dynostore.rs b/tansu-storage/src/dynostore.rs index 62e1a65..5981c22 100644 --- a/tansu-storage/src/dynostore.rs +++ b/tansu-storage/src/dynostore.rs @@ -13,9 +13,11 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . +use std::cmp::Ordering; use std::collections::BTreeSet; use std::io::BufReader; use std::sync::Arc; +use std::time::SystemTime; use std::{collections::BTreeMap, fmt::Debug, io::Cursor, str::FromStr, time::Duration}; use async_trait::async_trait; @@ -26,11 +28,12 @@ use object_store::{ path::Path, Attribute, AttributeValue, Attributes, ObjectStore, PutMode, PutOptions, PutPayload, TagSet, UpdateVersion, }; -use object_store::{DynObjectStore, PutResult}; +use object_store::{DynObjectStore, GetOptions, PutResult}; use rand::{prelude::*, thread_rng}; use serde::de::DeserializeOwned; use serde::{Deserialize, Serialize}; use tansu_kafka_sans_io::describe_configs_response::DescribeConfigsResourceResult; +use tansu_kafka_sans_io::txn_offset_commit_response::TxnOffsetCommitResponseTopic; use tansu_kafka_sans_io::{ create_topics_request::CreatableTopic, delete_records_request::DeleteRecordsTopic, @@ -41,13 +44,14 @@ use tansu_kafka_sans_io::{ record::{deflated, inflated}, ConfigResource, Encoder, ErrorCode, }; -use tansu_kafka_sans_io::{ConfigSource, ConfigType, Decoder}; +use tansu_kafka_sans_io::{ConfigSource, ConfigType, Decoder, IsolationLevel}; use tracing::{debug, error}; use uuid::Uuid; use crate::{ BrokerRegistationRequest, Error, GroupDetail, ListOffsetRequest, ListOffsetResponse, - MetadataResponse, OffsetCommitRequest, OffsetStage, Result, Storage, TopicId, Topition, + MetadataResponse, OffsetCommitRequest, OffsetStage, ProducerIdResponse, Result, Storage, + TopicId, Topition, TxnAddPartitionsRequest, TxnAddPartitionsResponse, TxnOffsetCommitRequest, UpdateError, Version, NULL_TOPIC_ID, }; @@ -57,10 +61,180 @@ const APPLICATION_JSON: &str = "application/json"; pub struct DynoStore { cluster: String, node: i32, + watermarks: BTreeMap>, + producers: ConditionData>, object_store: Arc, } +#[derive(Clone, Debug, Default, Eq, PartialEq)] +struct ConditionData { + path: Path, + version: Option, + tags: TagSet, + attributes: Attributes, + data: D, +} + +impl ConditionData +where + D: Clone + Debug + DeserializeOwned + Send + Serialize + Sync, +{ + async fn get(&mut self, object_store: &impl ObjectStore) -> Result<()> { + debug!(?self, ?object_store); + + let get_result = object_store + .get(&self.path) + .await + .inspect_err(|error| error!(?error, ?self)) + .map_err(|_error| Error::Api(ErrorCode::UnknownServerError))?; + + self.version = Some(UpdateVersion { + e_tag: get_result.meta.e_tag.clone(), + version: get_result.meta.version.clone(), + }); + + let encoded = get_result + .bytes() + .await + .inspect_err(|error| error!(?error, ?self)) + .map_err(|_error| Error::Api(ErrorCode::UnknownServerError))?; + + self.data = serde_json::from_slice::(&encoded[..]) + .inspect_err(|error| error!(?error, ?self)) + .map_err(|_error| Error::Api(ErrorCode::UnknownServerError))?; + + debug!(?self, ?object_store); + + Ok(()) + } + + async fn with_mut(&mut self, object_store: &impl ObjectStore, f: F) -> Result + where + E: Debug, + F: Fn(&mut D) -> Result, + { + debug!(?self, ?object_store); + + loop { + let outcome = f(&mut self.data)?; + debug!(?self, ?outcome); + + let payload = serde_json::to_vec(&self.data) + .map(Bytes::from) + .map(PutPayload::from)?; + + match object_store + .put_opts(&self.path, payload, PutOptions::from(&*self)) + .await + .inspect_err(|error| match error { + object_store::Error::AlreadyExists { .. } + | object_store::Error::Precondition { .. } => { + debug!(?error, ?self) + } + + _ => error!(?error, ?self), + }) + .inspect(|put_result| debug!(?self, ?put_result)) + { + Ok(result) => { + debug!(?self, ?result); + + self.version = Some(UpdateVersion { + e_tag: result.e_tag, + version: result.version, + }); + + return Ok(outcome); + } + + Err(pre_condition @ object_store::Error::Precondition { .. }) => { + debug!(?self, ?pre_condition); + self.get(object_store).await?; + continue; + } + + Err(already_exists @ object_store::Error::AlreadyExists { .. }) => { + debug!(?self, ?already_exists); + self.get(object_store).await?; + continue; + } + + Err(error) => { + return { + error!(?self, ?error); + Err(Error::Api(ErrorCode::UnknownServerError)) + } + } + } + } + } + + async fn with(&mut self, object_store: &impl ObjectStore, f: F) -> Result + where + F: Fn(&D) -> Result, + { + debug!(?self, ?object_store); + + match object_store + .get_opts( + &self.path, + GetOptions { + if_none_match: self + .version + .as_ref() + .and_then(|version| version.e_tag.clone()), + ..GetOptions::default() + }, + ) + .await + { + Ok(get_result) => { + self.version = Some(UpdateVersion { + e_tag: get_result.meta.e_tag.clone(), + version: get_result.meta.version.clone(), + }); + + let encoded = get_result + .bytes() + .await + .inspect_err(|error| error!(?error, ?self)) + .map_err(|_error| Error::Api(ErrorCode::UnknownServerError))?; + + self.data = serde_json::from_slice::(&encoded[..]) + .inspect_err(|error| error!(?error, ?self)) + .map_err(|_error| Error::Api(ErrorCode::UnknownServerError))?; + + Ok(()) + } + + Err(object_store::Error::NotModified { .. }) => Ok(()), + + Err(object_store::Error::NotFound { .. }) => { + self.with_mut(object_store, |_| Ok(())).await + } + + Err(error) => { + error!(?self, ?error); + Err(Error::Api(ErrorCode::UnknownServerError)) + } + } + .and(f(&self.data)) + } +} + +impl From<&ConditionData> for PutOptions { + fn from(value: &ConditionData) -> Self { + Self { + mode: value.version.as_ref().map_or(PutMode::Create, |existing| { + PutMode::Update(existing.to_owned()) + }), + tags: value.tags.to_owned(), + attributes: value.attributes.to_owned(), + } + } +} + #[derive(Clone, Debug, Default, Deserialize, Eq, Hash, Ord, PartialEq, PartialOrd, Serialize)] struct TopicMetadata { id: Uuid, @@ -72,42 +246,78 @@ struct Watermark { low: i64, high: i64, stable: i64, + producers: BTreeMap, } -impl DynoStore { - pub fn new(cluster: &str, node: i32, object_store: impl ObjectStore) -> Self { +impl ConditionData { + fn new(cluster: &str, topition: &Topition) -> Self { Self { - cluster: cluster.into(), - node, - object_store: Arc::new(object_store), + path: Path::from(format!( + "clusters/{}/topics/{}/partitions/{:0>10}/watermark.json", + cluster, topition.topic, topition.partition, + )), + data: Watermark::default(), + ..Default::default() } } +} - async fn get_watermark(&self, topition: &Topition) -> Result { - debug!(?topition); - let location = Path::from(format!( - "clusters/{}/topics/{}/partitions/{:0>10}/watermark.json", - self.cluster, topition.topic, topition.partition, - )); +#[derive(Clone, Debug, Deserialize, Eq, Hash, Ord, PartialEq, PartialOrd, Serialize)] +struct WatermarkSequence { + epoch: i16, + sequence: i32, + updated: SystemTime, +} - let get_result = self - .object_store - .get(&location) - .await - .inspect(|get_result| debug!(?get_result, ?topition)) - .inspect_err(|error| error!(?error, ?location)) - .map_err(|_error| Error::Api(ErrorCode::UnknownServerError))?; +impl Default for WatermarkSequence { + fn default() -> Self { + Self { + epoch: 0, + sequence: 0, + updated: SystemTime::now(), + } + } +} - let encoded = get_result - .bytes() - .await - .inspect_err(|error| error!(?error, ?location)) - .map_err(|_error| Error::Api(ErrorCode::UnknownServerError))?; +#[derive(Clone, Debug, Deserialize, Eq, Hash, Ord, PartialEq, PartialOrd, Serialize)] +struct Producer { + epoch: i16, + updated: SystemTime, +} - serde_json::from_slice::(&encoded[..]) - .inspect(|watermark| debug!(?watermark, ?topition)) - .inspect_err(|error| error!(?error, ?location)) - .map_err(|_error| Error::Api(ErrorCode::UnknownServerError)) +impl Default for Producer { + fn default() -> Self { + Self { + epoch: 0, + updated: SystemTime::now(), + } + } +} + +fn json_content_type() -> Attributes { + let mut attributes = Attributes::new(); + _ = attributes.insert( + Attribute::ContentType, + AttributeValue::from(APPLICATION_JSON), + ); + attributes +} + +impl DynoStore { + pub fn new(cluster: &str, node: i32, object_store: impl ObjectStore) -> Self { + Self { + cluster: cluster.into(), + node, + watermarks: BTreeMap::new(), + producers: ConditionData { + path: Path::from(format!("clusters/{}/producers.json", cluster)), + version: None, + attributes: Attributes::new(), + tags: TagSet::default(), + data: BTreeMap::new(), + }, + object_store: Arc::new(object_store), + } } async fn topic_metadata(&self, topic: &TopicId) -> Result { @@ -210,7 +420,10 @@ impl DynoStore { #[async_trait] impl Storage for DynoStore { - async fn register_broker(&self, broker_registration: BrokerRegistationRequest) -> Result<()> { + async fn register_broker( + &mut self, + broker_registration: BrokerRegistationRequest, + ) -> Result<()> { debug!(?broker_registration); let payload = serde_json::to_vec(&broker_registration) @@ -222,16 +435,10 @@ impl Storage for DynoStore { self.cluster, self.node )); - let mut attributes = Attributes::new(); - _ = attributes.insert( - Attribute::ContentType, - AttributeValue::from(APPLICATION_JSON), - ); - let options = PutOptions { mode: PutMode::Overwrite, tags: TagSet::default(), - attributes, + attributes: json_content_type(), }; let put_result = self @@ -244,7 +451,7 @@ impl Storage for DynoStore { Ok(()) } - async fn create_topic(&self, topic: CreatableTopic, validate_only: bool) -> Result { + async fn create_topic(&mut self, topic: CreatableTopic, validate_only: bool) -> Result { debug!(?topic, ?validate_only); let id = Uuid::now_v7(); @@ -255,16 +462,10 @@ impl Storage for DynoStore { .map(Bytes::from) .map(PutPayload::from)?; - let mut attributes = Attributes::new(); - _ = attributes.insert( - Attribute::ContentType, - AttributeValue::from(APPLICATION_JSON), - ); - let options = PutOptions { mode: PutMode::Create, tags: TagSet::default(), - attributes, + attributes: json_content_type(), }; match ( @@ -316,16 +517,10 @@ impl Storage for DynoStore { self.cluster, td.topic.name, partition, )); - let mut attributes = Attributes::new(); - _ = attributes.insert( - Attribute::ContentType, - AttributeValue::from(APPLICATION_JSON), - ); - let options = PutOptions { mode: PutMode::Create, tags: TagSet::default(), - attributes, + attributes: json_content_type(), }; match self @@ -362,14 +557,14 @@ impl Storage for DynoStore { } async fn delete_records( - &self, + &mut self, topics: &[DeleteRecordsTopic], ) -> Result> { debug!(?topics); todo!() } - async fn delete_topic(&self, topic: &TopicId) -> Result { + async fn delete_topic(&mut self, topic: &TopicId) -> Result { debug!(?topic); if let Ok(metadata) = self.topic_metadata(topic).await { @@ -445,7 +640,7 @@ impl Storage for DynoStore { } } - async fn brokers(&self) -> Result> { + async fn brokers(&mut self) -> Result> { let location = Path::from(format!("clusters/{}/brokers/", self.cluster)); debug!(?location); @@ -502,105 +697,126 @@ impl Storage for DynoStore { Ok(brokers) } - async fn produce(&self, topition: &Topition, deflated: deflated::Batch) -> Result { - debug!(?topition, ?deflated); - - let location = Path::from(format!( - "clusters/{}/topics/{}/partitions/{:0>10}/watermark.json", - self.cluster, topition.topic, topition.partition, - )); - - loop { - let get_result = self - .object_store - .get(&location) - .await - .inspect_err(|error| error!(?error, ?location)) - .map_err(|_error| Error::Api(ErrorCode::UnknownServerError))?; - - let update_version = UpdateVersion { - e_tag: get_result.meta.e_tag.clone(), - version: get_result.meta.version.clone(), - }; - - let encoded = get_result - .bytes() - .await - .inspect_err(|error| error!(?error, ?location)) - .map_err(|_error| Error::Api(ErrorCode::UnknownServerError))?; + async fn produce( + &mut self, + transaction_id: Option<&str>, + topition: &Topition, + deflated: deflated::Batch, + ) -> Result { + debug!(?transaction_id, ?topition, ?deflated); + + if deflated.producer_id > 0 { + self.producers + .with(&self.object_store, |producers| { + debug!(?producers, ?deflated.producer_id); + producers.get(&deflated.producer_id).map_or( + Err(Error::Api(ErrorCode::UnknownProducerId)), + |producer| { + if producer.epoch == deflated.producer_epoch { + Ok(()) + } else { + Err(Error::Api(ErrorCode::ProducerFenced)) + } + }, + ) + }) + .await? + } - let original = serde_json::from_slice::(&encoded[..]) - .inspect_err(|error| error!(?error, ?location)) - .map_err(|_error| Error::Api(ErrorCode::UnknownServerError))?; + let offset = self + .watermarks + .entry(topition.to_owned()) + .or_insert(ConditionData::::new( + self.cluster.as_str(), + topition, + )) + .with_mut(&self.object_store, |watermark| { + debug!(?watermark); + + if deflated.producer_id > 0 { + if let Some(ws) = watermark.producers.get_mut(&deflated.producer_id) { + debug!(?ws); + + match ws.epoch.cmp(&deflated.producer_epoch) { + Ordering::Equal => match ws.sequence.cmp(&deflated.base_sequence) { + Ordering::Equal => { + debug!(?ws, ?deflated.base_sequence); + + ws.sequence += deflated.last_offset_delta + 1; + + let offset = watermark.high; + watermark.high += deflated.last_offset_delta as i64 + 1i64; + Ok(offset) + } - let mut updated = original.clone(); - updated.high += deflated.record_count as i64; + Ordering::Greater => { + debug!(?ws, ?deflated.base_sequence); + Err(Error::Api(ErrorCode::DuplicateSequenceNumber)) + } - let payload = serde_json::to_vec(&updated) - .map(Bytes::from) - .map(PutPayload::from)?; + Ordering::Less => { + debug!(?ws, ?deflated.base_sequence); + Err(Error::Api(ErrorCode::OutOfOrderSequenceNumber)) + } + }, - let mut attributes = Attributes::new(); - _ = attributes.insert( - Attribute::ContentType, - AttributeValue::from(APPLICATION_JSON), - ); + Ordering::Greater => Err(Error::Api(ErrorCode::ProducerFenced)), - let options = PutOptions { - mode: PutMode::Update(update_version), - tags: TagSet::default(), - attributes, - }; + Ordering::Less => Err(Error::Api(ErrorCode::InvalidProducerEpoch)), + } + } else { + let offset = watermark.high; + watermark.high += deflated.last_offset_delta as i64 + 1i64; + _ = watermark.producers.insert( + deflated.producer_id, + WatermarkSequence { + epoch: deflated.producer_epoch, + sequence: deflated.last_offset_delta + 1, + ..Default::default() + }, + ); + Ok(offset) + } + } else { + let offset = watermark.high; + watermark.high += deflated.last_offset_delta as i64 + 1i64; + Ok(offset) + } + }) + .await?; - match self - .object_store - .put_opts(&location, payload, options) - .await - .inspect_err(|error| error!(?error, ?location)) - .inspect(|put_result| debug!(?location, ?put_result)) - { - Ok(_) => { - let location = Path::from(format!( - "clusters/{}/topics/{}/partitions/{:0>10}/records/{:0>20}.batch", - self.cluster, topition.topic, topition.partition, original.high, - )); + let location = Path::from(format!( + "clusters/{}/topics/{}/partitions/{:0>10}/records/{:0>20}.batch", + self.cluster, topition.topic, topition.partition, offset, + )); - let payload = self.encode(deflated)?; - let attributes = Attributes::new(); + let payload = self.encode(deflated)?; + let attributes = Attributes::new(); - let options = PutOptions { - mode: PutMode::Create, - tags: TagSet::default(), - attributes, - }; + let options = PutOptions { + mode: PutMode::Create, + tags: TagSet::default(), + attributes, + }; - _ = self - .object_store - .put_opts(&location, payload, options) - .await - .inspect_err(|error| error!(?error))?; + _ = self + .object_store + .put_opts(&location, payload, options) + .await + .inspect_err(|error| error!(?error))?; - return Ok(original.high); - } - Err(object_store::Error::Precondition { .. }) => continue, - Err(error) => { - return { - error!(?error); - Err(Error::Api(ErrorCode::UnknownServerError)) - } - } - } - } + Ok(offset) } async fn fetch( - &self, + &mut self, topition: &'_ Topition, offset: i64, min_bytes: u32, max_bytes: u32, + isolation: IsolationLevel, ) -> Result { - debug!(?topition, ?offset, ?min_bytes, ?max_bytes); + debug!(?topition, ?offset, ?min_bytes, ?max_bytes, ?isolation); let location = Path::from(format!( "clusters/{}/topics/{}/partitions/{:0>10}/records/", @@ -660,23 +876,31 @@ impl Storage for DynoStore { }) } - async fn offset_stage(&self, topition: &Topition) -> Result { + async fn offset_stage(&mut self, topition: &Topition) -> Result { debug!(?topition); - self.get_watermark(topition) - .await - .map(|watermark| OffsetStage { - last_stable: watermark.stable, - high_watermark: watermark.high, - log_start: watermark.low, + self.watermarks + .entry(topition.to_owned()) + .or_insert(ConditionData::::new( + self.cluster.as_str(), + topition, + )) + .with(&self.object_store, |watermark| { + Ok(OffsetStage { + last_stable: watermark.stable, + high_watermark: watermark.high, + log_start: watermark.low, + }) }) + .await } async fn list_offsets( - &self, + &mut self, + isolation_level: IsolationLevel, offsets: &[(Topition, ListOffsetRequest)], ) -> Result> { - debug!(?offsets); + debug!(?offsets, ?isolation_level); let mut responses = vec![]; @@ -685,22 +909,36 @@ impl Storage for DynoStore { topition.to_owned(), match offset_request { ListOffsetRequest::Earliest => { - self.get_watermark(topition) - .await - .map(|watermark| ListOffsetResponse { - error_code: ErrorCode::None, - timestamp: None, - offset: Some(watermark.low), - })? + self.watermarks + .entry(topition.to_owned()) + .or_insert(ConditionData::::new( + self.cluster.as_str(), + topition, + )) + .with(&self.object_store, |watermark| { + Ok(ListOffsetResponse { + error_code: ErrorCode::None, + timestamp: None, + offset: Some(watermark.low), + }) + }) + .await? } ListOffsetRequest::Latest => { - self.get_watermark(topition) - .await - .map(|watermark| ListOffsetResponse { - error_code: ErrorCode::None, - timestamp: None, - offset: Some(watermark.high), - })? + self.watermarks + .entry(topition.to_owned()) + .or_insert(ConditionData::::new( + self.cluster.as_str(), + topition, + )) + .with(&self.object_store, |watermark| { + Ok(ListOffsetResponse { + error_code: ErrorCode::None, + timestamp: None, + offset: Some(watermark.high), + }) + }) + .await? } ListOffsetRequest::Timestamp(..) => todo!(), }, @@ -711,7 +949,7 @@ impl Storage for DynoStore { } async fn offset_commit( - &self, + &mut self, group_id: &str, retention_time_ms: Option, offsets: &[(Topition, OffsetCommitRequest)], @@ -730,16 +968,10 @@ impl Storage for DynoStore { .map(Bytes::from) .map(PutPayload::from)?; - let mut attributes = Attributes::new(); - _ = attributes.insert( - Attribute::ContentType, - AttributeValue::from(APPLICATION_JSON), - ); - let options = PutOptions { mode: PutMode::Overwrite, tags: TagSet::default(), - attributes, + attributes: json_content_type(), }; let error_code = self @@ -755,7 +987,7 @@ impl Storage for DynoStore { } async fn offset_fetch( - &self, + &mut self, group_id: Option<&str>, topics: &[Topition], require_stable: Option, @@ -798,7 +1030,7 @@ impl Storage for DynoStore { Ok(responses) } - async fn metadata(&self, topics: Option<&[TopicId]>) -> Result { + async fn metadata(&mut self, topics: Option<&[TopicId]>) -> Result { debug!(?topics); let location = Path::from(format!("clusters/{}/brokers/", self.cluster)); @@ -1080,7 +1312,7 @@ impl Storage for DynoStore { } async fn describe_config( - &self, + &mut self, name: &str, resource: ConfigResource, keys: Option<&[String]>, @@ -1119,7 +1351,7 @@ impl Storage for DynoStore { } async fn update_group( - &self, + &mut self, group_id: &str, detail: GroupDetail, version: Option, @@ -1131,14 +1363,155 @@ impl Storage for DynoStore { self.cluster, group_id, )); - let mut attributes = Attributes::new(); - _ = attributes.insert( - Attribute::ContentType, - AttributeValue::from(APPLICATION_JSON), + self.put( + &location, + detail, + json_content_type(), + version.map(Into::into), + ) + .await + .map(Into::into) + } + + async fn init_producer( + &mut self, + transaction_id: Option<&str>, + transaction_timeout_ms: i32, + producer_id: Option, + producer_epoch: Option, + ) -> Result { + debug!( + ?transaction_id, + ?transaction_timeout_ms, + ?producer_id, + ?producer_epoch, ); - self.put(&location, detail, attributes, version.map(Into::into)) - .await - .map(Into::into) + if let Some(_transaction_id) = transaction_id { + self.producers + .with_mut(&self.object_store, |producers| { + debug!(?producers); + match (producer_id, producer_epoch) { + (Some(-1), Some(-1)) => { + let id = producers.last_key_value().map_or(1, |(k, _)| k + 1); + _ = producers.insert(id, Producer::default()); + + Ok(ProducerIdResponse { + id, + epoch: 0, + ..Default::default() + }) + } + + (Some(producer_id), Some(producer_epoch)) + if producer_id > 0 && producer_epoch >= 0 => + { + match producers.get(&producer_id) { + Some(Producer { epoch, .. }) if producer_epoch == *epoch => { + if let Some(epoch) = epoch.checked_add(1) { + _ = producers.insert( + producer_id, + Producer { + epoch, + ..Default::default() + }, + ); + + Ok(ProducerIdResponse { + id: producer_id, + epoch, + ..Default::default() + }) + } else { + let id = + producers.last_key_value().map_or(1, |(k, _)| k + 1); + _ = producers.insert(id, Producer::default()); + + Ok(ProducerIdResponse { + id, + epoch: 0, + ..Default::default() + }) + } + } + + Some(_) | None => Ok(ProducerIdResponse { + id: -1, + epoch: -1, + error: ErrorCode::InvalidProducerEpoch, + }), + } + } + + (None, None) => todo!(), + (None, Some(_)) => todo!(), + (Some(_), None) => todo!(), + (_, _) => todo!(), + } + }) + .await + } else { + self.producers + .with_mut(&self.object_store, |producers| { + debug!(?producers); + match (producer_id, producer_epoch) { + (Some(-1), Some(-1)) => { + let id = producers.last_key_value().map_or(1, |(k, _)| k + 1); + _ = producers.insert(id, Producer::default()); + + Ok(ProducerIdResponse { + id, + epoch: 0, + ..Default::default() + }) + } + + (None, None) => todo!(), + (None, Some(_)) => todo!(), + (Some(_), None) => todo!(), + (_, _) => todo!(), + } + }) + .await + } + } + + async fn txn_add_offsets( + &mut self, + transaction_id: &str, + producer_id: i64, + producer_epoch: i16, + group_id: &str, + ) -> Result { + debug!(?transaction_id, ?producer_id, ?producer_epoch, ?group_id); + + Ok(ErrorCode::None) + } + + async fn txn_add_partitions( + &mut self, + partitions: TxnAddPartitionsRequest, + ) -> Result { + debug!(?partitions); + todo!() + } + + async fn txn_offset_commit( + &mut self, + offsets: TxnOffsetCommitRequest, + ) -> Result> { + debug!(?offsets); + todo!() + } + + async fn txn_end( + &mut self, + transaction_id: &str, + producer_id: i64, + producer_epoch: i16, + committed: bool, + ) -> Result { + debug!(?transaction_id, ?producer_id, ?producer_epoch, ?committed); + Ok(ErrorCode::None) } } diff --git a/tansu-storage/src/lib.rs b/tansu-storage/src/lib.rs index 48a4a57..4e0f75a 100644 --- a/tansu-storage/src/lib.rs +++ b/tansu-storage/src/lib.rs @@ -35,6 +35,8 @@ use std::{ time::{Duration, SystemTime, SystemTimeError}, }; use tansu_kafka_sans_io::{ + add_partitions_to_txn_request::{AddPartitionsToTxnTopic, AddPartitionsToTxnTransaction}, + add_partitions_to_txn_response::{AddPartitionsToTxnResult, AddPartitionsToTxnTopicResult}, broker_registration_request::{Feature, Listener}, create_topics_request::CreatableTopic, delete_records_request::DeleteRecordsTopic, @@ -48,8 +50,12 @@ use tansu_kafka_sans_io::{ metadata_response::{MetadataResponseBroker, MetadataResponseTopic}, offset_commit_request::OffsetCommitRequestPartition, record::deflated, - to_system_time, to_timestamp, ConfigResource, ErrorCode, + to_system_time, to_timestamp, + txn_offset_commit_request::TxnOffsetCommitRequestTopic, + txn_offset_commit_response::TxnOffsetCommitResponseTopic, + Body, ConfigResource, ErrorCode, IsolationLevel, }; +use tracing::debug; use uuid::Uuid; pub mod dynostore; @@ -146,6 +152,9 @@ pub enum Error { #[error("try from slice: {0}")] TryFromSlice(#[from] TryFromSliceError), + #[error("body: {0:?}")] + UnexpectedBody(Body), + #[error("url: {0}")] Url(#[from] url::ParseError), } @@ -273,9 +282,9 @@ pub enum ListOffsetRequest { #[derive(Copy, Clone, Debug, Eq, Hash, Ord, PartialEq, PartialOrd)] pub struct ListOffsetResponse { - error_code: ErrorCode, - timestamp: Option, - offset: Option, + pub error_code: ErrorCode, + pub timestamp: Option, + pub offset: Option, } impl Default for ListOffsetResponse { @@ -555,6 +564,102 @@ pub struct Version { version: Option, } +#[derive(Copy, Clone, Debug, Deserialize, Eq, Hash, Ord, PartialEq, PartialOrd, Serialize)] +pub struct ProducerIdResponse { + pub error: ErrorCode, + pub id: i64, + pub epoch: i16, +} + +impl Default for ProducerIdResponse { + fn default() -> Self { + Self { + error: ErrorCode::None, + id: 1, + epoch: 0, + } + } +} + +#[derive(Clone, Debug, Deserialize, Eq, Hash, Ord, PartialEq, PartialOrd, Serialize)] +pub enum TxnAddPartitionsRequest { + VersionZeroToThree { + transaction_id: String, + producer_id: i64, + producer_epoch: i16, + topics: Vec, + }, + + VersionFourPlus { + transactions: Vec, + }, +} + +impl TryFrom for TxnAddPartitionsRequest { + type Error = Error; + + fn try_from(value: Body) -> result::Result { + match value { + Body::AddPartitionsToTxnRequest { + transactions: None, + v_3_and_below_transactional_id: Some(transactional_id), + v_3_and_below_producer_id: Some(producer_id), + v_3_and_below_producer_epoch: Some(producer_epoch), + v_3_and_below_topics: Some(topics), + } => Ok(Self::VersionZeroToThree { + transaction_id: transactional_id, + producer_id, + producer_epoch, + topics, + }), + + Body::AddPartitionsToTxnRequest { + transactions: Some(transactions), + v_3_and_below_transactional_id: None, + v_3_and_below_producer_id: None, + v_3_and_below_producer_epoch: None, + v_3_and_below_topics: None, + } => Ok(Self::VersionFourPlus { transactions }), + + unexpected => Err(Error::UnexpectedBody(unexpected)), + } + } +} + +#[derive(Clone, Debug, Deserialize, Eq, Hash, Ord, PartialEq, PartialOrd, Serialize)] +pub enum TxnAddPartitionsResponse { + VersionZeroToThree(Vec), + VersionFourPlus(Vec), +} + +impl TxnAddPartitionsResponse { + pub fn zero_to_three(&self) -> &[AddPartitionsToTxnTopicResult] { + match self { + Self::VersionZeroToThree(result) => result.as_slice(), + Self::VersionFourPlus(_) => &[][..], + } + } + + pub fn four_plus(&self) -> &[AddPartitionsToTxnResult] { + match self { + Self::VersionZeroToThree(_) => &[][..], + Self::VersionFourPlus(result) => result.as_slice(), + } + } +} + +#[derive(Clone, Debug, Deserialize, Eq, Hash, Ord, PartialEq, PartialOrd, Serialize)] +pub struct TxnOffsetCommitRequest { + pub transaction_id: String, + pub group_id: String, + pub producer_id: i64, + pub producer_epoch: i16, + pub generation_id: Option, + pub member_id: Option, + pub group_instance_id: Option, + pub topics: Vec, +} + #[async_trait] pub trait StorageProvider { async fn provide_storage(&mut self) -> impl Storage; @@ -562,65 +667,109 @@ pub trait StorageProvider { #[async_trait] pub trait Storage: Clone + Debug + Send + Sync + 'static { - async fn register_broker(&self, broker_registration: BrokerRegistationRequest) -> Result<()>; + async fn register_broker( + &mut self, + broker_registration: BrokerRegistationRequest, + ) -> Result<()>; - async fn create_topic(&self, topic: CreatableTopic, validate_only: bool) -> Result; + async fn create_topic(&mut self, topic: CreatableTopic, validate_only: bool) -> Result; async fn delete_records( - &self, + &mut self, topics: &[DeleteRecordsTopic], ) -> Result>; - async fn delete_topic(&self, topic: &TopicId) -> Result; + async fn delete_topic(&mut self, topic: &TopicId) -> Result; - async fn brokers(&self) -> Result>; + async fn brokers(&mut self) -> Result>; - async fn produce(&self, topition: &Topition, batch: deflated::Batch) -> Result; + async fn produce( + &mut self, + transaction_id: Option<&str>, + topition: &Topition, + batch: deflated::Batch, + ) -> Result; async fn fetch( - &self, + &mut self, topition: &'_ Topition, offset: i64, min_bytes: u32, max_bytes: u32, + isolation: IsolationLevel, ) -> Result; - async fn offset_stage(&self, topition: &Topition) -> Result; + async fn offset_stage(&mut self, topition: &Topition) -> Result; async fn list_offsets( - &self, + &mut self, + isolation_level: IsolationLevel, offsets: &[(Topition, ListOffsetRequest)], ) -> Result>; async fn offset_commit( - &self, + &mut self, group_id: &str, retention_time_ms: Option, offsets: &[(Topition, OffsetCommitRequest)], ) -> Result>; async fn offset_fetch( - &self, + &mut self, group_id: Option<&str>, topics: &[Topition], require_stable: Option, ) -> Result>; - async fn metadata(&self, topics: Option<&[TopicId]>) -> Result; + async fn metadata(&mut self, topics: Option<&[TopicId]>) -> Result; async fn describe_config( - &self, + &mut self, name: &str, resource: ConfigResource, keys: Option<&[String]>, ) -> Result; async fn update_group( - &self, + &mut self, group_id: &str, detail: GroupDetail, version: Option, ) -> Result>; + + async fn init_producer( + &mut self, + transaction_id: Option<&str>, + transaction_timeout_ms: i32, + producer_id: Option, + producer_epoch: Option, + ) -> Result; + + async fn txn_add_offsets( + &mut self, + transaction_id: &str, + producer_id: i64, + producer_epoch: i16, + group_id: &str, + ) -> Result; + + async fn txn_add_partitions( + &mut self, + partitions: TxnAddPartitionsRequest, + ) -> Result; + + async fn txn_offset_commit( + &mut self, + offsets: TxnOffsetCommitRequest, + ) -> Result>; + + async fn txn_end( + &mut self, + transaction_id: &str, + producer_id: i64, + producer_epoch: i16, + committed: bool, + ) -> Result; } #[derive(Debug, thiserror::Error)] @@ -642,14 +791,17 @@ pub enum StorageContainer { #[async_trait] impl Storage for StorageContainer { - async fn register_broker(&self, broker_registration: BrokerRegistationRequest) -> Result<()> { + async fn register_broker( + &mut self, + broker_registration: BrokerRegistationRequest, + ) -> Result<()> { match self { Self::Postgres(pg) => pg.register_broker(broker_registration).await, Self::DynoStore(dyn_store) => dyn_store.register_broker(broker_registration).await, } } - async fn create_topic(&self, topic: CreatableTopic, validate_only: bool) -> Result { + async fn create_topic(&mut self, topic: CreatableTopic, validate_only: bool) -> Result { match self { Self::Postgres(pg) => pg.create_topic(topic, validate_only).await, Self::DynoStore(dyn_store) => dyn_store.create_topic(topic, validate_only).await, @@ -657,7 +809,7 @@ impl Storage for StorageContainer { } async fn delete_records( - &self, + &mut self, topics: &[DeleteRecordsTopic], ) -> Result> { match self { @@ -666,45 +818,54 @@ impl Storage for StorageContainer { } } - async fn delete_topic(&self, topic: &TopicId) -> Result { + async fn delete_topic(&mut self, topic: &TopicId) -> Result { match self { Self::Postgres(pg) => pg.delete_topic(topic).await, Self::DynoStore(dyn_store) => dyn_store.delete_topic(topic).await, } } - async fn brokers(&self) -> Result> { + async fn brokers(&mut self) -> Result> { match self { Self::Postgres(pg) => pg.brokers().await, Self::DynoStore(dyn_store) => dyn_store.brokers().await, } } - async fn produce(&self, topition: &Topition, batch: deflated::Batch) -> Result { + async fn produce( + &mut self, + transaction_id: Option<&str>, + topition: &Topition, + batch: deflated::Batch, + ) -> Result { match self { - Self::Postgres(pg) => pg.produce(topition, batch).await, - Self::DynoStore(dyn_store) => dyn_store.produce(topition, batch).await, + Self::Postgres(pg) => pg.produce(transaction_id, topition, batch).await, + Self::DynoStore(dyn_store) => dyn_store.produce(transaction_id, topition, batch).await, } } async fn fetch( - &self, + &mut self, topition: &'_ Topition, offset: i64, min_bytes: u32, max_bytes: u32, + isolation: IsolationLevel, ) -> Result { match self { - Self::Postgres(pg) => pg.fetch(topition, offset, min_bytes, max_bytes).await, + Self::Postgres(pg) => { + pg.fetch(topition, offset, min_bytes, max_bytes, isolation) + .await + } Self::DynoStore(dyn_store) => { dyn_store - .fetch(topition, offset, min_bytes, max_bytes) + .fetch(topition, offset, min_bytes, max_bytes, isolation) .await } } } - async fn offset_stage(&self, topition: &Topition) -> Result { + async fn offset_stage(&mut self, topition: &Topition) -> Result { match self { Self::Postgres(pg) => pg.offset_stage(topition).await, Self::DynoStore(dyn_store) => dyn_store.offset_stage(topition).await, @@ -712,17 +873,18 @@ impl Storage for StorageContainer { } async fn list_offsets( - &self, + &mut self, + isolation_level: IsolationLevel, offsets: &[(Topition, ListOffsetRequest)], ) -> Result> { match self { - Self::Postgres(pg) => pg.list_offsets(offsets).await, - Self::DynoStore(dyn_store) => dyn_store.list_offsets(offsets).await, + Self::Postgres(pg) => pg.list_offsets(isolation_level, offsets).await, + Self::DynoStore(dyn_store) => dyn_store.list_offsets(isolation_level, offsets).await, } } async fn offset_commit( - &self, + &mut self, group_id: &str, retention_time_ms: Option, offsets: &[(Topition, OffsetCommitRequest)], @@ -738,7 +900,7 @@ impl Storage for StorageContainer { } async fn offset_fetch( - &self, + &mut self, group_id: Option<&str>, topics: &[Topition], require_stable: Option, @@ -753,7 +915,7 @@ impl Storage for StorageContainer { } } - async fn metadata(&self, topics: Option<&[TopicId]>) -> Result { + async fn metadata(&mut self, topics: Option<&[TopicId]>) -> Result { match self { Self::Postgres(pg) => pg.metadata(topics).await, Self::DynoStore(dyn_store) => dyn_store.metadata(topics).await, @@ -761,7 +923,7 @@ impl Storage for StorageContainer { } async fn describe_config( - &self, + &mut self, name: &str, resource: ConfigResource, keys: Option<&[String]>, @@ -773,7 +935,7 @@ impl Storage for StorageContainer { } async fn update_group( - &self, + &mut self, group_id: &str, detail: GroupDetail, version: Option, @@ -783,6 +945,103 @@ impl Storage for StorageContainer { Self::DynoStore(dyn_store) => dyn_store.update_group(group_id, detail, version).await, } } + + async fn init_producer( + &mut self, + transaction_id: Option<&str>, + transaction_timeout_ms: i32, + producer_id: Option, + producer_epoch: Option, + ) -> Result { + debug!( + ?transaction_id, + ?transaction_timeout_ms, + ?producer_id, + ?producer_epoch + ); + + match self { + Self::Postgres(pg) => { + pg.init_producer( + transaction_id, + transaction_timeout_ms, + producer_id, + producer_epoch, + ) + .await + } + Self::DynoStore(dyn_store) => { + dyn_store + .init_producer( + transaction_id, + transaction_timeout_ms, + producer_id, + producer_epoch, + ) + .await + } + } + } + + async fn txn_add_offsets( + &mut self, + transaction_id: &str, + producer_id: i64, + producer_epoch: i16, + group_id: &str, + ) -> Result { + match self { + Self::Postgres(pg) => { + pg.txn_add_offsets(transaction_id, producer_id, producer_epoch, group_id) + .await + } + Self::DynoStore(dyn_store) => { + dyn_store + .txn_add_offsets(transaction_id, producer_id, producer_epoch, group_id) + .await + } + } + } + + async fn txn_add_partitions( + &mut self, + partitions: TxnAddPartitionsRequest, + ) -> Result { + match self { + Self::Postgres(pg) => pg.txn_add_partitions(partitions).await, + Self::DynoStore(dyn_store) => dyn_store.txn_add_partitions(partitions).await, + } + } + + async fn txn_offset_commit( + &mut self, + offsets: TxnOffsetCommitRequest, + ) -> Result> { + match self { + Self::Postgres(pg) => pg.txn_offset_commit(offsets).await, + Self::DynoStore(dyn_store) => dyn_store.txn_offset_commit(offsets).await, + } + } + + async fn txn_end( + &mut self, + transaction_id: &str, + producer_id: i64, + producer_epoch: i16, + committed: bool, + ) -> Result { + match self { + Self::Postgres(pg) => { + pg.txn_end(transaction_id, producer_id, producer_epoch, committed) + .await + } + Self::DynoStore(dyn_store) => { + dyn_store + .txn_end(transaction_id, producer_id, producer_epoch, committed) + .await + } + } + } } #[cfg(test)] diff --git a/tansu-storage/src/pg.rs b/tansu-storage/src/pg.rs index fd5a5c7..f5201ba 100644 --- a/tansu-storage/src/pg.rs +++ b/tansu-storage/src/pg.rs @@ -26,6 +26,9 @@ use deadpool_postgres::{Manager, ManagerConfig, Object, Pool, RecyclingMethod}; use rand::{prelude::*, thread_rng}; use serde_json::Value; use tansu_kafka_sans_io::{ + add_partitions_to_txn_response::{ + AddPartitionsToTxnPartitionResult, AddPartitionsToTxnTopicResult, + }, create_topics_request::CreatableTopic, delete_records_request::DeleteRecordsTopic, delete_records_response::{DeleteRecordsPartitionResult, DeleteRecordsTopicResult}, @@ -33,78 +36,21 @@ use tansu_kafka_sans_io::{ describe_configs_response::{DescribeConfigsResourceResult, DescribeConfigsResult}, metadata_response::{MetadataResponseBroker, MetadataResponsePartition, MetadataResponseTopic}, record::{deflated, inflated, Header, Record}, - to_system_time, to_timestamp, ConfigResource, ConfigSource, ConfigType, ErrorCode, + to_system_time, to_timestamp, + txn_offset_commit_response::{TxnOffsetCommitResponsePartition, TxnOffsetCommitResponseTopic}, + ConfigResource, ConfigSource, ConfigType, ErrorCode, IsolationLevel, }; -use tokio_postgres::{error::SqlState, Config, NoTls, Transaction}; +use tokio_postgres::{error::SqlState, Config, NoTls}; use tracing::{debug, error}; use uuid::Uuid; use crate::{ BrokerRegistationRequest, Error, GroupDetail, ListOffsetRequest, ListOffsetResponse, - MetadataResponse, OffsetCommitRequest, OffsetStage, Result, Storage, TopicId, Topition, + MetadataResponse, OffsetCommitRequest, OffsetStage, ProducerIdResponse, Result, Storage, + TopicId, Topition, TxnAddPartitionsRequest, TxnAddPartitionsResponse, TxnOffsetCommitRequest, UpdateError, Version, NULL_TOPIC_ID, }; -const DELETE_CONSUMER_OFFSETS_FOR_TOPIC: &str = concat!( - "delete from consumer_offset", - " using", - " cluster", - ", topic", - " where", - " consumer_offset.topic = topic.id", - " and", - " topic.cluster = cluster.id", - " and", - " cluster.name = $1", - " and", - " topic. = $2" -); - -const DELETE_HEADERS_FOR_TOPIC: &str = concat!( - "delete from header", - " using", - " cluster", - ", record", - ", topic", - " where", - " header.record = record.id", - " and", - " record.topic = topic.id", - " and", - " topic.cluster = cluster.id", - " and", - " cluster.name = $1", - " and", - " topic. = $2" -); - -const DELETE_RECORDS_FOR_TOPIC: &str = concat!( - "delete from record", - " using", - " cluster", - ", topic", - " where", - " record.topic = topic.id", - " and", - " topic.cluster = cluster.id", - " and", - " cluster.name = $1", - " and", - " topic. = $2" -); - -const DELETE_TOPIC: &str = concat!( - "delete from topic", - " using", - " cluster", - " where", - " topic.cluster = cluster.id", - " and", - " cluster.name = $1", - " and", - " topic. = $2" -); - #[derive(Clone, Debug)] pub struct Postgres { cluster: String, @@ -188,41 +134,14 @@ impl Postgres { async fn connection(&self) -> Result { self.pool.get().await.map_err(Into::into) } - - async fn delete_for_topic( - &self, - tx: &Transaction<'_>, - sql: &str, - topic: &TopicId, - ) -> Result { - match topic { - TopicId::Id(id) => { - let sql = sql.replace("", "id"); - - let prepared = tx.prepare(&sql).await.inspect_err(|err| error!(?err))?; - - tx.execute(&prepared, &[&self.cluster, &id]) - .await - .inspect_err(|err| error!(?err)) - .map_err(Into::into) - } - TopicId::Name(name) => { - let sql = sql.replace("", "name"); - - let prepared = tx.prepare(&sql).await.inspect_err(|err| error!(?err))?; - - tx.execute(&prepared, &[&self.cluster, &name]) - .await - .inspect_err(|err| error!(?err)) - .map_err(Into::into) - } - } - } } #[async_trait] impl Storage for Postgres { - async fn register_broker(&self, broker_registration: BrokerRegistationRequest) -> Result<()> { + async fn register_broker( + &mut self, + broker_registration: BrokerRegistationRequest, + ) -> Result<()> { debug!(?broker_registration); let mut c = self.connection().await?; @@ -318,13 +237,13 @@ impl Storage for Postgres { Ok(()) } - async fn brokers(&self) -> Result> { + async fn brokers(&mut self) -> Result> { let c = self.connection().await?; let prepared = c .prepare(concat!( "select", - " broker.id, host, port, rack", + " node, host, port, rack", " from broker, cluster, listener", " where", " cluster.name = $1", @@ -357,30 +276,23 @@ impl Storage for Postgres { Ok(brokers) } - async fn create_topic(&self, topic: CreatableTopic, validate_only: bool) -> Result { + async fn create_topic(&mut self, topic: CreatableTopic, validate_only: bool) -> Result { debug!(?topic, ?validate_only); let mut c = self.connection().await?; let tx = c.transaction().await?; let prepared = tx - .prepare(concat!( - "insert into topic", - " (cluster, name, partitions, replication_factor, is_internal)", - " select cluster.id, $2, $3, $4, false", - " from cluster", - " where cluster.name = $1", - " returning topic.id", - )) + .prepare(include_str!("pg/topic_insert.sql")) .await .inspect_err(|err| error!(?err, ?topic, ?validate_only))?; - let topic_id = tx + let topic_uuid = tx .query_one( &prepared, &[ - &self.cluster.as_str(), - &topic.name.as_str(), + &self.cluster, + &topic.name, &topic.num_partitions, &(topic.replication_factor as i32), ], @@ -399,16 +311,37 @@ impl Storage for Postgres { } })?; - debug!(?topic_id); + debug!(?topic_uuid, ?self.cluster, ?topic.name); + + for partition in 0..topic.num_partitions { + let prepared = tx + .prepare(include_str!("pg/topition_insert.sql")) + .await + .inspect_err(|err| error!(?err))?; + + _ = tx + .query_one(&prepared, &[&self.cluster, &topic.name, &partition]) + .await + .inspect_err(|err| error!(?err))?; + } + + for partition in 0..topic.num_partitions { + let prepared = tx + .prepare(include_str!("pg/watermark_insert.sql")) + .await + .inspect_err(|err| error!(?err))?; + + _ = tx + .query_one(&prepared, &[&self.cluster, &topic.name, &partition]) + .await + .inspect_err(|err| error!(?err))?; + } if let Some(configs) = topic.configs { let prepared = tx - .prepare(concat!( - "insert into topic_configuration", - " (topic, name, value)", - " values ($1, $2, $3)" - )) - .await?; + .prepare(include_str!("pg/topic_configuration_insert.sql")) + .await + .inspect_err(|err| error!(?err))?; for config in configs { debug!(?config); @@ -416,7 +349,12 @@ impl Storage for Postgres { _ = tx .execute( &prepared, - &[&topic_id, &config.name.as_str(), &config.value.as_deref()], + &[ + &self.cluster, + &topic.name, + &config.name, + &config.value.as_deref(), + ], ) .await .inspect_err(|err| error!(?err, ?config)); @@ -425,11 +363,11 @@ impl Storage for Postgres { tx.commit().await.inspect_err(|err| error!(?err))?; - Ok(topic_id) + Ok(topic_uuid) } async fn delete_records( - &self, + &mut self, topics: &[DeleteRecordsTopic], ) -> Result> { let c = self.connection().await?; @@ -556,84 +494,169 @@ impl Storage for Postgres { Ok(responses) } - async fn delete_topic(&self, topic: &TopicId) -> Result { + async fn delete_topic(&mut self, topic: &TopicId) -> Result { let mut c = self.connection().await?; let tx = c.transaction().await?; + let row = match topic { + TopicId::Id(id) => { + let prepared = tx + .prepare(include_str!("pg/topic_select_uuid.sql")) + .await + .inspect_err(|err| error!(?err))?; + + tx.query_opt(&prepared, &[&self.cluster, &id]) + .await + .inspect_err(|err| error!(?err))? + } + + TopicId::Name(name) => { + let prepared = tx + .prepare(include_str!("pg/topic_select_name.sql")) + .await + .inspect_err(|err| error!(?err))?; + + tx.query_opt(&prepared, &[&self.cluster, &name]) + .await + .inspect_err(|err| error!(?err))? + } + }; + + let Some(row) = row else { + return Ok(ErrorCode::UnknownTopicOrPartition); + }; + + let topic_name = row.try_get::<_, String>(1)?; + for (description, sql) in [ - ("consumer offsets", DELETE_CONSUMER_OFFSETS_FOR_TOPIC), - ("headers", DELETE_HEADERS_FOR_TOPIC), - ("records", DELETE_RECORDS_FOR_TOPIC), + ( + "consumer offsets", + include_str!("pg/consumer_offset_delete_by_topic.sql"), + ), + ( + "watermarks", + include_str!("pg/watermark_delete_by_topic.sql"), + ), + ("headers", include_str!("pg/header_delete_by_topic.sql")), + ("records", include_str!("pg/record_delete_by_topic.sql")), + ("topitions", include_str!("pg/topition_delete_by_topic.sql")), ] { - let rows = self.delete_for_topic(&tx, sql, topic).await?; + let prepared = tx.prepare(sql).await.inspect_err(|err| error!(?err))?; + + let rows = tx + .execute(&prepared, &[&self.cluster, &topic_name]) + .await + .inspect_err(|err| error!(?err))?; + debug!(?topic, ?rows, ?description); } - let topic_deletion_result = - self.delete_for_topic(&tx, DELETE_TOPIC, topic) - .await - .map(|rows| match rows { - 0 => ErrorCode::UnknownTopicOrPartition, - 1 => ErrorCode::None, - otherwise => { - error!(?otherwise, ?DELETE_TOPIC, ?topic, ?self.cluster); - ErrorCode::None - } - }); + let prepared = tx + .prepare(include_str!("pg/topic_delete_by.sql")) + .await + .inspect_err(|err| error!(?err))?; - tx.commit() + _ = tx + .execute(&prepared, &[&self.cluster, &topic_name]) .await - .inspect_err(|err| error!(?err)) - .map_err(Into::into) - .and(topic_deletion_result) + .inspect_err(|err| error!(?err))?; + + tx.commit().await.inspect_err(|err| error!(?err))?; + + Ok(ErrorCode::None) } - async fn produce(&self, topition: &'_ Topition, deflated: deflated::Batch) -> Result { - debug!(?topition, ?deflated); + async fn produce( + &mut self, + transaction_id: Option<&str>, + topition: &Topition, + deflated: deflated::Batch, + ) -> Result { + debug!(?transaction_id, ?topition, ?deflated); + + let topic = topition.topic(); + let partition = topition.partition(); + let mut c = self.connection().await?; let tx = c.transaction().await?; + let prepared = tx + .prepare(include_str!("pg/txn_topition_select_txns.sql")) + .await + .inspect_err(|err| error!(?err))?; + + let topition_is_in_a_txn = tx + .query_one( + &prepared, + &[ + &self.cluster, + &deflated.producer_id, + &(deflated.producer_epoch as i32), + &topic, + &partition, + ], + ) + .await + .and_then(|row| row.try_get::<_, i64>(0)) + .inspect_err(|err| error!(?err)) + .inspect(|in_progress_transactions| debug!(?in_progress_transactions)) + .is_ok_and(|in_progress_transactions| in_progress_transactions > 0); + debug!(?topition_is_in_a_txn); + + let (low, high, stable) = { + let prepared = tx + .prepare(include_str!("pg/watermark_select_for_update.sql")) + .await + .inspect_err(|err| error!(?err))?; + + let row = tx + .query_one(&prepared, &[&self.cluster, &topic, &partition]) + .await + .inspect_err(|err| error!(?err))?; + + ( + row.try_get::<_, Option>(0) + .inspect_err(|err| error!(?err))?, + row.try_get::<_, Option>(1) + .inspect_err(|err| error!(?err))?, + row.try_get::<_, Option>(2) + .inspect_err(|err| error!(?err))?, + ) + }; + + debug!(?low, ?high, ?stable); + let insert_record = tx - .prepare(concat!( - "insert into record", - " (topic, partition, producer_id, sequence, timestamp, k, v)", - " select", - " topic.id, $2, $3, $4, $5, $6, $7", - " from topic", - " where topic.name = $1", - " returning id" - )) + .prepare(include_str!("pg/record_insert.sql")) .await .inspect_err(|err| error!(?err))?; let insert_header = tx - .prepare(concat!( - "insert into header", - " (record, k, v)", - " values", - " ($1, $2, $3)", - )) + .prepare(include_str!("pg/header_insert.sql")) .await .inspect_err(|err| error!(?err))?; let inflated = inflated::Batch::try_from(deflated).inspect_err(|err| error!(?err))?; - let mut offsets = vec![]; - for record in inflated.records { - debug!(?record); + let records_len = i64::try_from(inflated.records.len())?; - let topic = topition.topic(); - let partition = topition.partition(); + for (delta, record) in inflated.records.iter().enumerate() { + let delta = i64::try_from(delta)?; + let offset = high.map_or(delta, |high| high + delta + 1); let key = record.key.as_deref(); let value = record.value.as_deref(); - let row = tx - .query_one( + debug!(?delta, ?record, ?offset); + + _ = tx + .execute( &insert_record, &[ + &self.cluster, &topic, &partition, + &offset, &inflated.producer_id, &inflated.base_sequence, &(to_system_time(inflated.base_timestamp + record.timestamp_delta)?), @@ -642,67 +665,106 @@ impl Storage for Postgres { ], ) .await - .inspect_err(|err| error!(?err, ?topic, ?partition, ?key, ?value))?; + .inspect_err(|err| error!(?err, ?topic, ?partition, ?offset, ?key, ?value))?; - let offset: i64 = row.get(0); - debug!(?offset); - - for header in record.headers { + for header in record.headers.iter().as_ref() { let key = header.key.as_deref(); let value = header.value.as_deref(); _ = tx - .execute(&insert_header, &[&offset, &key, &value]) + .execute( + &insert_header, + &[&self.cluster, &topic, &partition, &offset, &key, &value], + ) .await .inspect_err(|err| { error!(?err, ?topic, ?partition, ?offset, ?key, ?value); }); } + } - offsets.push(offset); + if let Some(transaction_id) = transaction_id { + let txn_produce_offset_insert = tx + .prepare(include_str!("pg/txn_produce_offset_insert.sql")) + .await + .inspect_err(|err| error!(?err))?; + + _ = tx + .execute( + &txn_produce_offset_insert, + &[ + &self.cluster, + &transaction_id, + &inflated.producer_id, + &(inflated.producer_epoch as i32), + &topic, + &partition, + &high.unwrap_or_default(), + &high.map_or(records_len - 1, |high| high + records_len), + ], + ) + .await + .inspect_err(|err| error!(?err))?; } + let prepared = tx + .prepare(include_str!("pg/watermark_update.sql")) + .await + .inspect_err(|err| error!(?err))?; + + let stable = if topition_is_in_a_txn { + stable.unwrap_or_default() + } else { + stable.map_or(records_len - 1, |high| high + records_len) + }; + + _ = tx + .execute( + &prepared, + &[ + &self.cluster, + &topic, + &partition, + &low.unwrap_or_default(), + &high.map_or(records_len - 1, |high| high + records_len), + &stable, + ], + ) + .await + .inspect_err(|err| error!(?err))?; + tx.commit().await?; - Ok(offsets.first().copied().unwrap_or(-1)) + Ok(high.map_or(0, |high| high + 1)) } async fn fetch( - &self, + &mut self, topition: &Topition, offset: i64, min_bytes: u32, max_bytes: u32, + isolation: IsolationLevel, ) -> Result { - debug!(?topition, ?offset); + debug!(?topition, ?offset, ?isolation); + + let high_watermark = self.offset_stage(topition).await.map(|offset_stage| { + if isolation == IsolationLevel::ReadCommitted { + offset_stage.last_stable + } else { + offset_stage.high_watermark + } + })?; + let c = self.connection().await?; let select_batch = c - .prepare(concat!( - "with sized as (", - " select", - " record.id", - ", timestamp", - ", k", - ", v", - ", sum(coalesce(length(k), 0) + coalesce(length(v), 0))", - " over (order by record.id) as bytes", - " from cluster, record, topic", - " where", - " cluster.name = $1", - " and topic.name = $2", - " and record.partition = $3", - " and record.id >= $4", - " and topic.cluster = cluster.id", - " and record.topic = topic.id", - ") select * from sized", - " where bytes < $5", - )) + .prepare(include_str!("pg/record_fetch.sql")) .await .inspect_err(|err| error!(?err))?; let select_headers = c - .prepare(concat!("select k, v from header where record = $1")) + .prepare(include_str!("pg/header_fetch.sql")) .await .inspect_err(|err| error!(?err))?; @@ -715,9 +777,12 @@ impl Storage for Postgres { &topition.partition(), &offset, &(max_bytes as i64), + &high_watermark, ], ) - .await?; + .await + .inspect(|rows| debug!(?rows)) + .inspect_err(|err| error!(?err))?; let mut bytes = 0; @@ -735,6 +800,8 @@ impl Storage for Postgres { .base_timestamp(base_timestamp); for record in records.iter() { + debug!(?record); + let offset = record.try_get::<_, i64>(0)?; let offset_delta = i32::try_from(offset - base_offset)?; @@ -745,15 +812,18 @@ impl Storage for Postgres { to_timestamp(system_time) .map(|timestamp| timestamp - base_timestamp) .map_err(Into::into) - })?; + }) + .inspect(|timestamp| debug!(?timestamp))?; let k = record .try_get::<_, Option<&[u8]>>(2) - .map(|o| o.map(Bytes::copy_from_slice))?; + .map(|o| o.map(Bytes::copy_from_slice)) + .inspect(|k| debug!(?k))?; let v = record .try_get::<_, Option<&[u8]>>(3) - .map(|o| o.map(Bytes::copy_from_slice))?; + .map(|o| o.map(Bytes::copy_from_slice)) + .inspect(|v| debug!(?v))?; bytes += record.try_get::<_, i64>(4)?; @@ -763,7 +833,20 @@ impl Storage for Postgres { .key(k.into()) .value(v.into()); - for header in c.query(&select_headers, &[&offset]).await? { + for header in c + .query( + &select_headers, + &[ + &self.cluster, + &topition.topic(), + &topition.partition(), + &offset, + ], + ) + .await + .inspect(|row| debug!(?row)) + .inspect_err(|err| error!(?err))? + { let mut header_builder = Header::builder(); if let Some(k) = header.try_get::<_, Option<&[u8]>>(0)? { @@ -799,23 +882,14 @@ impl Storage for Postgres { .map_err(Into::into) } - async fn offset_stage(&self, topition: &'_ Topition) -> Result { + async fn offset_stage(&mut self, topition: &Topition) -> Result { + debug!(?topition); let c = self.connection().await?; let prepared = c - .prepare(concat!( - "select", - " coalesce(min(record.id), (select last_value from record_id_seq)) as log_start", - ", coalesce(max(record.id), (select last_value from record_id_seq)) as high_watermark", - " from cluster, record, topic", - " where", - " cluster.name = $1", - " and topic.name = $2", - " and record.partition = $3", - " and topic.cluster = cluster.id", - " and record.topic = topic.id", - )) - .await?; + .prepare(include_str!("pg/watermark_select.sql")) + .await + .inspect_err(|err| error!(?topition, ?err))?; let row = c .query_one( @@ -826,14 +900,19 @@ impl Storage for Postgres { .inspect_err(|err| error!(?topition, ?prepared, ?err))?; let log_start = row - .try_get::<_, i64>(0) - .inspect_err(|err| error!(?topition, ?prepared, ?err))?; + .try_get::<_, Option>(0) + .inspect_err(|err| error!(?topition, ?prepared, ?err))? + .unwrap_or_default(); let high_watermark = row - .try_get::<_, i64>(0) - .inspect_err(|err| error!(?topition, ?prepared, ?err))?; + .try_get::<_, Option>(1) + .inspect_err(|err| error!(?topition, ?prepared, ?err))? + .unwrap_or_default(); - let last_stable = high_watermark; + let last_stable = row + .try_get::<_, Option>(2) + .inspect_err(|err| error!(?topition, ?prepared, ?err))? + .unwrap_or_default(); Ok(OffsetStage { last_stable, @@ -843,7 +922,7 @@ impl Storage for Postgres { } async fn offset_commit( - &self, + &mut self, group: &str, retention: Option, offsets: &[(Topition, OffsetCommitRequest)], @@ -855,21 +934,9 @@ impl Storage for Postgres { let tx = c.transaction().await?; let prepared = tx - .prepare(concat!( - "insert into consumer_offset ", - " (grp, topic, partition, committed_offset, leader_epoch, timestamp, metadata) ", - " select", - " $1, topic.id, $3, $4, $5, $6, $7 ", - " from topic", - " where topic.name = $2", - " on conflict (grp, topic, partition)", - " do update set", - " committed_offset = excluded.committed_offset,", - " leader_epoch = excluded.leader_epoch,", - " timestamp = excluded.timestamp,", - " metadata = excluded.metadata", - )) - .await?; + .prepare(include_str!("pg/consumer_offset_insert.sql")) + .await + .inspect_err(|err| error!(?err))?; let mut responses = vec![]; @@ -878,27 +945,29 @@ impl Storage for Postgres { .execute( &prepared, &[ - &group, + &self.cluster, &topition.topic(), &topition.partition(), + &group, &offset.offset, &offset.leader_epoch, &offset.timestamp, &offset.metadata, ], ) - .await?; + .await + .inspect_err(|err| error!(?err))?; responses.push((topition.to_owned(), ErrorCode::None)); } - tx.commit().await?; + tx.commit().await.inspect_err(|err| error!(?err))?; Ok(responses) } async fn offset_fetch( - &self, + &mut self, group_id: Option<&str>, topics: &[Topition], require_stable: Option, @@ -910,23 +979,20 @@ impl Storage for Postgres { let c = self.connection().await?; let prepared = c - .prepare(concat!( - "select", - " committed_offset", - " from consumer_offset, topic", - " where grp=$1", - " and topic.name=$2", - " and consumer_offset.topic = topic.id", - " and partition=$3", - )) - .await?; + .prepare(include_str!("pg/consumer_offset_select.sql")) + .await + .inspect_err(|err| error!(?err))?; let mut offsets = BTreeMap::new(); for topic in topics { let offset = c - .query_opt(&prepared, &[&group_id, &topic.topic(), &topic.partition()]) - .await? + .query_opt( + &prepared, + &[&self.cluster, &group_id, &topic.topic(), &topic.partition()], + ) + .await + .inspect_err(|err| error!(?err))? .map_or(Ok(-1), |row| row.try_get::<_, i64>(0))?; _ = offsets.insert(topic.to_owned(), offset); @@ -936,7 +1002,8 @@ impl Storage for Postgres { } async fn list_offsets( - &self, + &mut self, + isolation_level: IsolationLevel, offsets: &[(Topition, ListOffsetRequest)], ) -> Result> { debug!(?offsets); @@ -946,43 +1013,17 @@ impl Storage for Postgres { let mut responses = vec![]; for (topition, offset_type) in offsets { - let query = match offset_type { - ListOffsetRequest::Earliest => include_str!("pg/list_earliest_offset.sql"), - ListOffsetRequest::Latest => concat!( - "select", - " id as offset, timestamp", - " from", - " record", - " join (", - " select", - " coalesce(max(record.id), (select last_value from record_id_seq)) as offset", - " from record, topic, cluster", - " where", - " topic.cluster = cluster.id", - " and cluster.name = $1", - " and topic.name = $2", - " and record.partition = $3", - " and record.topic = topic.id) as maximum", - " on record.id = maximum.offset", - ), - ListOffsetRequest::Timestamp(_) => concat!( - "select", - " id as offset, timestamp", - " from", - " record", - " join (", - " select", - " coalesce(min(record.id), (select last_value from record_id_seq)) as offset", - " from record, topic, cluster", - " where", - " topic.cluster = cluster.id", - " and cluster.name = $1", - " and topic.name = $2", - " and record.partition = $3", - " and record.timestamp >= $4", - " and record.topic = topic.id) as minimum", - " on record.id = minimum.offset", - ), + let query = match (offset_type, isolation_level) { + (ListOffsetRequest::Earliest, _) => include_str!("pg/list_earliest_offset.sql"), + (ListOffsetRequest::Latest, IsolationLevel::ReadCommitted) => { + include_str!("pg/list_latest_offset_committed.sql") + } + (ListOffsetRequest::Latest, IsolationLevel::ReadUncommitted) => { + include_str!("pg/list_latest_offset_uncommitted.sql") + } + (ListOffsetRequest::Timestamp(_), _) => { + include_str!("pg/list_latest_offset_timestamp.sql") + } }; debug!(?query); @@ -994,15 +1035,16 @@ impl Storage for Postgres { .inspect(|prepared| debug!(?prepared))?; let list_offset = match offset_type { - ListOffsetRequest::Earliest | ListOffsetRequest::Latest => { - c.query_opt( + ListOffsetRequest::Earliest | ListOffsetRequest::Latest => c + .query_opt( &prepared, &[&self.cluster, &topition.topic(), &topition.partition()], ) .await - } - ListOffsetRequest::Timestamp(timestamp) => { - c.query_opt( + .inspect_err(|err| error!(?err, ?self.cluster, ?topition)), + + ListOffsetRequest::Timestamp(timestamp) => c + .query_opt( &prepared, &[ &self.cluster.as_str(), @@ -1012,7 +1054,7 @@ impl Storage for Postgres { ], ) .await - } + .inspect_err(|err| error!(?err)), } .inspect_err(|err| { let cluster = self.cluster.as_str(); @@ -1051,20 +1093,13 @@ impl Storage for Postgres { Ok(responses).inspect(|r| debug!(?r)) } - async fn metadata(&self, topics: Option<&[TopicId]>) -> Result { + async fn metadata(&mut self, topics: Option<&[TopicId]>) -> Result { debug!(?topics); let c = self.connection().await.inspect_err(|err| error!(?err))?; let prepared = c - .prepare(concat!( - "select node, host, port, rack", - " from broker, cluster, listener", - " where cluster.name = $1", - " and broker.cluster = cluster.id", - " and listener.broker = broker.id", - " and listener.name = 'broker'" - )) + .prepare(include_str!("pg/broker_metadata_select.sql")) .await .inspect_err(|err| error!(?err))?; @@ -1099,21 +1134,17 @@ impl Storage for Postgres { responses.push(match topic { TopicId::Name(name) => { let prepared = c - .prepare(concat!( - "select", - " topic.id, topic.name, is_internal, partitions, replication_factor", - " from topic, cluster", - " where cluster.name = $1", - " and topic.name = $2", - " and topic.cluster = cluster.id", - )) + .prepare(include_str!("pg/topic_select_name.sql")) .await - .inspect_err(|err|{let cluster = self.cluster.as_str();error!(?err, ?cluster, ?name);})?; + .inspect_err(|err| { + let cluster = self.cluster.as_str(); + error!(?err, ?cluster, ?name); + })?; match c .query_opt(&prepared, &[&self.cluster, &name.as_str()]) .await - .inspect_err(|err|error!(?err)) + .inspect_err(|err| error!(?err)) { Ok(Some(row)) => { let error_code = ErrorCode::None.into(); @@ -1126,31 +1157,48 @@ impl Storage for Postgres { let partitions = row.try_get::<_, i32>(3)?; let replication_factor = row.try_get::<_, i32>(4)?; - debug!(?error_code, ?topic_id, ?name, ?is_internal, ?partitions, ?replication_factor); + debug!( + ?error_code, + ?topic_id, + ?name, + ?is_internal, + ?partitions, + ?replication_factor + ); let mut rng = thread_rng(); - let mut broker_ids:Vec<_> = brokers.iter().map(|broker|broker.node_id).collect(); + let mut broker_ids: Vec<_> = + brokers.iter().map(|broker| broker.node_id).collect(); broker_ids.shuffle(&mut rng); let mut brokers = broker_ids.into_iter().cycle(); - let partitions = Some((0..partitions).map(|partition_index| { - let leader_id = brokers.next().expect("cycling"); - - let replica_nodes = Some((0..replication_factor).map(|_replica|brokers.next().expect("cycling")).collect()); - let isr_nodes = replica_nodes.clone(); - - MetadataResponsePartition { - error_code, - partition_index, - leader_id, - leader_epoch: Some(-1), - replica_nodes, - isr_nodes, - offline_replicas: Some([].into()), - } - }).collect()); - + let partitions = Some( + (0..partitions) + .map(|partition_index| { + let leader_id = brokers.next().expect("cycling"); + + let replica_nodes = Some( + (0..replication_factor) + .map(|_replica| { + brokers.next().expect("cycling") + }) + .collect(), + ); + let isr_nodes = replica_nodes.clone(); + + MetadataResponsePartition { + error_code, + partition_index, + leader_id, + leader_epoch: Some(-1), + replica_nodes, + isr_nodes, + offline_replicas: Some([].into()), + } + }) + .collect(), + ); MetadataResponseTopic { error_code, @@ -1162,16 +1210,14 @@ impl Storage for Postgres { } } - Ok(None) => { - MetadataResponseTopic { - error_code: ErrorCode::UnknownTopicOrPartition.into(), - name: Some(name.into()), - topic_id: Some(NULL_TOPIC_ID), - is_internal: Some(false), - partitions: Some([].into()), - topic_authorized_operations: Some(-2147483648), - } - } + Ok(None) => MetadataResponseTopic { + error_code: ErrorCode::UnknownTopicOrPartition.into(), + name: Some(name.into()), + topic_id: Some(NULL_TOPIC_ID), + is_internal: Some(false), + partitions: Some([].into()), + topic_authorized_operations: Some(-2147483648), + }, Err(reason) => { debug!(?reason); @@ -1189,21 +1235,11 @@ impl Storage for Postgres { TopicId::Id(id) => { debug!(?id); let prepared = c - .prepare(concat!( - "select", - " topic.id, topic.name, is_internal, partitions, replication_factor", - " from topic, cluster", - " where cluster.name = $1", - " and topic.id = $2", - " and topic.cluster = cluster.id", - )) + .prepare(include_str!("pg/topic_select_uuid.sql")) .await - .inspect_err(|error|error!(?error))?; + .inspect_err(|error| error!(?error))?; - match c - .query_one(&prepared, &[&self.cluster, &id]) - .await - { + match c.query_one(&prepared, &[&self.cluster, &id]).await { Ok(row) => { let error_code = ErrorCode::None.into(); let topic_id = row @@ -1215,31 +1251,48 @@ impl Storage for Postgres { let partitions = row.try_get::<_, i32>(3)?; let replication_factor = row.try_get::<_, i32>(4)?; - debug!(?error_code, ?topic_id, ?name, ?is_internal, ?partitions, ?replication_factor); + debug!( + ?error_code, + ?topic_id, + ?name, + ?is_internal, + ?partitions, + ?replication_factor + ); let mut rng = thread_rng(); - let mut broker_ids:Vec<_> = brokers.iter().map(|broker|broker.node_id).collect(); + let mut broker_ids: Vec<_> = + brokers.iter().map(|broker| broker.node_id).collect(); broker_ids.shuffle(&mut rng); let mut brokers = broker_ids.into_iter().cycle(); - let partitions = Some((0..partitions).map(|partition_index| { - let leader_id = brokers.next().expect("cycling"); - - let replica_nodes = Some((0..replication_factor).map(|_replica|brokers.next().expect("cycling")).collect()); - let isr_nodes = replica_nodes.clone(); - - MetadataResponsePartition { - error_code, - partition_index, - leader_id, - leader_epoch: Some(-1), - replica_nodes, - isr_nodes, - offline_replicas: Some([].into()), - } - }).collect()); - + let partitions = Some( + (0..partitions) + .map(|partition_index| { + let leader_id = brokers.next().expect("cycling"); + + let replica_nodes = Some( + (0..replication_factor) + .map(|_replica| { + brokers.next().expect("cycling") + }) + .collect(), + ); + let isr_nodes = replica_nodes.clone(); + + MetadataResponsePartition { + error_code, + partition_index, + leader_id, + leader_epoch: Some(-1), + replica_nodes, + isr_nodes, + offline_replicas: Some([].into()), + } + }) + .collect(), + ); MetadataResponseTopic { error_code, @@ -1273,13 +1326,7 @@ impl Storage for Postgres { let mut responses = vec![]; let prepared = c - .prepare(concat!( - "select", - " topic.id, topic.name, is_internal, partitions, replication_factor", - " from topic, cluster", - " where cluster.name = $1", - " and topic.cluster = cluster.id", - )) + .prepare(include_str!("pg/topic_by_cluster.sql")) .await .inspect_err(|err| error!(?err))?; @@ -1377,7 +1424,7 @@ impl Storage for Postgres { } async fn describe_config( - &self, + &mut self, name: &str, resource: ConfigResource, keys: Option<&[String]>, @@ -1463,27 +1510,18 @@ impl Storage for Postgres { } async fn update_group( - &self, + &mut self, group_id: &str, detail: GroupDetail, version: Option, ) -> Result> { + debug!(?group_id, ?detail, ?version); + let mut c = self.connection().await?; let tx = c.transaction().await?; let prepared = tx - .prepare(concat!( - "insert into consumer_group", - " (grp, cluster, e_tag, detail)", - " select $1, cluster.id, $3, $4", - " from cluster", - " where cluster.name = $2", - " on conflict (grp, cluster)", - " do update set", - " detail = excluded.detail, e_tag = $5", - " where consumer_group.e_tag = $3", - " returning grp, cluster, e_tag, detail", - )) + .prepare(include_str!("pg/consumer_group_insert.sql")) .await .inspect_err(|err| error!(?err))?; @@ -1497,21 +1535,23 @@ impl Storage for Postgres { Uuid::from_str(e_tag.as_str()).map_err(Into::into) }) }) - .inspect_err(|err| error!(?err))?; + .inspect_err(|err| error!(?err)) + .inspect(|existing_e_tag| debug!(?existing_e_tag))?; let new_e_tag = Uuid::new_v4(); + debug!(?new_e_tag); - let value = serde_json::to_value(detail)?; + let value = serde_json::to_value(detail).inspect(|value| debug!(?value))?; let outcome = if let Some(row) = tx .query_opt( &prepared, &[ + &self.cluster, &group_id, - &self.cluster.as_str(), &existing_e_tag, - &value, &new_e_tag, + &value, ], ) .await @@ -1527,17 +1567,10 @@ impl Storage for Postgres { e_tag, version: None, }) + .inspect(|version| debug!(?version)) } else { let prepared = tx - .prepare(concat!( - "select", - " cg.e_tag, cg.detail", - " from cluster c, consumer_group cg", - " where", - " cg.grp = $1", - " and c.name = $2", - " and c.id = cg.cluster" - )) + .prepare(include_str!("pg/consumer_group_detail.sql")) .await .inspect_err(|err| error!(?err))?; @@ -1555,7 +1588,8 @@ impl Storage for Postgres { .map(|e_tag| Version { e_tag, version: None, - })?; + }) + .inspect(|version| debug!(?version))?; let value = row.try_get::<_, Value>(1)?; let current = serde_json::from_value::(value)?; @@ -1565,6 +1599,295 @@ impl Storage for Postgres { tx.commit().await.inspect_err(|err| error!(?err))?; + debug!(?outcome); + outcome } + + async fn init_producer( + &mut self, + transaction_id: Option<&str>, + transaction_timeout_ms: i32, + producer_id: Option, + producer_epoch: Option, + ) -> Result { + let Some(producer_id) = producer_id else { + return Ok(ProducerIdResponse { + error: ErrorCode::UnknownProducerId, + id: -1, + epoch: -1, + }); + }; + + let Some(producer_epoch) = producer_epoch else { + return Ok(ProducerIdResponse { + error: ErrorCode::UnknownProducerId, + id: -1, + epoch: -1, + }); + }; + + let mut c = self.connection().await.inspect_err(|err| error!(?err))?; + let tx = c.transaction().await?; + + let row = if producer_id == -1 && producer_epoch == -1 { + let prepared = tx + .prepare(include_str!("pg/init_producer.sql")) + .await + .inspect_err(|err| error!(?err))?; + + debug!(?prepared); + + tx.query_one(&prepared, &[&self.cluster]) + .await + .inspect_err(|err| error!(?err))? + } else { + let prepared = tx + .prepare(include_str!("pg/init_txn_producer.sql")) + .await + .inspect_err(|err| error!(?err))?; + + debug!(?prepared); + + tx.query_one(&prepared, &[&self.cluster, &producer_id, &producer_epoch]) + .await + .inspect_err(|err| error!(?err))? + }; + + let id = row.get(0); + let epoch: i32 = row.get(1); + + if let Some(transaction_id) = transaction_id { + let prepared = tx + .prepare(include_str!("pg/init_txn.sql")) + .await + .inspect_err(|err| error!(?err))?; + + debug!(?prepared); + + _ = tx + .execute( + &prepared, + &[&self.cluster, &transaction_id, &transaction_timeout_ms, &id], + ) + .await; + } + + tx.commit().await.inspect_err(|err| error!(?err))?; + + i16::try_from(epoch) + .map(|epoch| ProducerIdResponse { + error: ErrorCode::None, + id, + epoch, + }) + .map_err(Into::into) + } + + async fn txn_add_offsets( + &mut self, + transaction_id: &str, + producer_id: i64, + producer_epoch: i16, + group_id: &str, + ) -> Result { + debug!(?transaction_id, ?producer_id, ?producer_epoch, ?group_id); + + Ok(ErrorCode::None) + } + + async fn txn_add_partitions( + &mut self, + partitions: TxnAddPartitionsRequest, + ) -> Result { + debug!(?partitions); + + match partitions { + TxnAddPartitionsRequest::VersionZeroToThree { + transaction_id, + producer_id, + producer_epoch, + topics, + } => { + debug!(?transaction_id, ?producer_id, ?producer_epoch, ?topics); + + let mut c = self.connection().await.inspect_err(|err| error!(?err))?; + let tx = c.transaction().await.inspect_err(|err| error!(?err))?; + + let prepared = tx + .prepare(include_str!("pg/txn_topition_insert.sql")) + .await + .inspect_err(|err| error!(?err))?; + + let mut results = vec![]; + + for topic in topics { + let mut results_by_partition = vec![]; + + for partition_index in topic.partitions.unwrap_or(vec![]) { + _ = tx + .execute( + &prepared, + &[ + &self.cluster, + &topic.name, + &partition_index, + &transaction_id, + ], + ) + .await + .inspect_err(|err| error!(?err))?; + + results_by_partition.push(AddPartitionsToTxnPartitionResult { + partition_index, + partition_error_code: i16::from(ErrorCode::None), + }); + } + + results.push(AddPartitionsToTxnTopicResult { + name: topic.name, + results_by_partition: Some(results_by_partition), + }) + } + + tx.commit().await?; + + Ok(TxnAddPartitionsResponse::VersionZeroToThree(results)) + } + + TxnAddPartitionsRequest::VersionFourPlus { .. } => { + todo!() + } + } + } + + async fn txn_offset_commit( + &mut self, + offsets: TxnOffsetCommitRequest, + ) -> Result> { + debug!(?offsets); + + let mut c = self.connection().await.inspect_err(|err| error!(?err))?; + let tx = c.transaction().await.inspect_err(|err| error!(?err))?; + + let prepared = tx + .prepare(include_str!("pg/txn_offset_commit_insert.sql")) + .await + .inspect_err(|err| error!(?err))?; + + _ = tx + .execute( + &prepared, + &[ + &self.cluster, + &offsets.transaction_id, + &offsets.group_id, + &offsets.producer_id, + &(offsets.producer_epoch as i32), + &offsets.generation_id, + &offsets.member_id, + ], + ) + .await + .inspect_err(|err| error!(?err))?; + + let prepared = tx + .prepare(include_str!("pg/txn_offset_commit_tp_insert.sql")) + .await + .inspect_err(|err| error!(?err))?; + + let mut topics = vec![]; + + for topic in offsets.topics { + let mut partitions = vec![]; + + for partition in topic.partitions.unwrap_or(vec![]) { + _ = tx + .execute( + &prepared, + &[ + &self.cluster, + &offsets.transaction_id, + &offsets.group_id, + &offsets.producer_id, + &(offsets.producer_epoch as i32), + &topic.name, + &partition.partition_index, + &partition.committed_offset, + &partition.committed_leader_epoch, + &partition.committed_metadata, + ], + ) + .await + .inspect_err(|err| error!(?err))?; + + partitions.push(TxnOffsetCommitResponsePartition { + partition_index: partition.partition_index, + error_code: i16::from(ErrorCode::None), + }); + } + + topics.push(TxnOffsetCommitResponseTopic { + name: topic.name, + partitions: Some(partitions), + }); + } + + tx.commit().await?; + + Ok(topics) + } + + async fn txn_end( + &mut self, + transaction_id: &str, + producer_id: i64, + producer_epoch: i16, + committed: bool, + ) -> Result { + debug!(?transaction_id, ?producer_id, ?producer_epoch, ?committed); + + let mut c = self.connection().await.inspect_err(|err| error!(?err))?; + let tx = c.transaction().await.inspect_err(|err| error!(?err))?; + + let prepared = tx + .prepare(include_str!("pg/watermark_insert_from_txn.sql")) + .await + .inspect_err(|err| error!(?err))?; + + _ = tx + .execute( + &prepared, + &[ + &self.cluster, + &transaction_id, + &producer_id, + &(producer_epoch as i32), + ], + ) + .await + .inspect_err(|err| error!(?err))?; + + let prepared = tx + .prepare(include_str!("pg/consumer_offset_insert_from_txn.sql")) + .await + .inspect_err(|err| error!(?err))?; + + _ = tx + .execute( + &prepared, + &[ + &self.cluster, + &transaction_id, + &producer_id, + &(producer_epoch as i32), + ], + ) + .await + .inspect_err(|err| error!(?err))?; + + tx.commit().await?; + + Ok(ErrorCode::None) + } } diff --git a/tansu-storage/src/pg/broker_metadata_select.sql b/tansu-storage/src/pg/broker_metadata_select.sql new file mode 100644 index 0000000..ebb2beb --- /dev/null +++ b/tansu-storage/src/pg/broker_metadata_select.sql @@ -0,0 +1,22 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +select b.node, l.host, l.port, b.rack +from broker b, cluster c, listener l +where c.name = $1 +and b.cluster = c.id +and l.broker = b.id +and l.name = 'broker'; diff --git a/tansu-storage/src/pg/consumer_group_detail.sql b/tansu-storage/src/pg/consumer_group_detail.sql new file mode 100644 index 0000000..bedde8e --- /dev/null +++ b/tansu-storage/src/pg/consumer_group_detail.sql @@ -0,0 +1,21 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +select cg.e_tag, cg.detail +from cluster c, consumer_group cg +where cg.name = $1 +and c.name = $2 +and cg.cluster = c.id; diff --git a/tansu-storage/src/pg/consumer_group_insert.sql b/tansu-storage/src/pg/consumer_group_insert.sql new file mode 100644 index 0000000..92fba89 --- /dev/null +++ b/tansu-storage/src/pg/consumer_group_insert.sql @@ -0,0 +1,28 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +-- prepare cg_update (text, text, uuid, uuid, json) as +insert into consumer_group +(cluster, name, e_tag, detail) +select c.id, $2, $4, $5 +from cluster c +where c.name = $1 +on conflict (cluster, name) +do update set +detail = excluded.detail, +e_tag = $4 +where consumer_group.e_tag = $3 +returning name, cluster, e_tag, detail; diff --git a/tansu-storage/src/pg/consumer_offset_delete_by_topic.sql b/tansu-storage/src/pg/consumer_offset_delete_by_topic.sql new file mode 100644 index 0000000..c3cdc47 --- /dev/null +++ b/tansu-storage/src/pg/consumer_offset_delete_by_topic.sql @@ -0,0 +1,24 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +delete from consumer_offset +using cluster c,consumer_group cg, topic t, topition tp +where c.name = $1 +and t.name = $2 +and cg.cluster = c.id +and tp.topic = t.id +and consumer_offset.topition = tp.id +and consumer_offset.consumer_group = cg.id diff --git a/tansu-storage/src/pg/consumer_offset_insert.sql b/tansu-storage/src/pg/consumer_offset_insert.sql new file mode 100644 index 0000000..078c8a3 --- /dev/null +++ b/tansu-storage/src/pg/consumer_offset_insert.sql @@ -0,0 +1,33 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into consumer_offset +(consumer_group, topition, committed_offset, leader_epoch, timestamp, metadata) +select cg.id, tp.id, $5, $6, $7, $8 +from cluster c, consumer_group cg, topic t, topition tp +where c.name = $1 +and t.name = $2 +and tp.partition = $3 +and cg.name = $4 +and t.cluster = c.id +and tp.topic = t.id +and cg.cluster = c.id +on conflict (consumer_group, topition) +do update set +committed_offset = excluded.committed_offset, +leader_epoch = excluded.leader_epoch, +timestamp = excluded.timestamp, +metadata = excluded.metadata; diff --git a/tansu-storage/src/pg/consumer_offset_insert_from_txn.sql b/tansu-storage/src/pg/consumer_offset_insert_from_txn.sql new file mode 100644 index 0000000..3cdcaa7 --- /dev/null +++ b/tansu-storage/src/pg/consumer_offset_insert_from_txn.sql @@ -0,0 +1,72 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into consumer_offset +(consumer_group, +topition, +committed_offset, +leader_epoch, +timestamp, +metadata) + +select + +cg.id, +tp.id, +txn_oc_tp.committed_offset, +txn_oc_tp.leader_epoch, +txn_oc_tp.created_at, +txn_oc_tp.metadata + +from + +cluster c, +consumer_group cg, +producer p, +topic t, +topition tp, +txn, +txn_offset_commit txn_oc, +txn_offset_commit_tp txn_oc_tp + +where + +c.name = $1 +and txn.name = $2 +and p.id = $3 +and p.epoch = $4 + +and cg.cluster = c.id +and p.cluster = c.id +and t.cluster = c.id +and txn.cluster = c.id + +and tp.topic = t.id + +and txn_oc.transaction = txn.id +and txn_oc.consumer_group = cg.id +and txn_oc.producer_id = p.id + +and txn_oc_tp.offset_commit = txn_oc.id +and txn_oc_tp.topition = tp.id + +on conflict (consumer_group, topition) + +do update set +committed_offset = excluded.committed_offset, +leader_epoch = excluded.leader_epoch, +timestamp = excluded.timestamp, +metadata = excluded.metadata; diff --git a/tansu-storage/src/pg/consumer_offset_select.sql b/tansu-storage/src/pg/consumer_offset_select.sql new file mode 100644 index 0000000..fd256c9 --- /dev/null +++ b/tansu-storage/src/pg/consumer_offset_select.sql @@ -0,0 +1,26 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +select co.committed_offset +from cluster c,consumer_group cg, consumer_offset co, topic t, topition tp +where c.name = $1 +and cg.name = $2 +and t.name = $3 +and tp.partition = $4 +and cg.cluster = c.id +and co.consumer_group = cg.id +and co.topition = tp.id +and tp.topic = t.id; diff --git a/tansu-storage/src/pg/header_delete_by_topic.sql b/tansu-storage/src/pg/header_delete_by_topic.sql new file mode 100644 index 0000000..53be708 --- /dev/null +++ b/tansu-storage/src/pg/header_delete_by_topic.sql @@ -0,0 +1,24 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +delete from header +using cluster c, record r, topic t, topition tp +where c.name = $1 +and t.name = $2 +and t.cluster = c.id +and tp.topic = t.id +and r.topition = tp.id +and header.record = r.id; diff --git a/tansu-storage/src/pg/header_fetch.sql b/tansu-storage/src/pg/header_fetch.sql new file mode 100644 index 0000000..9adca4d --- /dev/null +++ b/tansu-storage/src/pg/header_fetch.sql @@ -0,0 +1,26 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +select h.k, h.v +from cluster c, header h, record r, topic t, topition tp +where c.name = $1 +and t.name = $2 +and tp.partition = $3 +and r.offset_id = $4 +and t.cluster = c.id +and tp.topic = t.id +and r.topition = tp.id +and h.record = r.id; diff --git a/tansu-storage/src/pg/header_insert.sql b/tansu-storage/src/pg/header_insert.sql new file mode 100644 index 0000000..a41b9a5 --- /dev/null +++ b/tansu-storage/src/pg/header_insert.sql @@ -0,0 +1,28 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into header +(record, k, v) +select r.id, $5, $6 +from cluster c, record r, topic t, topition tp +where c.name = $1 +and t.name = $2 +and tp.partition = $3 +and r.offset_id = $4 +and t.cluster = c.id +and tp.topic = t.id +and r.topition = tp.id +returning header.id; diff --git a/tansu-storage/src/pg/init_producer.sql b/tansu-storage/src/pg/init_producer.sql new file mode 100644 index 0000000..a403817 --- /dev/null +++ b/tansu-storage/src/pg/init_producer.sql @@ -0,0 +1,20 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into producer (cluster) +select cluster.id +from cluster where cluster.name = $1 +returning id, epoch; diff --git a/tansu-storage/src/pg/init_txn.sql b/tansu-storage/src/pg/init_txn.sql new file mode 100644 index 0000000..8899fa2 --- /dev/null +++ b/tansu-storage/src/pg/init_txn.sql @@ -0,0 +1,24 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into txn (cluster, name, transaction_timeout_ms, producer) +select c.id, $2, $3, $4 +from cluster c +where c.name = $1 +on conflict (cluster, name) +do update set +producer = excluded.producer, +last_updated = excluded.last_updated; diff --git a/tansu-storage/src/pg/init_txn_producer.sql b/tansu-storage/src/pg/init_txn_producer.sql new file mode 100644 index 0000000..98724b0 --- /dev/null +++ b/tansu-storage/src/pg/init_txn_producer.sql @@ -0,0 +1,24 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into producer (id, epoch, cluster) +select $2, $3, cluster.id +from cluster where cluster.name = $1 +on conflict (id, epoch) +do update set +epoch = excluded.epoch + 1, +last_updated = excluded.last_updated +returning id, epoch; diff --git a/tansu-storage/src/pg/list_earliest_offset.sql b/tansu-storage/src/pg/list_earliest_offset.sql index 61aa452..663aa30 100644 --- a/tansu-storage/src/pg/list_earliest_offset.sql +++ b/tansu-storage/src/pg/list_earliest_offset.sql @@ -14,24 +14,14 @@ -- You should have received a copy of the GNU Affero General Public License -- along with this program. If not, see . --- prepare earliest (text, text, integer) as -with default_value as (select $2 as topic, last_value as offset, current_timestamp as timestamp from record_id_seq), -minimum_value as (select - $2 topic, record.id as offset, record.timestamp - from - record - join ( - select - min(record.id) as offset - from record, topic, cluster - where - topic.cluster = cluster.id - and cluster.name = $1 - and topic.name = $2 - and record.partition = $3 - and record.topic = topic.id) as minimum - on record.id = minimum.offset) -select coalesce(minimum_value.offset, default_value.offset) as offset, - coalesce(minimum_value.timestamp, default_value.timestamp) as timestamp - from default_value - left join minimum_value on default_value.topic=minimum_value.topic; +-- prepare list_earliest_offset (text, text, integer) as +select r.offset_id, r.timestamp +from cluster c, record r, topic t, topition tp, watermark w +where c.name = $1 +and t.name = $2 +and tp.partition = $3 +and t.cluster = c.id +and tp.topic = t.id +and w.topition = tp.id +and r.offset_id = w.low +and r.topition = tp.id; diff --git a/tansu-storage/src/pg/list_latest_offset_committed.sql b/tansu-storage/src/pg/list_latest_offset_committed.sql new file mode 100644 index 0000000..dd253dd --- /dev/null +++ b/tansu-storage/src/pg/list_latest_offset_committed.sql @@ -0,0 +1,27 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +-- prepare list_latest_offset (text, text, integer) as +select r.offset_id, r.timestamp +from cluster c, record r, topic t, topition tp, watermark w +where c.name = $1 +and t.name = $2 +and tp.partition = $3 +and t.cluster = c.id +and tp.topic = t.id +and w.topition = tp.id +and r.offset_id = w.stable +and r.topition = tp.id; diff --git a/tansu-storage/src/pg/list_latest_offset_timestamp.sql b/tansu-storage/src/pg/list_latest_offset_timestamp.sql new file mode 100644 index 0000000..1bcd15b --- /dev/null +++ b/tansu-storage/src/pg/list_latest_offset_timestamp.sql @@ -0,0 +1,32 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +select +id as offset, timestamp +from +record +join ( +select +coalesce(min(record.id), (select last_value from record_id_seq)) as offset +from record, topic, cluster +where +topic.cluster = cluster.id +and cluster.name = $1 +and topic.name = $2 +and record.partition = $3 +and record.timestamp >= $4 +and record.topic = topic.id) as minimum +on record.id = minimum.offset; diff --git a/tansu-storage/src/pg/list_latest_offset_uncommitted.sql b/tansu-storage/src/pg/list_latest_offset_uncommitted.sql new file mode 100644 index 0000000..2868e18 --- /dev/null +++ b/tansu-storage/src/pg/list_latest_offset_uncommitted.sql @@ -0,0 +1,27 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +-- prepare list_latest_offset (text, text, integer) as +select r.offset_id, r.timestamp +from cluster c, record r, topic t, topition tp, watermark w +where c.name = $1 +and t.name = $2 +and tp.partition = $3 +and t.cluster = c.id +and tp.topic = t.id +and w.topition = tp.id +and r.offset_id = w.high +and r.topition = tp.id; diff --git a/tansu-storage/src/pg/record_delete_by_topic.sql b/tansu-storage/src/pg/record_delete_by_topic.sql new file mode 100644 index 0000000..5c43e29 --- /dev/null +++ b/tansu-storage/src/pg/record_delete_by_topic.sql @@ -0,0 +1,23 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +delete from record +using cluster c, topic t, topition tp +where c.name = $1 +and t.name = $2 +and t.cluster = c.id +and tp.topic = t.id +and record.topition = tp.id; diff --git a/tansu-storage/src/pg/record_fetch.sql b/tansu-storage/src/pg/record_fetch.sql new file mode 100644 index 0000000..60c1775 --- /dev/null +++ b/tansu-storage/src/pg/record_fetch.sql @@ -0,0 +1,31 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +with sized as ( +select +r.offset_id, r.timestamp, r.k, r.v, sum(coalesce(length(r.k), 0) + coalesce(length(r.v), 0)) +over (order by r.offset_id) as bytes +from cluster c, record r, topic t, topition tp +where c.name = $1 +and t.name = $2 +and tp.partition = $3 +and r.offset_id >= $4 +and r.offset_id <= $6 +and t.cluster = c.id +and tp.topic = t.id +and r.topition = tp.id) + +select * from sized where bytes < $5; diff --git a/tansu-storage/src/pg/record_insert.sql b/tansu-storage/src/pg/record_insert.sql new file mode 100644 index 0000000..ffc7e27 --- /dev/null +++ b/tansu-storage/src/pg/record_insert.sql @@ -0,0 +1,25 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into record +(topition, offset_id, producer_id, sequence, timestamp, k, v) +select tp.id, $4, $5, $6, $7, $8, $9 +from cluster c, topic t, topition tp +where c.name = $1 +and t.name = $2 +and tp.partition = $3 +and t.cluster = c.id +and tp.topic = t.id diff --git a/tansu-storage/src/pg/topic_by_cluster.sql b/tansu-storage/src/pg/topic_by_cluster.sql new file mode 100644 index 0000000..6905176 --- /dev/null +++ b/tansu-storage/src/pg/topic_by_cluster.sql @@ -0,0 +1,20 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +select t.uuid, t.name, is_internal, partitions, replication_factor +from topic t, cluster c +where c.name = $1 +and t.cluster = c.id; diff --git a/tansu-storage/src/pg/topic_by_uuid.sql b/tansu-storage/src/pg/topic_by_uuid.sql new file mode 100644 index 0000000..6935b47 --- /dev/null +++ b/tansu-storage/src/pg/topic_by_uuid.sql @@ -0,0 +1,21 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +select t.uuid, t.name, is_internal, partitions, replication_factor +from topic t, cluster c +where c.name = $1 +and t.uuid = $2 +and t.cluster = c.id; diff --git a/tansu-storage/src/pg/topic_configuration_insert.sql b/tansu-storage/src/pg/topic_configuration_insert.sql new file mode 100644 index 0000000..53d5a4b --- /dev/null +++ b/tansu-storage/src/pg/topic_configuration_insert.sql @@ -0,0 +1,23 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into topic_configuration (topic, name, value) +select t.id, $3, $4 +from cluster c, topic t +where c.name = $1 +and t.name = $2 +and t.cluster = c.id +returning topic_configuration.id; diff --git a/tansu-storage/src/pg/topic_delete_by.sql b/tansu-storage/src/pg/topic_delete_by.sql new file mode 100644 index 0000000..1e39d01 --- /dev/null +++ b/tansu-storage/src/pg/topic_delete_by.sql @@ -0,0 +1,21 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +delete from topic +using cluster c +where c.name = $1 +and topic.name = $2 +and topic.cluster = c.id diff --git a/tansu-storage/src/pg/topic_insert.sql b/tansu-storage/src/pg/topic_insert.sql new file mode 100644 index 0000000..28c0efc --- /dev/null +++ b/tansu-storage/src/pg/topic_insert.sql @@ -0,0 +1,22 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into topic +(cluster, name, partitions, replication_factor, is_internal) +select c.id, $2, $3, $4, false +from cluster c +where c.name = $1 +returning topic.uuid; diff --git a/tansu-storage/src/pg/topic_select_name.sql b/tansu-storage/src/pg/topic_select_name.sql new file mode 100644 index 0000000..ae7dd9b --- /dev/null +++ b/tansu-storage/src/pg/topic_select_name.sql @@ -0,0 +1,21 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +select t.uuid, t.name, is_internal, partitions, replication_factor +from topic t, cluster c +where c.name = $1 +and t.name = $2 +and t.cluster = c.id; diff --git a/tansu-storage/src/pg/topic_select_uuid.sql b/tansu-storage/src/pg/topic_select_uuid.sql new file mode 100644 index 0000000..6935b47 --- /dev/null +++ b/tansu-storage/src/pg/topic_select_uuid.sql @@ -0,0 +1,21 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +select t.uuid, t.name, is_internal, partitions, replication_factor +from topic t, cluster c +where c.name = $1 +and t.uuid = $2 +and t.cluster = c.id; diff --git a/tansu-storage/src/pg/topition_delete_by_topic.sql b/tansu-storage/src/pg/topition_delete_by_topic.sql new file mode 100644 index 0000000..bcf0098 --- /dev/null +++ b/tansu-storage/src/pg/topition_delete_by_topic.sql @@ -0,0 +1,22 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +delete from topition +using cluster c, topic t +where c.name = $1 +and t.name = $2 +and t.cluster = c.id +and topition.topic = t.id; diff --git a/tansu-storage/src/pg/topition_insert.sql b/tansu-storage/src/pg/topition_insert.sql new file mode 100644 index 0000000..aef7bb9 --- /dev/null +++ b/tansu-storage/src/pg/topition_insert.sql @@ -0,0 +1,24 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into topition +(topic, partition) +select t.id, $3 +from cluster c, topic t +where c.name = $1 +and t.name = $2 +and t.cluster = c.id +returning topition.id; diff --git a/tansu-storage/src/pg/txn_offset_commit_insert.sql b/tansu-storage/src/pg/txn_offset_commit_insert.sql new file mode 100644 index 0000000..921e8c0 --- /dev/null +++ b/tansu-storage/src/pg/txn_offset_commit_insert.sql @@ -0,0 +1,28 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into txn_offset_commit +(transaction, consumer_group, producer_id, generation_id, member_id) +select txn.id, cg.id, p.id, $6, $7 +from cluster c, consumer_group cg, producer p, txn +where c.name = $1 +and txn.name = $2 +and cg.name = $3 +and p.id = $4 +and p.epoch = $5 +and txn.cluster = c.id +and cg.cluster = c.id +and p.cluster = c.id diff --git a/tansu-storage/src/pg/txn_offset_commit_tp_insert.sql b/tansu-storage/src/pg/txn_offset_commit_tp_insert.sql new file mode 100644 index 0000000..8d29851 --- /dev/null +++ b/tansu-storage/src/pg/txn_offset_commit_tp_insert.sql @@ -0,0 +1,40 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into txn_offset_commit_tp +(offset_commit, topition, committed_offset, leader_epoch, metadata) +select oc.id, tp.id, $8, $9, $10 +from cluster c, +consumer_group cg, +producer p, +topic t, +topition tp, +txn_offset_commit oc, +txn +where c.name = $1 +and txn.name = $2 +and cg.name = $3 +and p.id = $4 +and p.epoch = $5 +and t.name = $6 +and tp.partition = $7 +and txn.cluster = c.id +and cg.cluster = c.id +and p.cluster = c.id +and t.cluster = c.id +and tp.topic = t.id +and oc.transaction = txn.id +and oc.consumer_group = cg.id diff --git a/tansu-storage/src/pg/txn_produce_offset_insert.sql b/tansu-storage/src/pg/txn_produce_offset_insert.sql new file mode 100644 index 0000000..60685a1 --- /dev/null +++ b/tansu-storage/src/pg/txn_produce_offset_insert.sql @@ -0,0 +1,47 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into txn_produce_offset +(txn_topition, offset_start, offset_end) + +select txn_tp.id, $7, $8 + +from cluster c, +producer p, +topic t, +topition tp, +txn_topition txn_tp, +txn + +where c.name = $1 +and txn.name = $2 +and p.id = $3 +and p.epoch = $4 +and t.name = $5 +and tp.partition = $6 + +and p.cluster = c.id +and t.cluster = c.id +and txn.cluster = c.id + +and txn.producer = p.id +and tp.topic = t.id +and txn_tp.transaction = t.id +and txn_tp.topition = tp.id + +on conflict (txn_topition) +do update set offset_end = excluded.offset_end, +last_updated = excluded.last_updated; diff --git a/tansu-storage/src/pg/txn_topition_insert.sql b/tansu-storage/src/pg/txn_topition_insert.sql new file mode 100644 index 0000000..d989e3a --- /dev/null +++ b/tansu-storage/src/pg/txn_topition_insert.sql @@ -0,0 +1,27 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into txn_topition +(transaction, topition) +select txn.id, tp.id +from cluster c, topic t, topition tp, txn +where c.name = $1 +and t.name = $2 +and tp.partition = $3 +and txn.name = $4 +and t.cluster = c.id +and tp.topic = t.id +and txn.cluster = c.id; diff --git a/tansu-storage/src/pg/txn_topition_select.sql b/tansu-storage/src/pg/txn_topition_select.sql new file mode 100644 index 0000000..12673f1 --- /dev/null +++ b/tansu-storage/src/pg/txn_topition_select.sql @@ -0,0 +1,35 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +select t.name, tp.partition, txnp.offset_start, txnp.offset_end +from cluster c, +producer p, +topic t, +topition tp, +txn, +txn_topition txn_tp +where c.name = $1 +and p.id = $2 +and p.epoch = $3 +and t.name = $4 +and tp.partition = $5 +and txn.cluster = c.id +and txn.producer = p.id +and p.cluster = c.id +and t.cluster = c.id +and tp.topic = t.id +and txn_tp.transaction = txn.id +and txn_tp.topition = tp.id; diff --git a/tansu-storage/src/pg/txn_topition_select_txns.sql b/tansu-storage/src/pg/txn_topition_select_txns.sql new file mode 100644 index 0000000..a9137c1 --- /dev/null +++ b/tansu-storage/src/pg/txn_topition_select_txns.sql @@ -0,0 +1,35 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +select count(distinct txn.name) +from cluster c, +producer p, +topic t, +topition tp, +txn, +txn_topition txn_tp +where c.name = $1 +and p.id = $2 +and p.epoch = $3 +and t.name = $4 +and tp.partition = $5 +and txn.cluster = c.id +and txn.producer = p.id +and p.cluster = c.id +and t.cluster = c.id +and tp.topic = t.id +and txn_tp.transaction = txn.id +and txn_tp.topition = tp.id; diff --git a/tansu-storage/src/pg/watermark_delete_by_topic.sql b/tansu-storage/src/pg/watermark_delete_by_topic.sql new file mode 100644 index 0000000..abd88c6 --- /dev/null +++ b/tansu-storage/src/pg/watermark_delete_by_topic.sql @@ -0,0 +1,23 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +delete from watermark +using cluster c, topic t, topition tp +where c.name = $1 +and t.name = $2 +and t.cluster = c.id +and tp.topic = t.id +and watermark.topition = tp.id; diff --git a/tansu-storage/src/pg/watermark_insert.sql b/tansu-storage/src/pg/watermark_insert.sql new file mode 100644 index 0000000..0327a28 --- /dev/null +++ b/tansu-storage/src/pg/watermark_insert.sql @@ -0,0 +1,25 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into watermark (topition) +select tp.id +from cluster c, topic t, topition tp +where c.name = $1 +and t.name = $2 +and tp.partition = $3 +and t.cluster = c.id +and tp.topic = t.id +returning watermark.id; diff --git a/tansu-storage/src/pg/watermark_insert_from_txn.sql b/tansu-storage/src/pg/watermark_insert_from_txn.sql new file mode 100644 index 0000000..6c78b72 --- /dev/null +++ b/tansu-storage/src/pg/watermark_insert_from_txn.sql @@ -0,0 +1,48 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +insert into watermark +(topition, stable) + +select tp.id, txn_po.offset_end + +from + +cluster c, +producer p, +topic t, +topition tp, +txn, +txn_produce_offset txn_po, +txn_topition txn_tp + +where + +c.name = $1 +and txn.name = $2 +and p.id = $3 +and p.epoch = $4 + +and p.cluster = c.id +and t.cluster = c.id +and txn.cluster = c.id + +and tp.topic = t.id + +and txn_po.txn_topition = txn_tp.id + +and txn_tp.transaction = txn.id +and txn_tp.topition = tp.id; diff --git a/tansu-storage/src/pg/watermark_select.sql b/tansu-storage/src/pg/watermark_select.sql new file mode 100644 index 0000000..bfce224 --- /dev/null +++ b/tansu-storage/src/pg/watermark_select.sql @@ -0,0 +1,24 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +select w.low, w.high, w.stable +from cluster c, topic t, topition tp, watermark w +where c.name = $1 +and t.name = $2 +and tp.partition = $3 +and t.cluster = c.id +and tp.topic = t.id +and w.topition = tp.id; diff --git a/tansu-storage/src/pg/watermark_select_for_update.sql b/tansu-storage/src/pg/watermark_select_for_update.sql new file mode 100644 index 0000000..9a30cbc --- /dev/null +++ b/tansu-storage/src/pg/watermark_select_for_update.sql @@ -0,0 +1,25 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +select w.low, w.high, w.stable +from cluster c, topic t, topition tp, watermark w +where c.name = $1 +and t.name = $2 +and tp.partition = $3 +and t.cluster = c.id +and tp.topic = t.id +and w.topition = tp.id +for no key update; diff --git a/tansu-storage/src/pg/watermark_update.sql b/tansu-storage/src/pg/watermark_update.sql new file mode 100644 index 0000000..a1a11d7 --- /dev/null +++ b/tansu-storage/src/pg/watermark_update.sql @@ -0,0 +1,25 @@ +-- -*- mode: sql; sql-product: postgres; -*- +-- Copyright ⓒ 2024 Peter Morgan +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- This program is distributed in the hope that it will be useful, +-- but WITHOUT ANY WARRANTY; without even the implied warranty of +-- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +-- GNU Affero General Public License for more details. +-- +-- You should have received a copy of the GNU Affero General Public License +-- along with this program. If not, see . + +update watermark w +set low = $4, high = $5, stable = $6 +from cluster c, topic t, topition tp +where c.name = $1 +and t.name = $2 +and tp.partition = $3 +and t.cluster = c.id +and tp.topic = t.id +and w.topition = tp.id; diff --git a/tansu-storage/tests/pg_list_offsets.rs b/tansu-storage/tests/pg_list_offsets.rs new file mode 100644 index 0000000..051dec7 --- /dev/null +++ b/tansu-storage/tests/pg_list_offsets.rs @@ -0,0 +1,114 @@ +// Copyright ⓒ 2024 Peter Morgan +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use rand::{distributions::Alphanumeric, prelude::*, thread_rng}; +use tansu_kafka_sans_io::{broker_registration_request::Listener, ErrorCode, IsolationLevel}; +use tansu_storage::{ + pg::Postgres, BrokerRegistationRequest, Error, ListOffsetRequest, Result, Storage, + StorageContainer, Topition, +}; +use tracing::subscriber::DefaultGuard; +use uuid::Uuid; + +fn init_tracing() -> Result { + use std::{fs::File, sync::Arc, thread}; + + use tracing::Level; + use tracing_subscriber::fmt::format::FmtSpan; + + Ok(tracing::subscriber::set_default( + tracing_subscriber::fmt() + .with_level(true) + .with_line_number(true) + .with_thread_names(false) + .with_max_level(Level::DEBUG) + .with_span_events(FmtSpan::ACTIVE) + .with_writer( + thread::current() + .name() + .ok_or(Error::Message(String::from("unnamed thread"))) + .and_then(|name| { + File::create(format!("../logs/{}/{name}.log", env!("CARGO_PKG_NAME"))) + .map_err(Into::into) + }) + .map(Arc::new)?, + ) + .finish(), + )) +} + +fn storage_container(cluster: impl Into, node: i32) -> Result { + Postgres::builder("postgres://postgres:postgres@localhost") + .map(|builder| builder.cluster(cluster)) + .map(|builder| builder.node(node)) + .map(|builder| builder.build()) + .map(StorageContainer::Postgres) +} + +#[tokio::test] +async fn list_offset() -> Result<()> { + let _guard = init_tracing()?; + + let mut rng = thread_rng(); + + let cluster_id = Uuid::now_v7(); + let broker_id = rng.gen_range(0..i32::MAX); + let incarnation_id = Uuid::now_v7(); + + let mut storage_container = storage_container(cluster_id, broker_id)?; + let port = rng.gen_range(1024..u16::MAX); + let security_protocol = rng.gen_range(0..i16::MAX); + + let broker_registration = BrokerRegistationRequest { + broker_id, + cluster_id: cluster_id.into(), + incarnation_id, + listeners: vec![Listener { + name: "broker".into(), + host: "test.local".into(), + port, + security_protocol, + }], + features: vec![], + rack: None, + }; + + storage_container + .register_broker(broker_registration) + .await?; + + let name: String = thread_rng() + .sample_iter(&Alphanumeric) + .take(15) + .map(char::from) + .collect(); + + let num_partitions = rng.gen_range(1..64); + + let partition = rng.gen_range(0..num_partitions); + + let topition = Topition::new(name, partition); + + let earliest = storage_container + .list_offsets( + IsolationLevel::ReadUncommitted, + &[(topition, ListOffsetRequest::Earliest)], + ) + .await?; + + assert_eq!(ErrorCode::None, earliest[0].1.error_code); + + Ok(()) +} diff --git a/tansu-storage/tests/pg_producer.rs b/tansu-storage/tests/pg_producer.rs new file mode 100644 index 0000000..7c61616 --- /dev/null +++ b/tansu-storage/tests/pg_producer.rs @@ -0,0 +1,193 @@ +// Copyright ⓒ 2024 Peter Morgan +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use bytes::Bytes; +use rand::{distributions::Alphanumeric, prelude::*, thread_rng}; +use tansu_kafka_sans_io::{ + broker_registration_request::Listener, + create_topics_request::CreatableTopic, + record::{inflated, Record}, + ErrorCode, IsolationLevel, +}; +use tansu_storage::{ + pg::Postgres, BrokerRegistationRequest, Error, ListOffsetRequest, Result, Storage, + StorageContainer, TopicId, Topition, +}; +use tracing::{debug, subscriber::DefaultGuard}; +use uuid::Uuid; + +fn init_tracing() -> Result { + use std::{fs::File, sync::Arc, thread}; + + use tracing::Level; + use tracing_subscriber::fmt::format::FmtSpan; + + Ok(tracing::subscriber::set_default( + tracing_subscriber::fmt() + .with_level(true) + .with_line_number(true) + .with_thread_names(false) + .with_max_level(Level::DEBUG) + .with_span_events(FmtSpan::ACTIVE) + .with_writer( + thread::current() + .name() + .ok_or(Error::Message(String::from("unnamed thread"))) + .and_then(|name| { + File::create(format!("../logs/{}/{name}.log", env!("CARGO_PKG_NAME"))) + .map_err(Into::into) + }) + .map(Arc::new)?, + ) + .finish(), + )) +} + +fn storage_container(cluster: impl Into, node: i32) -> Result { + Postgres::builder("postgres://postgres:postgres@localhost") + .map(|builder| builder.cluster(cluster)) + .map(|builder| builder.node(node)) + .map(|builder| builder.build()) + .map(StorageContainer::Postgres) +} + +#[tokio::test] +async fn produce() -> Result<()> { + let _guard = init_tracing()?; + + let mut rng = thread_rng(); + + let cluster_id = Uuid::now_v7(); + let broker_id = rng.gen_range(0..i32::MAX); + let incarnation_id = Uuid::now_v7(); + + let mut storage_container = storage_container(cluster_id, broker_id)?; + let port = rng.gen_range(1024..u16::MAX); + let security_protocol = rng.gen_range(0..i16::MAX); + + let broker_registration = BrokerRegistationRequest { + broker_id, + cluster_id: cluster_id.into(), + incarnation_id, + listeners: vec![Listener { + name: "broker".into(), + host: "test.local".into(), + port, + security_protocol, + }], + features: vec![], + rack: None, + }; + + storage_container + .register_broker(broker_registration) + .await?; + + let name: String = thread_rng() + .sample_iter(&Alphanumeric) + .take(15) + .map(char::from) + .collect(); + + let num_partitions = rng.gen_range(1..64); + let replication_factor = rng.gen_range(0..64); + let assignments = Some([].into()); + let configs = Some([].into()); + + let creatable = CreatableTopic { + name: name.clone(), + num_partitions, + replication_factor, + assignments, + configs, + }; + + let id = storage_container + .create_topic(creatable.clone(), false) + .await?; + + let partition = rng.gen_range(0..num_partitions); + + let topition = Topition::new(name, partition); + + let before_produce_earliest = storage_container + .list_offsets( + IsolationLevel::ReadUncommitted, + &[(topition.clone(), ListOffsetRequest::Earliest)], + ) + .await + .inspect(|offset| debug!(?offset))?; + + assert_eq!(1, before_produce_earliest.len()); + assert_eq!(ErrorCode::None, before_produce_earliest[0].1.error_code); + + let before_produce_latest = storage_container + .list_offsets( + IsolationLevel::ReadUncommitted, + &[(topition.clone(), ListOffsetRequest::Latest)], + ) + .await + .inspect(|offset| debug!(?offset))?; + + assert_eq!(1, before_produce_latest.len()); + assert_eq!(ErrorCode::None, before_produce_latest[0].1.error_code); + + let batch = inflated::Batch::builder() + .record(Record::builder().value(Bytes::from_static(b"Lorem ipsum dolor sit amet").into())) + .build() + .and_then(TryInto::try_into) + .inspect(|deflated| debug!(?deflated))?; + + let offset = storage_container + .produce(None, &topition, batch) + .await + .inspect(|offset| debug!(?offset))?; + + assert_eq!(before_produce_latest[0].1.offset, Some(offset)); + + let after_produce_earliest = storage_container + .list_offsets( + IsolationLevel::ReadUncommitted, + &[(topition.clone(), ListOffsetRequest::Earliest)], + ) + .await + .inspect(|offset| debug!(?offset))?; + + assert_eq!(1, after_produce_earliest.len()); + assert_eq!(ErrorCode::None, after_produce_earliest[0].1.error_code); + assert_eq!( + before_produce_earliest[0].1.offset, + after_produce_earliest[0].1.offset + ); + + let after_produce_latest = storage_container + .list_offsets( + IsolationLevel::ReadUncommitted, + &[(topition.clone(), ListOffsetRequest::Latest)], + ) + .await + .inspect(|offset| debug!(?offset))?; + + assert_eq!(1, after_produce_latest.len()); + assert_eq!(ErrorCode::None, after_produce_latest[0].1.error_code); + assert_eq!(Some(offset), after_produce_latest[0].1.offset); + + assert_eq!( + ErrorCode::None, + storage_container.delete_topic(&TopicId::from(id)).await? + ); + + Ok(()) +} diff --git a/tansu-storage/tests/pg_register_broker.rs b/tansu-storage/tests/pg_register_broker.rs new file mode 100644 index 0000000..f944e30 --- /dev/null +++ b/tansu-storage/tests/pg_register_broker.rs @@ -0,0 +1,98 @@ +// Copyright ⓒ 2024 Peter Morgan +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use rand::{prelude::*, thread_rng}; +use tansu_kafka_sans_io::broker_registration_request::Listener; +use tansu_storage::{ + pg::Postgres, BrokerRegistationRequest, Error, Result, Storage, StorageContainer, +}; +use tracing::subscriber::DefaultGuard; +use uuid::Uuid; + +fn init_tracing() -> Result { + use std::{fs::File, sync::Arc, thread}; + + use tracing::Level; + use tracing_subscriber::fmt::format::FmtSpan; + + Ok(tracing::subscriber::set_default( + tracing_subscriber::fmt() + .with_level(true) + .with_line_number(true) + .with_thread_names(false) + .with_max_level(Level::DEBUG) + .with_span_events(FmtSpan::ACTIVE) + .with_writer( + thread::current() + .name() + .ok_or(Error::Message(String::from("unnamed thread"))) + .and_then(|name| { + File::create(format!("../logs/{}/{name}.log", env!("CARGO_PKG_NAME"))) + .map_err(Into::into) + }) + .map(Arc::new)?, + ) + .finish(), + )) +} + +fn storage_container(cluster: impl Into, node: i32) -> Result { + Postgres::builder("postgres://postgres:postgres@localhost") + .map(|builder| builder.cluster(cluster)) + .map(|builder| builder.node(node)) + .map(|builder| builder.build()) + .map(StorageContainer::Postgres) +} + +#[tokio::test] +async fn register_broker() -> Result<()> { + let _guard = init_tracing()?; + + let mut rng = thread_rng(); + + let cluster_id = Uuid::now_v7(); + let broker_id = rng.gen_range(0..i32::MAX); + let incarnation_id = Uuid::now_v7(); + + let mut storage_container = storage_container(cluster_id, broker_id)?; + let port = rng.gen_range(1024..u16::MAX); + let security_protocol = rng.gen_range(0..i16::MAX); + + let broker_registration = BrokerRegistationRequest { + broker_id, + cluster_id: cluster_id.into(), + incarnation_id, + listeners: vec![Listener { + name: "broker".into(), + host: "test.local".into(), + port, + security_protocol, + }], + features: vec![], + rack: None, + }; + + storage_container + .register_broker(broker_registration) + .await?; + + let brokers = storage_container.brokers().await?; + assert_eq!(1, brokers.len()); + assert_eq!(broker_id, brokers[0].broker_id); + assert_eq!(port, u16::try_from(brokers[0].port)?); + assert_eq!("test.local", brokers[0].host); + + Ok(()) +} diff --git a/tansu-storage/tests/pg_topic_lifecycle.rs b/tansu-storage/tests/pg_topic_lifecycle.rs new file mode 100644 index 0000000..c63133d --- /dev/null +++ b/tansu-storage/tests/pg_topic_lifecycle.rs @@ -0,0 +1,170 @@ +// Copyright ⓒ 2024 Peter Morgan +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use rand::{distributions::Alphanumeric, prelude::*, thread_rng}; +use tansu_kafka_sans_io::{ + broker_registration_request::Listener, create_topics_request::CreatableTopic, ErrorCode, +}; +use tansu_storage::{ + pg::Postgres, BrokerRegistationRequest, Error, Result, Storage, StorageContainer, TopicId, +}; +use tracing::subscriber::DefaultGuard; +use uuid::Uuid; + +fn init_tracing() -> Result { + use std::{fs::File, sync::Arc, thread}; + + use tracing::Level; + use tracing_subscriber::fmt::format::FmtSpan; + + Ok(tracing::subscriber::set_default( + tracing_subscriber::fmt() + .with_level(true) + .with_line_number(true) + .with_thread_names(false) + .with_max_level(Level::DEBUG) + .with_span_events(FmtSpan::ACTIVE) + .with_writer( + thread::current() + .name() + .ok_or(Error::Message(String::from("unnamed thread"))) + .and_then(|name| { + File::create(format!("../logs/{}/{name}.log", env!("CARGO_PKG_NAME"))) + .map_err(Into::into) + }) + .map(Arc::new)?, + ) + .finish(), + )) +} + +fn storage_container(cluster: impl Into, node: i32) -> Result { + Postgres::builder("postgres://postgres:postgres@localhost") + .map(|builder| builder.cluster(cluster)) + .map(|builder| builder.node(node)) + .map(|builder| builder.build()) + .map(StorageContainer::Postgres) +} + +#[tokio::test] +async fn topic_lifecycle() -> Result<()> { + let _guard = init_tracing()?; + + let mut rng = thread_rng(); + + let cluster_id = Uuid::now_v7(); + let broker_id = rng.gen_range(0..i32::MAX); + let incarnation_id = Uuid::now_v7(); + + let mut storage_container = storage_container(cluster_id, broker_id)?; + let port = rng.gen_range(1024..u16::MAX); + let security_protocol = rng.gen_range(0..i16::MAX); + + let broker_registration = BrokerRegistationRequest { + broker_id, + cluster_id: cluster_id.into(), + incarnation_id, + listeners: vec![Listener { + name: "broker".into(), + host: "test.local".into(), + port, + security_protocol, + }], + features: vec![], + rack: None, + }; + + storage_container + .register_broker(broker_registration) + .await?; + + let name: String = thread_rng() + .sample_iter(&Alphanumeric) + .take(15) + .map(char::from) + .collect(); + + let num_partitions = rng.gen_range(1..64); + let replication_factor = rng.gen_range(0..64); + let assignments = Some([].into()); + let configs = Some([].into()); + + let creatable = CreatableTopic { + name: name.clone(), + num_partitions, + replication_factor, + assignments, + configs, + }; + + let id = storage_container + .create_topic(creatable.clone(), false) + .await?; + + // metadata via topic uuid + // + let metadata = storage_container + .metadata(Some(&[TopicId::from(id)])) + .await?; + + assert_eq!(1, metadata.topics().len()); + assert_eq!( + ErrorCode::None, + ErrorCode::try_from(metadata.topics()[0].error_code)? + ); + assert_eq!(Some(name.clone()), metadata.topics()[0].name); + assert_eq!( + id, + metadata.topics()[0].topic_id.map(Uuid::from_bytes).unwrap() + ); + + // metadata via topic name + // + let metadata = storage_container + .metadata(Some(&[TopicId::from(name.clone())])) + .await?; + + assert_eq!(1, metadata.topics().len()); + assert_eq!( + ErrorCode::None, + ErrorCode::try_from(metadata.topics()[0].error_code)? + ); + assert_eq!(Some(name.clone()), metadata.topics()[0].name); + assert_eq!( + id, + metadata.topics()[0].topic_id.map(Uuid::from_bytes).unwrap() + ); + + // creating a topic with the same name causes an API error: topic already exists + // + assert!(matches!( + storage_container.create_topic(creatable, false).await, + Err(Error::Api(ErrorCode::TopicAlreadyExists)) + )); + + assert!(matches!( + storage_container + .delete_topic(&TopicId::from(name.clone())) + .await, + Ok(ErrorCode::None) + )); + + assert!(matches!( + storage_container.delete_topic(&TopicId::from(name)).await, + Ok(ErrorCode::UnknownTopicOrPartition) + )); + + Ok(()) +} diff --git a/work-dir/initdb.d/010-schema.sql b/work-dir/initdb.d/010-schema.sql index c86aeea..8da1e76 100644 --- a/work-dir/initdb.d/010-schema.sql +++ b/work-dir/initdb.d/010-schema.sql @@ -16,7 +16,7 @@ begin; -create table cluster ( +create table if not exists cluster ( id int generated always as identity primary key, name text not null, unique (name), @@ -24,90 +24,94 @@ create table cluster ( created_at timestamp default current_timestamp not null ); -create table broker ( +create table if not exists broker ( id int generated always as identity primary key, - cluster integer references cluster(id) not null, - node integer not null, + + cluster int references cluster(id) not null, + node int not null, + unique (cluster, node), + rack text, incarnation uuid not null, - unique (cluster, node), last_updated timestamp default current_timestamp not null, created_at timestamp default current_timestamp not null ); -create table listener ( +create table if not exists listener ( id int generated always as identity primary key, - broker integer references broker(id) not null, + + broker int references broker(id) not null, name text not null, - host text not null, - port integer not null, unique (broker, name), + + host text not null, + port int not null, last_updated timestamp default current_timestamp not null, created_at timestamp default current_timestamp not null ); -create table topic ( - id uuid primary key default gen_random_uuid(), - cluster integer references cluster(id) not null, +create table if not exists topic ( + id int generated always as identity primary key, + + cluster int references cluster(id) not null, name text not null, unique (cluster, name), - partitions integer not null, - replication_factor integer not null, + + uuid uuid default gen_random_uuid(), + partitions int not null, + replication_factor int not null, is_internal bool default false not null, last_updated timestamp default current_timestamp not null, created_at timestamp default current_timestamp not null ); -create table topic_leader ( - topic uuid references topic(id) not null, - partition integer not null, - leader integer not null, - epoch integer, - primary key (topic, partition, leader), - last_updated timestamp default current_timestamp not null, - created_at timestamp default current_timestamp not null -); +create table if not exists topition ( + id int generated always as identity primary key, + + topic int references topic(id), + partition int, + unique (topic, partition), -create table topic_replica_node ( - topic uuid references topic(id), - partition integer, - replica integer, - primary key (topic, partition), last_updated timestamp default current_timestamp not null, created_at timestamp default current_timestamp not null ); -create table topic_isr_node ( - topic uuid references topic(id), - partition integer, - replica integer, - primary key (topic, partition), + +create table if not exists watermark ( + id int generated always as identity primary key, + + topition int references topition(id), + unique(topition), + + low bigint, + high bigint, + stable bigint, + last_updated timestamp default current_timestamp not null, created_at timestamp default current_timestamp not null ); -create table topic_configuration ( - topic uuid references topic(id), +create table if not exists topic_configuration ( + id int generated always as identity primary key, + + topic int references topic(id), name text not null, + unique (topic, name), + value text, - primary key (topic, name), last_updated timestamp default current_timestamp not null, created_at timestamp default current_timestamp not null ); -create type "Compression" as enum ( - 'Gzip', - 'Snappy', - 'Lz4', - 'Zstd' -); +create table if not exists record ( + id bigint generated always as identity primary key, + + topition int references topition(id), + offset_id bigint not null, + unique (topition, offset_id), -create table record ( - id bigint primary key generated always as identity (minvalue 0), - topic uuid references topic(id), - partition integer, producer_id bigint, - sequence integer, + sequence int, timestamp timestamp, k bytea, v bytea, @@ -115,35 +119,129 @@ create table record ( created_at timestamp default current_timestamp not null ); -create table header ( - record bigint references record(id), +create table if not exists header ( + id bigint generated always as identity primary key, + + record int references record(id), k bytea, + unique (record, k), + v bytea, + last_updated timestamp default current_timestamp not null, created_at timestamp default current_timestamp not null ); -create table consumer_offset ( - grp text, - topic uuid references topic(id), - partition integer, - primary key (grp, topic, partition), +create table if not exists consumer_group ( + id int generated always as identity primary key, + + cluster int references cluster(id) not null, + name text not null, + unique (cluster, name), + + e_tag uuid not null, + detail json not null, + last_updated timestamp default current_timestamp not null, + created_at timestamp default current_timestamp not null +); + +create table if not exists consumer_offset ( + id int generated always as identity primary key, + + consumer_group int references consumer_group(id), + topition int references topition(id), + unique (consumer_group, topition), + committed_offset bigint, - leader_epoch integer, + leader_epoch int, timestamp timestamp, metadata text, last_updated timestamp default current_timestamp not null, created_at timestamp default current_timestamp not null ); -create table consumer_group ( - grp text not null, - cluster integer references cluster(id) not null, - primary key (grp, cluster), - e_tag uuid not null, - detail json not null, +-- InitProducerIdRequest +-- +create table if not exists producer ( + id bigint generated by default as identity primary key, + epoch int default 0 not null, + unique (id, epoch), + + cluster int references cluster(id) not null, + last_updated timestamp default current_timestamp not null, + created_at timestamp default current_timestamp not null +); + +-- InitProducerIdRequest (txn detail) +-- +create table if not exists txn ( + id int generated always as identity primary key, + + cluster int references cluster(id), + name text, + unique (cluster, name), + + transaction_timeout_ms int not null, + producer bigint references producer(id), last_updated timestamp default current_timestamp not null, created_at timestamp default current_timestamp not null ); +-- AddPartitionsToTxnRequest +-- +create table if not exists txn_topition ( + id int generated always as identity primary key, + + transaction int references txn(id), + topition int references topition(id), + unique (transaction, topition), + + last_updated timestamp default current_timestamp not null, + created_at timestamp default current_timestamp not null +); + +create table if not exists txn_produce_offset ( + id int generated always as identity primary key, + + txn_topition int references txn_topition(id), + unique (txn_topition), + + offset_start bigint, + offset_end bigint, + + last_updated timestamp default current_timestamp not null, + created_at timestamp default current_timestamp not null +); + +create table if not exists txn_offset_commit ( + id int generated always as identity primary key, + + transaction int references txn(id), + consumer_group int references consumer_group(id), + unique (transaction, consumer_group), + + producer_id bigint references producer(id), + + generation_id int, + member_id text, + + last_updated timestamp default current_timestamp not null, + created_at timestamp default current_timestamp not null +); + +create table if not exists txn_offset_commit_tp ( + id int generated always as identity primary key, + + offset_commit int references txn_offset_commit(id), + topition int references topition(id), + unique (offset_commit, topition), + + committed_offset bigint, + leader_epoch int, + metadata text, + last_updated timestamp default current_timestamp not null, + created_at timestamp default current_timestamp not null +); + + commit;