-
Notifications
You must be signed in to change notification settings - Fork 593
/
partition.cc
1326 lines (1161 loc) · 45 KB
/
partition.cc
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2020 Redpanda Data, Inc.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.md
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0
#include "cluster/partition.h"
#include "archival/ntp_archiver_service.h"
#include "archival/upload_housekeeping_service.h"
#include "cloud_storage/async_manifest_view.h"
#include "cloud_storage/read_path_probes.h"
#include "cloud_storage/remote_partition.h"
#include "cluster/logger.h"
#include "cluster/tm_stm_cache_manager.h"
#include "cluster/types.h"
#include "config/configuration.h"
#include "features/feature_table.h"
#include "model/fundamental.h"
#include "model/metadata.h"
#include "model/namespace.h"
#include "prometheus/prometheus_sanitize.h"
#include "raft/types.h"
#include <seastar/coroutine/as_future.hh>
#include <seastar/util/defer.hh>
#include <exception>
namespace cluster {
static bool is_id_allocator_topic(model::ntp ntp) {
return ntp.ns == model::kafka_internal_namespace
&& ntp.tp.topic == model::id_allocator_topic;
}
static bool is_tx_manager_topic(const model::ntp& ntp) {
return ntp.ns == model::kafka_internal_namespace
&& ntp.tp.topic == model::tx_manager_topic;
}
partition::partition(
consensus_ptr r,
ss::sharded<cluster::tx_gateway_frontend>& tx_gateway_frontend,
ss::sharded<cloud_storage::remote>& cloud_storage_api,
ss::sharded<cloud_storage::cache>& cloud_storage_cache,
ss::lw_shared_ptr<const archival::configuration> archival_conf,
ss::sharded<features::feature_table>& feature_table,
ss::sharded<cluster::tm_stm_cache_manager>& tm_stm_cache_manager,
ss::sharded<archival::upload_housekeeping_service>& upload_hks,
storage::kvstore& kvstore,
config::binding<uint64_t> max_concurrent_producer_ids,
std::optional<cloud_storage_clients::bucket_name> read_replica_bucket)
: _raft(r)
, _partition_mem_tracker(
ss::make_shared<util::mem_tracker>(_raft->ntp().path()))
, _probe(std::make_unique<replicated_partition_probe>(*this))
, _tx_gateway_frontend(tx_gateway_frontend)
, _feature_table(feature_table)
, _tm_stm_cache_manager(tm_stm_cache_manager)
, _is_tx_enabled(config::shard_local_cfg().enable_transactions.value())
, _is_idempotence_enabled(
config::shard_local_cfg().enable_idempotence.value())
, _archival_conf(archival_conf)
, _cloud_storage_api(cloud_storage_api)
, _cloud_storage_cache(cloud_storage_cache)
, _cloud_storage_probe(
ss::make_shared<cloud_storage::partition_probe>(_raft->ntp()))
, _upload_housekeeping(upload_hks)
, _kvstore(kvstore) {
auto stm_manager = _raft->log()->stm_manager();
if (is_id_allocator_topic(_raft->ntp())) {
_id_allocator_stm = ss::make_shared<cluster::id_allocator_stm>(
clusterlog, _raft.get());
} else if (is_tx_manager_topic(_raft->ntp())) {
if (
_raft->log_config().is_collectable()
&& !storage::deletion_exempt(_raft->ntp())) {
_log_eviction_stm = ss::make_shared<cluster::log_eviction_stm>(
_raft.get(), clusterlog, _as, _kvstore);
stm_manager->add_stm(_log_eviction_stm);
}
if (_is_tx_enabled) {
auto tm_stm_cache = _tm_stm_cache_manager.local().get(
_raft->ntp().tp.partition);
_tm_stm = ss::make_shared<cluster::tm_stm>(
clusterlog, _raft.get(), feature_table, tm_stm_cache);
stm_manager->add_stm(_tm_stm);
}
} else {
if (
_raft->log_config().is_collectable()
&& !storage::deletion_exempt(_raft->ntp())) {
_log_eviction_stm = ss::make_shared<cluster::log_eviction_stm>(
_raft.get(), clusterlog, _as, _kvstore);
stm_manager->add_stm(_log_eviction_stm);
}
const model::topic_namespace tp_ns(
_raft->ntp().ns, _raft->ntp().tp.topic);
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()
&& !is_group_ntp;
if (has_rm_stm) {
_rm_stm = ss::make_shared<cluster::rm_stm>(
clusterlog,
_raft.get(),
_tx_gateway_frontend,
_feature_table,
max_concurrent_producer_ids);
stm_manager->add_stm(_rm_stm);
}
// Construct cloud_storage read path (remote_partition)
if (
config::shard_local_cfg().cloud_storage_enabled()
&& _cloud_storage_api.local_is_initialized()
&& _raft->ntp().ns == model::kafka_namespace) {
_archival_meta_stm
= ss::make_shared<cluster::archival_metadata_stm>(
_raft.get(),
_cloud_storage_api.local(),
_feature_table.local(),
clusterlog,
_partition_mem_tracker);
stm_manager->add_stm(_archival_meta_stm);
if (cloud_storage_cache.local_is_initialized()) {
const auto& bucket_config
= cloud_storage::configuration::get_bucket_config();
auto bucket = bucket_config.value();
if (
read_replica_bucket
&& _raft->log_config().is_read_replica_mode_enabled()) {
vlog(
clusterlog.info,
"{} Remote topic bucket is {}",
_raft->ntp(),
read_replica_bucket);
// Override the bucket for read replicas
_read_replica_bucket = read_replica_bucket;
bucket = read_replica_bucket;
}
if (!bucket) {
throw std::runtime_error{fmt::format(
"configuration property {} is not set",
bucket_config.name())};
}
_cloud_storage_manifest_view
= ss::make_shared<cloud_storage::async_manifest_view>(
_cloud_storage_api,
cloud_storage_cache,
_archival_meta_stm->manifest(),
cloud_storage_clients::bucket_name{*bucket});
_cloud_storage_partition
= ss::make_shared<cloud_storage::remote_partition>(
_cloud_storage_manifest_view,
_cloud_storage_api.local(),
cloud_storage_cache.local(),
cloud_storage_clients::bucket_name{*bucket},
*_cloud_storage_probe);
}
}
// Construct cloud_storage write path (ntp_archiver)
maybe_construct_archiver();
}
}
partition::~partition() {}
ss::future<std::error_code> partition::prefix_truncate(
model::offset rp_start_offset,
kafka::offset kafka_start_offset,
ss::lowres_clock::time_point deadline) {
if (!_log_eviction_stm) {
vlog(
clusterlog.info,
"Cannot prefix-truncate topic/partition {} retention settings not "
"applied",
_raft->ntp());
co_return make_error_code(errc::topic_invalid_config);
}
if (!feature_table().local().is_active(features::feature::delete_records)) {
vlog(
clusterlog.info,
"Cannot prefix-truncate topic/partition {} feature is currently "
"disabled",
_raft->ntp());
co_return make_error_code(cluster::errc::feature_disabled);
}
vlog(
clusterlog.info,
"Truncating {} to redpanda offset {} kafka offset {}",
_raft->ntp(),
rp_start_offset,
kafka_start_offset);
auto res = co_await _log_eviction_stm->truncate(
rp_start_offset, kafka_start_offset, deadline, _as);
if (res.has_error()) {
co_return res.error();
}
if (_archival_meta_stm) {
// The archival metadata stm also listens for prefix_truncate batches.
auto truncate_batch_offset = res.value();
auto applied = co_await _archival_meta_stm->wait_no_throw(
truncate_batch_offset, deadline, _as);
if (applied) {
co_return errc::success;
}
if (_as.abort_requested()) {
co_return errc::shutting_down;
}
co_return errc::timeout;
}
co_return errc::success;
}
ss::future<std::vector<rm_stm::tx_range>> partition::aborted_transactions_cloud(
const cloud_storage::offset_range& offsets) {
return _cloud_storage_partition->aborted_transactions(offsets);
}
cluster::cloud_storage_mode partition::get_cloud_storage_mode() const {
if (!config::shard_local_cfg().cloud_storage_enabled()) {
return cluster::cloud_storage_mode::disabled;
}
const auto& cfg = _raft->log_config();
if (cfg.is_read_replica_mode_enabled()) {
return cluster::cloud_storage_mode::read_replica;
}
if (cfg.is_tiered_storage()) {
return cluster::cloud_storage_mode::full;
}
if (cfg.is_archival_enabled()) {
return cluster::cloud_storage_mode::write_only;
}
if (cfg.is_remote_fetch_enabled()) {
return cluster::cloud_storage_mode::read_only;
}
return cluster::cloud_storage_mode::disabled;
}
partition_cloud_storage_status partition::get_cloud_storage_status() const {
auto wrap_model_offset =
[this](model::offset o) -> std::optional<kafka::offset> {
if (o == model::offset{}) {
return std::nullopt;
}
return model::offset_cast(
get_offset_translator_state()->from_log_offset(o));
};
auto time_point_to_delta = [](ss::lowres_clock::time_point tp)
-> std::optional<std::chrono::milliseconds> {
if (tp.time_since_epoch().count() == 0) {
return std::nullopt;
}
return std::chrono::duration_cast<std::chrono::milliseconds>(
ss::lowres_clock::now() - tp);
};
partition_cloud_storage_status status;
status.mode = get_cloud_storage_mode();
const auto local_log = _raft->log();
status.local_log_size_bytes = local_log->size_bytes();
status.local_log_segment_count = local_log->segment_count();
const auto local_log_offsets = local_log->offsets();
status.local_log_start_offset = wrap_model_offset(
local_log_offsets.start_offset);
status.local_log_last_offset = wrap_model_offset(
local_log_offsets.committed_offset);
if (status.mode != cloud_storage_mode::disabled && _archival_meta_stm) {
const auto& manifest = _archival_meta_stm->manifest();
status.cloud_metadata_update_pending
= _archival_meta_stm->get_dirty()
== archival_metadata_stm::state_dirty::dirty;
status.cloud_log_size_bytes = manifest.cloud_log_size();
status.stm_region_size_bytes = manifest.stm_region_size_bytes();
status.archive_size_bytes = manifest.archive_size_bytes();
status.stm_region_segment_count = manifest.size();
if (manifest.size() > 0) {
status.cloud_log_start_offset
= manifest.full_log_start_kafka_offset();
status.stm_region_start_offset = manifest.get_start_kafka_offset();
status.cloud_log_last_offset = manifest.get_last_kafka_offset();
}
// Calculate local space usage that does not overlap with cloud space
const auto local_space_excl = status.cloud_log_last_offset
? _raft->log()->size_bytes_after_offset(
manifest.get_last_offset())
: status.local_log_size_bytes;
status.total_log_size_bytes = status.cloud_log_size_bytes
+ local_space_excl;
} else {
status.total_log_size_bytes = status.local_log_size_bytes;
}
if (is_leader() && _archiver) {
if (
status.mode == cloud_storage_mode::write_only
|| status.mode == cloud_storage_mode::full) {
status.since_last_manifest_upload = time_point_to_delta(
_archiver->get_last_manfiest_upload_time());
status.since_last_segment_upload = time_point_to_delta(
_archiver->get_last_segment_upload_time());
} else if (status.mode == cloud_storage_mode::read_replica) {
const auto last_sync_at = _archiver->get_last_sync_time();
if (last_sync_at) {
status.since_last_manifest_sync = time_point_to_delta(
*last_sync_at);
} else {
status.since_last_manifest_sync = std::nullopt;
}
}
}
return status;
}
bool partition::is_remote_fetch_enabled() const {
const auto& cfg = _raft->log_config();
if (_feature_table.local().is_active(features::feature::cloud_retention)) {
// Since 22.3, the ntp_config is authoritative.
return cfg.is_remote_fetch_enabled();
} else {
// We are in the process of an upgrade: apply <22.3 behavior of acting
// as if every partition has remote read enabled if the cluster
// default is true.
return cfg.is_remote_fetch_enabled()
|| config::shard_local_cfg().cloud_storage_enable_remote_read();
}
}
bool partition::cloud_data_available() const {
return static_cast<bool>(_cloud_storage_partition)
&& _cloud_storage_partition->is_data_available();
}
std::optional<uint64_t> partition::cloud_log_size() const {
if (_cloud_storage_partition == nullptr) {
return std::nullopt;
}
return _cloud_storage_partition->cloud_log_size();
}
model::offset partition::start_cloud_offset() const {
vassert(
cloud_data_available(),
"Method can only be called if cloud data is available, ntp: {}",
_raft->ntp());
return kafka::offset_cast(
_cloud_storage_partition->first_uploaded_offset());
}
model::offset partition::next_cloud_offset() const {
vassert(
cloud_data_available(),
"Method can only be called if cloud data is available, ntp: {}",
_raft->ntp());
return kafka::offset_cast(_cloud_storage_partition->next_kafka_offset());
}
ss::future<storage::translating_reader> partition::make_cloud_reader(
storage::log_reader_config config,
std::optional<model::timeout_clock::time_point> deadline) {
vassert(
cloud_data_available(),
"Method can only be called if cloud data is available, ntp: {}",
_raft->ntp());
return _cloud_storage_partition->make_reader(config, deadline);
}
ss::future<result<kafka_result>> partition::replicate(
model::record_batch_reader&& r, raft::replicate_options opts) {
using ret_t = result<kafka_result>;
auto res = co_await _raft->replicate(std::move(r), opts);
if (!res) {
co_return ret_t(res.error());
}
co_return ret_t(
kafka_result{kafka::offset(get_offset_translator_state()->from_log_offset(
res.value().last_offset)())});
}
ss::shared_ptr<cluster::rm_stm> partition::rm_stm() {
if (!_rm_stm) {
if (!_is_tx_enabled && !_is_idempotence_enabled) {
vlog(
clusterlog.error,
"Can't process transactional and idempotent requests to {}. The "
"feature is disabled.",
_raft->ntp());
} else {
vlog(
clusterlog.error,
"Topic {} doesn't support idempotency and transactional "
"processing.",
_raft->ntp());
}
}
return _rm_stm;
}
kafka_stages partition::replicate_in_stages(
model::batch_identity bid,
model::record_batch_reader&& r,
raft::replicate_options opts) {
using ret_t = result<kafka_result>;
if (bid.is_transactional) {
if (!_is_tx_enabled) {
vlog(
clusterlog.error,
"Can't process a transactional request to {}. Transactional "
"processing isn't enabled.",
_raft->ntp());
return kafka_stages(raft::errc::timeout);
}
if (!_rm_stm) {
vlog(
clusterlog.error,
"Topic {} doesn't support transactional processing.",
_raft->ntp());
return kafka_stages(raft::errc::timeout);
}
}
if (bid.has_idempotent()) {
if (!_is_idempotence_enabled) {
vlog(
clusterlog.error,
"Can't process an idempotent request to {}. Idempotency isn't "
"enabled.",
_raft->ntp());
return kafka_stages(raft::errc::timeout);
}
if (!_rm_stm) {
vlog(
clusterlog.error,
"Topic {} doesn't support idempotency.",
_raft->ntp());
return kafka_stages(raft::errc::timeout);
}
}
if (_rm_stm) {
return _rm_stm->replicate_in_stages(bid, std::move(r), opts);
}
auto res = _raft->replicate_in_stages(std::move(r), opts);
auto replicate_finished = res.replicate_finished.then(
[this](result<raft::replicate_result> r) {
if (!r) {
return ret_t(r.error());
}
auto old_offset = r.value().last_offset;
auto new_offset = kafka::offset(
get_offset_translator_state()->from_log_offset(old_offset)());
return ret_t(kafka_result{new_offset});
});
return kafka_stages(
std::move(res.request_enqueued), std::move(replicate_finished));
}
ss::future<> partition::start() {
auto ntp = _raft->ntp();
_probe.setup_metrics(ntp);
auto f = _raft->start();
if (is_id_allocator_topic(ntp)) {
return f.then([this] { return _id_allocator_stm->start(); });
} else if (_log_eviction_stm) {
f = f.then([this] { return _log_eviction_stm->start(); });
}
if (_rm_stm) {
f = f.then([this] { return _rm_stm->start(); });
}
if (_tm_stm) {
f = f.then([this] { return _tm_stm->start(); });
}
if (_archival_meta_stm) {
f = f.then([this] { return _archival_meta_stm->start(); });
}
if (_cloud_storage_manifest_view) {
f = f.then([this] { return _cloud_storage_manifest_view->start(); });
}
if (_cloud_storage_partition) {
f = f.then([this] { return _cloud_storage_partition->start(); });
}
if (_archiver) {
f = f.then([this] { return _archiver->start(); });
}
return f;
}
ss::future<> partition::stop() {
auto partition_ntp = ntp();
vlog(clusterlog.debug, "Stopping partition: {}", partition_ntp);
_as.request_abort();
{
// `partition_manager::do_shutdown` (caller of stop) will assert
// out on any thrown exceptions. Hence, acquire the units without
// a timeout or abort source.
auto archiver_reset_guard = co_await ss::get_units(
_archiver_reset_mutex, 1);
if (_archiver) {
_upload_housekeeping.local().deregister_jobs(
_archiver->get_housekeeping_jobs());
vlog(
clusterlog.debug,
"Stopping archiver on partition: {}",
partition_ntp);
co_await _archiver->stop();
}
}
if (_archival_meta_stm) {
vlog(
clusterlog.debug,
"Stopping archival_meta_stm on partition: {}",
partition_ntp);
co_await _archival_meta_stm->stop();
}
if (_cloud_storage_partition) {
vlog(
clusterlog.debug,
"Stopping cloud_storage_partition on partition: {}",
partition_ntp);
co_await _cloud_storage_partition->stop();
}
if (_cloud_storage_manifest_view) {
vlog(
clusterlog.debug,
"Stopping cloud_storage_manifest_view on partition: {}",
partition_ntp);
co_await _cloud_storage_manifest_view->stop();
}
if (_id_allocator_stm) {
vlog(
clusterlog.debug,
"Stopping id_allocator_stm on partition: {}",
partition_ntp);
co_await _id_allocator_stm->stop();
}
if (_log_eviction_stm) {
vlog(
clusterlog.debug,
"Stopping log_eviction_stm on partition: {}",
partition_ntp);
co_await _log_eviction_stm->stop();
}
if (_rm_stm) {
vlog(
clusterlog.debug, "Stopping rm_stm on partition: {}", partition_ntp);
co_await _rm_stm->stop();
}
if (_tm_stm) {
vlog(
clusterlog.debug, "Stopping tm_stm on partition: {}", partition_ntp);
co_await _tm_stm->stop();
}
_probe.clear_metrics();
vlog(clusterlog.debug, "Stopped partition {}", partition_ntp);
}
ss::future<std::optional<storage::timequery_result>>
partition::timequery(storage::timequery_config cfg) {
// Read replicas never consider local raft data
if (_raft->log_config().is_read_replica_mode_enabled()) {
co_return co_await cloud_storage_timequery(cfg);
}
if (_raft->log()->start_timestamp() <= cfg.time) {
// The query is ahead of the local data's start_timestamp: this means
// it _might_ hit on local data: start_timestamp is not precise, so
// once we query we might still fall back to cloud storage
auto result = co_await local_timequery(cfg);
if (!result.has_value()) {
// The local storage hit a case where it needs to fall back
// to querying cloud storage.
co_return co_await cloud_storage_timequery(cfg);
} else {
co_return result;
}
} else {
if (
may_read_from_cloud()
&& _cloud_storage_partition->bounds_timestamp(cfg.time)) {
// Timestamp is before local storage but within cloud storage
co_return co_await cloud_storage_timequery(cfg);
} else {
// No cloud data: queries earlier than the start of the log
// will hit on the start of the log.
co_return co_await local_timequery(cfg);
}
}
}
bool partition::may_read_from_cloud() const {
return _cloud_storage_partition
&& _cloud_storage_partition->is_data_available();
}
ss::future<std::optional<storage::timequery_result>>
partition::cloud_storage_timequery(storage::timequery_config cfg) {
if (may_read_from_cloud()) {
// We have data in the remote partition, and all the data in the raft
// log is ahead of the query timestamp or the topic is a read replica,
// so proceed to query the remote partition to try and find the earliest
// data that has timestamp >= the query time.
vlog(
clusterlog.debug,
"timequery (cloud) {} t={} max_offset(k)={}",
_raft->ntp(),
cfg.time,
cfg.max_offset);
// remote_partition pre-translates offsets for us, so no call into
// the offset translator here
auto result = co_await _cloud_storage_partition->timequery(cfg);
if (result) {
vlog(
clusterlog.debug,
"timequery (cloud) {} t={} max_offset(r)={} result(r)={}",
_raft->ntp(),
cfg.time,
cfg.max_offset,
result->offset);
}
co_return result;
}
co_return std::nullopt;
}
ss::future<std::optional<storage::timequery_result>>
partition::local_timequery(storage::timequery_config cfg) {
vlog(
clusterlog.debug,
"timequery (raft) {} t={} max_offset(k)={}",
_raft->ntp(),
cfg.time,
cfg.max_offset);
cfg.max_offset = _raft->get_offset_translator_state()->to_log_offset(
cfg.max_offset);
auto result = co_await _raft->timequery(cfg);
bool may_answer_from_cloud = may_read_from_cloud()
&& _cloud_storage_partition->bounds_timestamp(
cfg.time);
if (result) {
if (
_raft->log()->start_timestamp() > cfg.time && may_answer_from_cloud) {
// Query raced with prefix truncation
vlog(
clusterlog.debug,
"timequery (raft) {} ts={} raced with truncation "
"(start_timestamp {}, result {})",
_raft->ntp(),
cfg.time,
_raft->log()->start_timestamp(),
result->time);
co_return std::nullopt;
}
if (
_raft->log()->start_timestamp() <= cfg.time && result->time > cfg.time
&& may_answer_from_cloud) {
// start_timestamp() points to the beginning of the oldest segment,
// but start_offset points to somewhere within a segment. If our
// timequery hits the range between the start of segment and
// the start_offset, consensus::timequery may answer with
// the start offset rather than the pre-start-offset location
// where the timestamp is actually found.
// Ref https://github.com/redpanda-data/redpanda/issues/9669
vlog(
clusterlog.debug,
"Timequery (raft) {} ts={} miss on local log (start_timestamp "
"{}, result {})",
_raft->ntp(),
cfg.time,
_raft->log()->start_timestamp(),
result->time);
co_return std::nullopt;
}
if (result->offset == _raft->log()->offsets().start_offset) {
// If we hit at the start of the local log, this is ambiguous:
// there could be earlier batches prior to start_offset which
// have the same timestamp and are present in cloud storage.
vlog(
clusterlog.debug,
"Timequery (raft) {} ts={} hit start_offset in local log "
"(start_offset {} start_timestamp {}, result {})",
_raft->ntp(),
_raft->log()->offsets().start_offset,
cfg.time,
_raft->log()->start_timestamp(),
cfg.time);
if (
_cloud_storage_partition
&& _cloud_storage_partition->is_data_available()
&& may_answer_from_cloud) {
// Even though we hit data with the desired timestamp, we cannot
// be certain that this is the _first_ batch with the desired
// timestamp: return null so that the caller will fall back
// to cloud storage.
co_return std::nullopt;
}
}
vlog(
clusterlog.debug,
"timequery (raft) {} t={} max_offset(r)={} result(r)={}",
_raft->ntp(),
cfg.time,
cfg.max_offset,
result->offset);
result->offset = _raft->get_offset_translator_state()->from_log_offset(
result->offset);
}
co_return result;
}
bool partition::should_construct_archiver() {
// NOTE: construct and archiver even if shadow indexing isn't enabled, e.g.
// in the case of read replicas -- we still need the archiver to drive
// manifest updates, etc.
const auto& ntp_config = _raft->log()->config();
return config::shard_local_cfg().cloud_storage_enabled()
&& _cloud_storage_api.local_is_initialized()
&& _raft->ntp().ns == model::kafka_namespace
&& (ntp_config.is_archival_enabled() || ntp_config.is_read_replica_mode_enabled());
}
void partition::maybe_construct_archiver() {
if (should_construct_archiver()) {
const auto& ntp_config = _raft->log()->config();
_archiver = std::make_unique<archival::ntp_archiver>(
ntp_config,
_archival_conf,
_cloud_storage_api.local(),
_cloud_storage_cache.local(),
*this,
_cloud_storage_manifest_view);
if (!ntp_config.is_read_replica_mode_enabled()) {
_upload_housekeeping.local().register_jobs(
_archiver->get_housekeeping_jobs());
}
}
}
uint64_t partition::non_log_disk_size_bytes() const {
uint64_t raft_size = _raft->get_snapshot_size();
std::optional<uint64_t> rm_size;
if (_rm_stm) {
rm_size = _rm_stm->get_snapshot_size();
}
std::optional<uint64_t> tm_size;
if (_tm_stm) {
tm_size = _tm_stm->get_snapshot_size();
}
std::optional<uint64_t> archival_size;
if (_archival_meta_stm) {
archival_size = _archival_meta_stm->get_snapshot_size();
}
std::optional<uint64_t> idalloc_size;
if (_id_allocator_stm) {
idalloc_size = _id_allocator_stm->get_snapshot_size();
}
vlog(
clusterlog.trace,
"non-log disk size: raft {} rm {} tm {} archival {} idalloc {}",
raft_size,
rm_size,
tm_size,
archival_size,
idalloc_size);
return raft_size + rm_size.value_or(0) + tm_size.value_or(0)
+ archival_size.value_or(0) + idalloc_size.value_or(0);
}
ss::future<> partition::update_configuration(topic_properties properties) {
auto& old_ntp_config = _raft->log()->config();
auto new_ntp_config = properties.get_ntp_cfg_overrides();
// Before applying change, consider whether it changes cloud storage
// mode
bool cloud_storage_changed = false;
bool new_archival = new_ntp_config.shadow_indexing_mode
&& model::is_archival_enabled(
new_ntp_config.shadow_indexing_mode.value());
bool new_compaction_status
= new_ntp_config.cleanup_policy_bitflags.has_value()
&& (new_ntp_config.cleanup_policy_bitflags.value()
& model::cleanup_policy_bitflags::compaction)
== model::cleanup_policy_bitflags::compaction;
if (
old_ntp_config.is_archival_enabled() != new_archival
|| old_ntp_config.is_read_replica_mode_enabled()
!= new_ntp_config.read_replica
|| old_ntp_config.is_compacted() != new_compaction_status) {
cloud_storage_changed = true;
}
// Pass the configuration update into the storage layer
co_await _raft->log()->update_configuration(new_ntp_config);
// Update cached instance of topic properties
if (_topic_cfg) {
_topic_cfg->properties = std::move(properties);
}
// If this partition's cloud storage mode changed, rebuild the archiver.
// This must happen after raft update, because it reads raft's
// ntp_config to decide whether to construct an archiver.
if (cloud_storage_changed) {
vlog(
clusterlog.debug,
"update_configuration[{}]: updating archiver for config {}",
new_ntp_config,
_raft->ntp());
auto archiver_reset_guard = co_await ssx::with_timeout_abortable(
ss::get_units(_archiver_reset_mutex, 1),
ss::lowres_clock::now() + archiver_reset_mutex_timeout,
_as);
if (_archiver) {
_upload_housekeeping.local().deregister_jobs(
_archiver->get_housekeeping_jobs());
co_await _archiver->stop();
_archiver = nullptr;
}
maybe_construct_archiver();
if (_archiver) {
_archiver->notify_topic_config();
co_await _archiver->start();
}
} else {
vlog(
clusterlog.trace,
"update_configuration[{}]: no cloud storage change, archiver "
"exists={}",
_raft->ntp(),
bool(_archiver));
if (_archiver) {
// Assume that a partition config may also mean a topic
// configuration change. This could be optimized by hooking
// in separate updates from the controller when our topic
// configuration changes.
_archiver->notify_topic_config();
}
}
}
std::optional<model::offset>
partition::get_term_last_offset(model::term_id term) const {
auto o = _raft->log()->get_term_last_offset(term);
if (!o) {
return std::nullopt;
}
// Kafka defines leader epoch last offset as a first offset of next
// leader epoch
return model::next_offset(*o);
}
ss::future<std::optional<model::offset>>
partition::get_cloud_term_last_offset(model::term_id term) const {
auto o = co_await _cloud_storage_partition->get_term_last_offset(term);
if (!o) {
co_return std::nullopt;
}
// Kafka defines leader epoch last offset as a first offset of next
// leader epoch
co_return model::next_offset(kafka::offset_cast(*o));
}
ss::future<> partition::remove_persistent_state() {
if (_rm_stm) {
co_await _rm_stm->remove_persistent_state();
}
if (_tm_stm) {
co_await _tm_stm->remove_persistent_state();
}
if (_archival_meta_stm) {
co_await _archival_meta_stm->remove_persistent_state();
}
if (_id_allocator_stm) {
co_await _id_allocator_stm->remove_persistent_state();
}
if (_log_eviction_stm) {
co_await _log_eviction_stm->remove_persistent_state();
}
}
/**
* Return the index of this node in the list of voters, or nullopt if it
* is not a voter.
*/
static std::optional<size_t>
voter_position(raft::vnode self, const raft::group_configuration& raft_config) {
const auto& voters = raft_config.current_config().voters;
auto position = std::find(voters.begin(), voters.end(), self);
if (position == voters.end()) {
return std::nullopt;
} else {
return position - voters.begin();
}
}
// To reduce redundant re-uploads in the typical case of all replicas
// being alive, have all non-0th replicas delay before attempting to
// reconcile the manifest. This is just a best-effort thing, it is
// still okay for them to step on each other: finalization is best
// effort and the worst case outcome is to leave behind a few orphan
// objects if writes were ongoing while deletion happened.
static ss::future<bool> should_finalize(
ss::abort_source& as,
raft::vnode self,
const raft::group_configuration& raft_config) {
static constexpr ss::lowres_clock::duration erase_non_0th_delay = 200ms;
auto my_position = voter_position(self, raft_config);
if (my_position.has_value()) {
auto p = my_position.value();
if (p != 0) {
co_await ss::sleep_abortable(erase_non_0th_delay * p, as);
}
co_return true;
} else {
co_return false;
}
}
ss::future<> partition::finalize_remote_partition(ss::abort_source& as) {
if (!_feature_table.local().is_active(
features::feature::cloud_storage_manifest_format_v2)) {
// this is meant to prevent uploading manifests with new format while
// the cluster is in a mixed state
vlog(
clusterlog.info, "skipping finalize of remote partition {}", ntp());
co_return;
}
const bool tiered_storage = get_ntp_config().is_archival_enabled();