Skip to content

Commit

Permalink
Merge pull request redpanda-data#7507 from jcsp/remove-consumer-offse…
Browse files Browse the repository at this point in the history
…ts-migration

features: remove consumer offset migration code + test
  • Loading branch information
mmaslankaprv committed Nov 28, 2022
2 parents 29c753e + f9d030b commit f40fd80
Show file tree
Hide file tree
Showing 15 changed files with 57 additions and 1,208 deletions.
7 changes: 2 additions & 5 deletions src/v/cluster/health_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -196,12 +196,9 @@ ss::future<> health_manager::do_tick() {
* back off and wait before trying to fix replication for any
* other internal topics.
*/
auto ok = co_await ensure_topic_replication(model::kafka_group_nt);
auto ok = co_await ensure_topic_replication(
model::kafka_consumer_offsets_nt);

if (ok) {
ok = co_await ensure_topic_replication(
model::kafka_consumer_offsets_nt);
}
if (ok) {
ok = co_await ensure_topic_replication(model::id_allocator_nt);
}
Expand Down
3 changes: 1 addition & 2 deletions src/v/cluster/partition.cc
Original file line number Diff line number Diff line change
Expand Up @@ -66,8 +66,7 @@ partition::partition(
}
const model::topic_namespace tp_ns(
_raft->ntp().ns, _raft->ntp().tp.topic);
bool is_group_ntp = tp_ns == model::kafka_group_nt
|| tp_ns == model::kafka_consumer_offsets_nt;
bool is_group_ntp = tp_ns == model::kafka_consumer_offsets_nt;

bool has_rm_stm = (_is_tx_enabled || _is_idempotence_enabled)
&& model::controller_ntp != _raft->ntp()
Expand Down
3 changes: 0 additions & 3 deletions src/v/cluster/topics_frontend.cc
Original file line number Diff line number Diff line change
Expand Up @@ -747,9 +747,6 @@ ss::future<std::error_code> topics_frontend::move_partition_replicas(
std::vector<model::broker_shard> new_replica_set,
model::timeout_clock::time_point tout,
std::optional<model::term_id> term) {
if (_partition_movement_disabled) {
return ss::make_ready_future<std::error_code>(errc::feature_disabled);
}
move_partition_replicas_cmd cmd(std::move(ntp), std::move(new_replica_set));

return replicate_and_wait(_stm, _features, _as, std::move(cmd), tout, term);
Expand Down
5 changes: 0 additions & 5 deletions src/v/cluster/topics_frontend.h
Original file line number Diff line number Diff line change
Expand Up @@ -110,9 +110,6 @@ class topics_frontend {

ss::future<bool> validate_shard(model::node_id node, uint32_t shard) const;

void disable_partition_movement() { _partition_movement_disabled = true; }
void enable_partition_movement() { _partition_movement_disabled = false; }

ss::future<result<std::vector<move_cancellation_result>>>
cancel_moving_partition_replicas_node(
model::node_id,
Expand Down Expand Up @@ -223,8 +220,6 @@ class topics_frontend {

config::binding<unsigned> _hard_max_disk_usage_ratio;

bool _partition_movement_disabled = false;

static constexpr std::chrono::seconds _get_health_report_timeout = 10s;
};

Expand Down
1 change: 0 additions & 1 deletion src/v/features/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,6 @@ v_cc_library(
feature_table_snapshot.cc
feature_migrator.cc
migrators/cloud_storage_config.cc
migrators/group_metadata.cc
logger.cc
DEPS
Seastar::seastar
Expand Down
2 changes: 1 addition & 1 deletion src/v/features/feature_table.h
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ constexpr static std::array feature_schema{
"consumer_offsets",
feature::consumer_offsets,
feature_spec::available_policy::always,
feature_spec::prepare_policy::requires_migration},
feature_spec::prepare_policy::always},
feature_spec{
cluster::cluster_version{3},
"maintenance_mode",
Expand Down
1 change: 0 additions & 1 deletion src/v/features/migrators.h
Original file line number Diff line number Diff line change
Expand Up @@ -10,4 +10,3 @@
// This header is a convenience to avoid application.cc having to list
// out the headers for each migrator class
#include "features/migrators/cloud_storage_config.h"
#include "features/migrators/group_metadata.h"
Loading

0 comments on commit f40fd80

Please sign in to comment.