-
Notifications
You must be signed in to change notification settings - Fork 594
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
tx/group compaction fixes #24637
base: dev
Are you sure you want to change the base?
tx/group compaction fixes #24637
Changes from 4 commits
03c425e
5e8358a
00df369
b80c4d9
e247e1f
1db8d69
4f292ff
15a0d0a
796ac17
c1c904d
e5b9b46
10ea2db
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -29,6 +29,7 @@ | |
#include "kafka/server/group.h" | ||
#include "kafka/server/group_metadata.h" | ||
#include "kafka/server/group_recovery_consumer.h" | ||
#include "kafka/server/group_tx_tracker_stm.h" | ||
#include "kafka/server/logger.h" | ||
#include "model/fundamental.h" | ||
#include "model/namespace.h" | ||
|
@@ -978,7 +979,7 @@ ss::future<> group_manager::do_recover_group( | |
if (session.tx) { | ||
auto& tx = *session.tx; | ||
group::ongoing_transaction group_tx( | ||
tx.tx_seq, tx.tm_partition, tx.timeout); | ||
tx.tx_seq, tx.tm_partition, tx.timeout, tx.begin_offset); | ||
for (auto& [tp, o_md] : tx.offsets) { | ||
group_tx.offsets[tp] = group::pending_tx_offset{ | ||
.offset_metadata = group_tx::partition_offset{ | ||
|
@@ -1620,6 +1621,64 @@ group_manager::describe_group(const model::ntp& ntp, const kafka::group_id& g) { | |
return group->describe(); | ||
} | ||
|
||
partition_response | ||
group_manager::describe_partition_producers(const model::ntp& ntp) { | ||
vlog(klog.debug, "describe producers: {}", ntp); | ||
partition_response response; | ||
response.partition_index = ntp.tp.partition; | ||
auto it = _partitions.find(ntp); | ||
if (it == _partitions.end() || !it->second->partition->is_leader()) { | ||
response.error_code = error_code::not_leader_for_partition; | ||
return response; | ||
} | ||
response.error_code = kafka::error_code::none; | ||
// snapshot the list of groups attached to this partition | ||
fragmented_vector<std::pair<group_id, group_ptr>> groups; | ||
std::copy_if( | ||
_groups.begin(), | ||
_groups.end(), | ||
std::back_inserter(groups), | ||
[&ntp](auto g_pair) { | ||
const auto& [group_id, group] = g_pair; | ||
return group->partition()->ntp() == ntp; | ||
}); | ||
for (auto& [gid, group] : groups) { | ||
if (group->in_state(group_state::dead)) { | ||
continue; | ||
} | ||
auto partition = group->partition(); | ||
if (!partition) { | ||
// unlikely, conservative check | ||
continue; | ||
} | ||
for (const auto& [id, state] : group->producers()) { | ||
auto& tx = state.transaction; | ||
int64_t start_offset = -1; | ||
if (tx && tx->begin_offset >= model::offset{0}) { | ||
start_offset = partition->get_offset_translator_state() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. is it possible that |
||
->from_log_offset(tx->begin_offset); | ||
} | ||
int64_t last_timetamp = -1; | ||
if (tx) { | ||
auto time_since_last_update = model::timeout_clock::now() | ||
- tx->last_update; | ||
auto last_update_ts | ||
= (model::timestamp_clock::now() - time_since_last_update); | ||
last_timetamp = last_update_ts.time_since_epoch() / 1ms; | ||
} | ||
response.active_producers.push_back({ | ||
.producer_id = id, | ||
.producer_epoch = state.epoch, | ||
.last_sequence = tx ? tx->tx_seq : -1, | ||
.last_timestamp = last_timetamp, | ||
.coordinator_epoch = -1, | ||
.current_txn_start_offset = start_offset, | ||
}); | ||
} | ||
} | ||
return response; | ||
} | ||
|
||
ss::future<std::vector<deletable_group_result>> group_manager::delete_groups( | ||
std::vector<std::pair<model::ntp, group_id>> groups) { | ||
std::vector<deletable_group_result> results; | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -26,6 +26,7 @@ | |
#include "kafka/protocol/offset_delete.h" | ||
#include "kafka/protocol/offset_fetch.h" | ||
#include "kafka/protocol/schemata/delete_groups_response.h" | ||
#include "kafka/protocol/schemata/describe_producers_response.h" | ||
#include "kafka/protocol/schemata/list_groups_response.h" | ||
#include "kafka/protocol/sync_group.h" | ||
#include "kafka/protocol/txn_offset_commit.h" | ||
|
@@ -185,6 +186,8 @@ class group_manager { | |
|
||
described_group describe_group(const model::ntp&, const kafka::group_id&); | ||
|
||
partition_response describe_partition_producers(const model::ntp&); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: |
||
|
||
ss::future<std::vector<deletable_group_result>> | ||
delete_groups(std::vector<std::pair<model::ntp, group_id>>); | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -52,12 +52,30 @@ ss::future<> group_tx_tracker_stm::do_apply(const model::record_batch& b) { | |
|
||
model::offset group_tx_tracker_stm::max_collectible_offset() { | ||
auto result = last_applied_offset(); | ||
for (const auto& [_, group_state] : _all_txs) { | ||
for (const auto& [gid, group_state] : _all_txs) { | ||
if (!group_state.begin_offsets.empty()) { | ||
result = std::min( | ||
result, model::prev_offset(*group_state.begin_offsets.begin())); | ||
auto group_least_begin = *group_state.begin_offsets.begin(); | ||
result = std::min(result, model::prev_offset(group_least_begin)); | ||
vlog( | ||
klog.trace, | ||
"[{}] group: {}, earliest tx begin offset: {}", | ||
_raft->ntp(), | ||
gid, | ||
group_least_begin); | ||
if (klog.is_enabled(ss::log_level::trace)) { | ||
for (auto& [pid, offset] : group_state.producer_to_begin) { | ||
vlog( | ||
klog.trace, | ||
"[{}] group: {}, producer: {}, begin: {}", | ||
_raft->ntp(), | ||
Comment on lines
+69
to
+70
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: can we add more context to this log line ? i.e. it is not known what the begin is |
||
gid, | ||
pid, | ||
offset); | ||
} | ||
} | ||
} | ||
} | ||
|
||
return result; | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -26,6 +26,30 @@ class group_tx_tracker_stm final | |
, public group_data_parser<group_tx_tracker_stm> { | ||
public: | ||
static constexpr std::string_view name = "group_tx_tracker_stm"; | ||
struct per_group_state | ||
: serde::envelope< | ||
per_group_state, | ||
serde::version<0>, | ||
serde::compat_version<0>> { | ||
per_group_state() = default; | ||
|
||
per_group_state(model::producer_identity pid, model::offset offset) { | ||
maybe_add_tx_begin(pid, offset); | ||
} | ||
|
||
void | ||
maybe_add_tx_begin(model::producer_identity pid, model::offset offset); | ||
|
||
absl::btree_set<model::offset> begin_offsets; | ||
|
||
absl::btree_map<model::producer_identity, model::offset> | ||
producer_to_begin; | ||
|
||
auto serde_fields() { | ||
return std::tie(begin_offsets, producer_to_begin); | ||
} | ||
}; | ||
using all_txs_t = absl::btree_map<kafka::group_id, per_group_state>; | ||
|
||
group_tx_tracker_stm(ss::logger&, raft::consensus*); | ||
|
||
|
@@ -72,31 +96,9 @@ class group_tx_tracker_stm final | |
model::record_batch_header, kafka::group_tx::commit_metadata); | ||
ss::future<> handle_version_fence(features::feature_table::version_fence); | ||
|
||
private: | ||
struct per_group_state | ||
: serde::envelope< | ||
per_group_state, | ||
serde::version<0>, | ||
serde::compat_version<0>> { | ||
per_group_state() = default; | ||
|
||
per_group_state(model::producer_identity pid, model::offset offset) { | ||
maybe_add_tx_begin(pid, offset); | ||
} | ||
|
||
void | ||
maybe_add_tx_begin(model::producer_identity pid, model::offset offset); | ||
|
||
absl::btree_set<model::offset> begin_offsets; | ||
|
||
absl::btree_map<model::producer_identity, model::offset> | ||
producer_to_begin; | ||
const all_txs_t& inflight_transactions() const { return _all_txs; } | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This part seems not relevant for current commit ? |
||
|
||
auto serde_fields() { | ||
return std::tie(begin_offsets, producer_to_begin); | ||
} | ||
}; | ||
using all_txs_t = absl::btree_map<kafka::group_id, per_group_state>; | ||
private: | ||
struct snapshot | ||
: serde::envelope<snapshot, serde::version<0>, serde::compat_version<0>> { | ||
all_txs_t transactions; | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
chunked_vector ?