Skip to content

Commit

Permalink
Merge pull request redpanda-data#17002 from BenPope/reduce_header_dep…
Browse files Browse the repository at this point in the history
…endencies

treewide: Reduce header dependencies
  • Loading branch information
BenPope authored Mar 16, 2024
2 parents 3772047 + cf3f1d8 commit 0ecbec8
Show file tree
Hide file tree
Showing 224 changed files with 526 additions and 702 deletions.
3 changes: 0 additions & 3 deletions src/v/archival/adjacent_segment_merger.h
Original file line number Diff line number Diff line change
Expand Up @@ -11,9 +11,6 @@

#include "archival/ntp_archiver_service.h"
#include "archival/types.h"
#include "cloud_storage/remote.h"
#include "cluster/partition.h"
#include "config/bounded_property.h"
#include "config/property.h"
#include "utils/retry_chain_node.h"

Expand Down
4 changes: 4 additions & 0 deletions src/v/archival/ntp_archiver_service.cc
Original file line number Diff line number Diff line change
Expand Up @@ -3091,6 +3091,10 @@ ntp_archiver::prepare_transfer_leadership(ss::lowres_clock::duration timeout) {
co_return true;
}

const storage::ntp_config& ntp_archiver::ntp_config() const {
return _parent.log()->config();
}

void ntp_archiver::complete_transfer_leadership() {
vlog(
_rtclog.trace,
Expand Down
7 changes: 1 addition & 6 deletions src/v/archival/ntp_archiver_service.h
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,7 @@
#include "cloud_storage/remote.h"
#include "cloud_storage/remote_segment_index.h"
#include "cloud_storage/types.h"
#include "cloud_storage_clients/client.h"
#include "cluster/fwd.h"
#include "cluster/partition.h"
#include "container/intrusive_list_helpers.h"
#include "model/fundamental.h"
#include "model/metadata.h"
#include "model/record.h"
Expand Down Expand Up @@ -319,9 +316,7 @@ class ntp_archiver {
/// it will resume uploads.
void complete_transfer_leadership();

const storage::ntp_config& ntp_config() const {
return _parent.log()->config();
}
const storage::ntp_config& ntp_config() const;

/// If we have a projected manifest clean offset, then flush it to
/// the persistent stm clean offset.
Expand Down
5 changes: 0 additions & 5 deletions src/v/archival/scrubber.h
Original file line number Diff line number Diff line change
Expand Up @@ -14,13 +14,8 @@
#include "archival/scrubber_scheduler.h"
#include "archival/types.h"
#include "cloud_storage/anomalies_detector.h"
#include "cloud_storage/base_manifest.h"
#include "cloud_storage/fwd.h"
#include "cluster/fwd.h"
#include "cluster/types.h"
#include "features/feature_table.h"
#include "random/simple_time_jitter.h"
#include "storage/fwd.h"

#include <seastar/core/future.hh>

Expand Down
7 changes: 0 additions & 7 deletions src/v/archival/tests/service_fixture.h
Original file line number Diff line number Diff line change
Expand Up @@ -14,13 +14,9 @@
#include "cloud_storage/remote.h"
#include "cloud_storage/types.h"
#include "cloud_storage_clients/client_pool.h"
#include "cluster/partition_leaders_table.h"
#include "cluster/types.h"
#include "config/configuration.h"
#include "http/tests/http_imposter.h"
#include "model/fundamental.h"
#include "model/metadata.h"
#include "model/namespace.h"
#include "redpanda/tests/fixture.h"

#include <seastar/core/future.hh>
Expand All @@ -29,9 +25,6 @@
#include <seastar/http/httpd.hh>
#include <seastar/util/tmp_file.hh>

#include <chrono>
#include <exception>
#include <map>
#include <vector>

namespace archival {
Expand Down
2 changes: 0 additions & 2 deletions src/v/archival/types.h
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,6 @@
#include <seastar/core/io_priority_class.hh>
#include <seastar/core/scheduling.hh>

#include <filesystem>

namespace archival {

using cloud_storage::connection_limit;
Expand Down
10 changes: 0 additions & 10 deletions src/v/cloud_storage/remote_partition.h
Original file line number Diff line number Diff line change
Expand Up @@ -11,20 +11,13 @@
#pragma once

#include "cloud_storage/fwd.h"
#include "cloud_storage/logger.h"
#include "cloud_storage/offset_translation_layer.h"
#include "cloud_storage/partition_manifest.h"
#include "cloud_storage/read_path_probes.h"
#include "cloud_storage/remote.h"
#include "cloud_storage/remote_segment.h"
#include "cloud_storage/segment_state.h"
#include "cloud_storage/types.h"
#include "cloud_storage_clients/types.h"
#include "container/intrusive_list_helpers.h"
#include "model/fundamental.h"
#include "model/metadata.h"
#include "raft/types.h"
#include "storage/ntp_config.h"
#include "storage/translating_reader.h"
#include "storage/types.h"
#include "utils/retry_chain_node.h"
Expand All @@ -38,9 +31,6 @@
#include <boost/iterator/iterator_categories.hpp>
#include <boost/iterator/iterator_facade.hpp>

#include <chrono>
#include <functional>

namespace cloud_storage {

using namespace std::chrono_literals;
Expand Down
1 change: 0 additions & 1 deletion src/v/cluster/archival_metadata_stm.h
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@
#include "cluster/state_machine_registry.h"
#include "features/fwd.h"
#include "model/fundamental.h"
#include "model/metadata.h"
#include "model/record.h"
#include "raft/persisted_stm.h"
#include "storage/record_batch_builder.h"
Expand Down
4 changes: 2 additions & 2 deletions src/v/cluster/bootstrap_types.h
Original file line number Diff line number Diff line change
Expand Up @@ -10,9 +10,9 @@

#include "cluster/types.h"
#include "model/fundamental.h"
#include "serde/serde.h"
#include "serde/envelope.h"

#include <fmt/core.h>
#include <fmt/ostream.h>

#include <vector>

Expand Down
2 changes: 1 addition & 1 deletion src/v/cluster/cloud_metadata/offsets_recoverer.cc
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,11 @@
#include "cloud_storage/remote_file.h"
#include "cluster/cloud_metadata/offsets_lookup_batcher.h"
#include "cluster/cloud_metadata/offsets_recovery_rpc_types.h"
#include "cluster/logger.h"
#include "kafka/protocol/errors.h"
#include "kafka/server/group_manager.h"
#include "model/metadata.h"
#include "model/namespace.h"
#include "model/record_batch_reader.h"
#include "rpc/connection_cache.h"

#include <seastar/core/lowres_clock.hh>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@
#include "cluster/tests/topic_properties_generator.h"
#include "cluster/types.h"
#include "model/fundamental.h"
#include "raft/consensus.h"
#include "security/acl.h"
#include "storage/types.h"

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
#include "cluster/feature_manager.h"
#include "cluster/id_allocator_frontend.h"
#include "cluster/partition.h"
#include "cluster/security_frontend.h"
#include "cluster/tests/topic_properties_generator.h"
#include "cluster/tests/tx_compaction_utils.h"
#include "cluster/types.h"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
#include "cluster/cluster_recovery_reconciler.h"
#include "cluster/controller_snapshot.h"
#include "cluster/feature_manager.h"
#include "cluster/security_frontend.h"
#include "cluster/tests/topic_properties_generator.h"
#include "cluster/types.h"
#include "config/configuration.h"
Expand Down
1 change: 1 addition & 0 deletions src/v/cluster/cloud_storage_size_reducer.cc
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
#include "cluster/cloud_storage_size_reducer.h"

#include "cluster/controller_service.h"
#include "cluster/logger.h"
#include "cluster/members_table.h"
#include "cluster/partition_leaders_table.h"

Expand Down
1 change: 0 additions & 1 deletion src/v/cluster/cluster_discovery.h
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@
#include "model/metadata.h"
#include "model/timeout_clock.h"
#include "random/simple_time_jitter.h"
#include "utils/mutex.h"

#include <seastar/core/future.hh>

Expand Down
5 changes: 0 additions & 5 deletions src/v/cluster/cluster_recovery_reconciler.h
Original file line number Diff line number Diff line change
Expand Up @@ -10,15 +10,10 @@
*/
#pragma once

#include "cloud_storage/topic_manifest.h"
#include "cluster/cluster_recovery_table.h"
#include "cluster/security_frontend.h"
#include "cluster/topic_table.h"
#include "cluster/types.h"
#include "config/config_store.h"
#include "features/feature_table.h"
#include "model/fundamental.h"
#include "security/acl_store.h"
#include "security/credential_store.h"

#include <absl/container/flat_hash_map.h>
Expand Down
1 change: 0 additions & 1 deletion src/v/cluster/cluster_recovery_state.h
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@
#include "cluster/types.h"
#include "model/fundamental.h"
#include "serde/envelope.h"
#include "serde/serde.h"

#include <seastar/util/bool_class.hh>

Expand Down
1 change: 1 addition & 0 deletions src/v/cluster/cluster_recovery_table.h
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@

#include "cluster/cloud_metadata/cluster_manifest.h"
#include "cluster/cluster_recovery_state.h"
#include "cluster/commands.h"
#include "cluster/controller_snapshot.h"
#include "cluster/types.h"
#include "model/fundamental.h"
Expand Down
3 changes: 0 additions & 3 deletions src/v/cluster/cluster_utils.h
Original file line number Diff line number Diff line change
Expand Up @@ -13,13 +13,10 @@
#include "cluster/controller_stm.h"
#include "cluster/errc.h"
#include "cluster/fwd.h"
#include "cluster/logger.h"
#include "cluster/members_table.h"
#include "cluster/types.h"
#include "config/node_config.h"
#include "config/tls_config.h"
#include "net/dns.h"
#include "outcome_future_utils.h"
#include "rpc/connection_cache.h"
#include "rpc/rpc_utils.h"
#include "rpc/types.h"
Expand Down
1 change: 0 additions & 1 deletion src/v/cluster/cluster_uuid.h
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@

#pragma once

#include "bytes/bytes.h"
#include "storage/kvstore.h"

#include <seastar/core/sharded.hh>
Expand Down
2 changes: 0 additions & 2 deletions src/v/cluster/commands.h
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@

#pragma once
#include "bytes/iobuf_parser.h"
#include "cluster/cloud_metadata/cluster_manifest.h"
#include "cluster/simple_batch_builder.h"
#include "cluster/types.h"
#include "model/metadata.h"
Expand All @@ -23,7 +22,6 @@
#include "security/role.h"
#include "security/scram_credential.h"
#include "security/types.h"
#include "serde/serde.h"
#include "utils/named_type.h"

#include <seastar/core/coroutine.hh>
Expand Down
1 change: 1 addition & 0 deletions src/v/cluster/config_frontend.cc
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@

#include "cluster/cluster_utils.h"
#include "cluster/controller_service.h"
#include "cluster/logger.h"
#include "cluster/partition_leaders_table.h"

namespace cluster {
Expand Down
3 changes: 1 addition & 2 deletions src/v/cluster/controller_backend.h
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,7 @@
#include <absl/container/node_hash_map.h>

#include <cstdint>
#include <deque>
#include <ostream>
#include <iosfwd>

namespace cluster {

Expand Down
2 changes: 0 additions & 2 deletions src/v/cluster/controller_log_limiter.h
Original file line number Diff line number Diff line change
Expand Up @@ -11,9 +11,7 @@

#pragma once

#include "base/vlog.h"
#include "cluster/commands.h"
#include "config/configuration.h"
#include "metrics/metrics.h"
#include "utils/token_bucket.h"

Expand Down
1 change: 0 additions & 1 deletion src/v/cluster/controller_probe.h
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@

#pragma once

#include "base/seastarx.h"
#include "cluster/fwd.h"
#include "cluster/types.h"
#include "metrics/metrics.h"
Expand Down
4 changes: 0 additions & 4 deletions src/v/cluster/controller_snapshot.h
Original file line number Diff line number Diff line change
Expand Up @@ -12,14 +12,10 @@
#pragma once

#include "cluster/cluster_recovery_state.h"
#include "cluster/config_manager.h"
#include "cluster/types.h"
#include "container/fragmented_vector.h"
#include "features/feature_table_snapshot.h"
#include "security/scram_credential.h"
#include "security/types.h"
#include "serde/envelope.h"
#include "serde/serde.h"

#include <absl/container/btree_map.h>
#include <absl/container/flat_hash_map.h>
Expand Down
4 changes: 4 additions & 0 deletions src/v/cluster/distributed_kv_stm.h
Original file line number Diff line number Diff line change
Expand Up @@ -11,8 +11,12 @@

#pragma once

#include "base/outcome.h"
#include "base/units.h"
#include "cluster/errc.h"
#include "cluster/logger.h"
#include "distributed_kv_stm_types.h"
#include "raft/consensus.h"
#include "raft/persisted_stm.h"
#include "utils/fixed_string.h"

Expand Down
2 changes: 1 addition & 1 deletion src/v/cluster/drain_manager.cc
Original file line number Diff line number Diff line change
@@ -1,9 +1,9 @@
#include "cluster/drain_manager.h"

#include "base/vlog.h"
#include "cluster/controller_service.h"
#include "cluster/logger.h"
#include "cluster/partition_manager.h"
#include "cluster/types.h"
#include "random/generators.h"

#include <seastar/core/lowres_clock.hh>
Expand Down
2 changes: 1 addition & 1 deletion src/v/cluster/drain_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
#include "base/seastarx.h"
#include "cluster/fwd.h"
#include "reflection/adl.h"
#include "serde/serde.h"
#include "serde/envelope.h"
#include "ssx/semaphore.h"

#include <seastar/core/abort_source.hh>
Expand Down
1 change: 0 additions & 1 deletion src/v/cluster/ephemeral_credential_frontend.h
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@
#include "cluster/errc.h"
#include "features/fwd.h"
#include "model/metadata.h"
#include "model/timeout_clock.h"
#include "rpc/fwd.h"
#include "security/ephemeral_credential.h"
#include "security/fwd.h"
Expand Down
3 changes: 0 additions & 3 deletions src/v/cluster/ephemeral_credential_serde.h
Original file line number Diff line number Diff line change
Expand Up @@ -12,13 +12,10 @@

#include "cluster/errc.h"
#include "security/acl.h"
#include "security/ephemeral_credential.h"
#include "security/scram_credential.h"
#include "security/types.h"
#include "serde/envelope.h"

#include <iosfwd>

namespace cluster {

struct put_ephemeral_credential_request
Expand Down
2 changes: 1 addition & 1 deletion src/v/cluster/feature_backend.h
Original file line number Diff line number Diff line change
Expand Up @@ -13,11 +13,11 @@

#include "cluster/commands.h"
#include "cluster/fwd.h"
#include "cluster/types.h"
#include "features/fwd.h"
#include "storage/fwd.h"

#include <seastar/core/future.hh>
#include <seastar/core/sharded.hh>

#include <system_error>

Expand Down
1 change: 1 addition & 0 deletions src/v/cluster/feature_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
#include "cluster/commands.h"
#include "cluster/controller_service.h"
#include "cluster/health_monitor_frontend.h"
#include "cluster/logger.h"
#include "cluster/members_table.h"
#include "config/configuration.h"
#include "config/validators.h"
Expand Down
Loading

0 comments on commit 0ecbec8

Please sign in to comment.