Skip to content

Commit abb577e

Browse files
CalvinNeoti-chi-bot[bot]JaySon-Huang
authored
UniPS: Catch exception when dumping incr snapshot (#9407)
close #9406 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> Co-authored-by: JaySon <[email protected]>
1 parent b30c1f5 commit abb577e

11 files changed

+160
-41
lines changed

Diff for: dbms/src/Common/FailPoint.cpp

+2-1
Original file line numberDiff line numberDiff line change
@@ -115,7 +115,8 @@ namespace DB
115115
M(delta_tree_create_node_fail) \
116116
M(disable_flush_cache) \
117117
M(force_agg_two_level_hash_table_before_merge) \
118-
M(force_thread_0_no_agg_spill)
118+
M(force_thread_0_no_agg_spill) \
119+
M(force_checkpoint_dump_throw_datafile)
119120

120121
#define APPLY_FOR_PAUSEABLE_FAILPOINTS_ONCE(M) \
121122
M(pause_with_alter_locks_acquired) \

Diff for: dbms/src/Storages/KVStore/tests/gtest_kvstore_fast_add_peer.cpp

+30
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,7 @@ namespace FailPoints
4444
extern const char force_fap_worker_throw[];
4545
extern const char force_set_fap_candidate_store_id[];
4646
extern const char force_not_clean_fap_on_destroy[];
47+
extern const char force_checkpoint_dump_throw_datafile[];
4748
} // namespace FailPoints
4849

4950
namespace tests
@@ -577,6 +578,35 @@ try
577578
}
578579
CATCH
579580

581+
582+
TEST_F(RegionKVStoreTestFAP, DumpCheckpointError)
583+
try
584+
{
585+
auto & global_context = TiFlashTestEnv::getGlobalContext();
586+
uint64_t region_id = 1;
587+
auto peer_id = 1;
588+
KVStore & kvs = getKVS();
589+
auto page_storage = global_context.getWriteNodePageStorage();
590+
591+
proxy_instance->bootstrapWithRegion(kvs, global_context.getTMTContext(), region_id, std::nullopt);
592+
auto region = proxy_instance->getRegion(region_id);
593+
auto store_id = kvs.getStore().store_id.load();
594+
region->addPeer(store_id, peer_id, metapb::PeerRole::Learner);
595+
596+
// Write some data, and persist meta.
597+
auto [index, term]
598+
= proxy_instance->normalWrite(region_id, {34}, {"v2"}, {WriteCmdType::Put}, {ColumnFamilyType::Default});
599+
kvs.setRegionCompactLogConfig(0, 0, 0, 0);
600+
persistAfterWrite(global_context, kvs, proxy_instance, page_storage, region_id, index);
601+
602+
auto s3_client = S3::ClientFactory::instance().sharedTiFlashClient();
603+
ASSERT_TRUE(::DB::tests::TiFlashTestEnv::createBucketIfNotExist(*s3_client));
604+
FailPointHelper::enableFailPoint(FailPoints::force_checkpoint_dump_throw_datafile);
605+
EXPECT_NO_THROW(dumpCheckpoint());
606+
FailPointHelper::disableFailPoint(FailPoints::force_checkpoint_dump_throw_datafile);
607+
}
608+
CATCH
609+
580610
// Test cancel from peer select
581611
TEST_F(RegionKVStoreTestFAP, Cancel1)
582612
try

Diff for: dbms/src/Storages/KVStore/tests/gtest_new_kvstore.cpp

-1
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,6 @@
3333
#include <limits>
3434

3535
extern std::shared_ptr<MemoryTracker> root_of_kvstore_mem_trackers;
36-
3736
namespace DB::tests
3837
{
3938

Diff for: dbms/src/Storages/Page/V3/CheckpointFile/CPFilesWriter.cpp

+25-2
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,8 @@
1212
// See the License for the specific language governing permissions and
1313
// limitations under the License.
1414

15+
#include <Common/Stopwatch.h>
16+
#include <Poco/File.h>
1517
#include <Storages/Page/V3/CheckpointFile/CPDumpStat.h>
1618
#include <Storages/Page/V3/CheckpointFile/CPFilesWriter.h>
1719
#include <Storages/Page/V3/PageEntriesEdit.h>
@@ -180,10 +182,15 @@ CPDataDumpStats CPFilesWriter::writeEditsAndApplyCheckpointInfo(
180182

181183
// 2. For entry edits without the checkpoint info, or it is stored on an existing data file that needs compact,
182184
// write the entry data to the data file, and assign a new checkpoint info.
185+
Stopwatch sw;
183186
try
184187
{
185188
auto page = data_source->read({rec_edit.page_id, rec_edit.entry});
186-
RUNTIME_CHECK_MSG(page.isValid(), "failed to read page, record={}", rec_edit);
189+
RUNTIME_CHECK_MSG(
190+
page.isValid(),
191+
"failed to read page, record={} elapsed={:.3f}s",
192+
rec_edit,
193+
sw.elapsedSeconds());
187194
auto data_location
188195
= data_writer->write(rec_edit.page_id, rec_edit.version, page.data.begin(), page.data.size());
189196
// the page data size uploaded in this checkpoint
@@ -207,7 +214,7 @@ CPDataDumpStats CPFilesWriter::writeEditsAndApplyCheckpointInfo(
207214
}
208215
catch (...)
209216
{
210-
LOG_ERROR(log, "failed to read page, record={}", rec_edit);
217+
LOG_ERROR(log, "failed to read and write page, record={} elapsed={:.3f}s", rec_edit, sw.elapsedSeconds());
211218
tryLogCurrentException(__PRETTY_FUNCTION__);
212219
throw;
213220
}
@@ -255,6 +262,7 @@ void CPFilesWriter::newDataWriter()
255262
fmt::runtime(data_file_path_pattern),
256263
fmt::arg("seq", sequence),
257264
fmt::arg("index", data_file_index)));
265+
258266
data_writer = CPDataFileWriter::create({
259267
.file_path = data_file_paths.back(),
260268
.file_id = fmt::format(
@@ -268,4 +276,19 @@ void CPFilesWriter::newDataWriter()
268276
++data_file_index;
269277
}
270278

279+
void CPFilesWriter::abort()
280+
{
281+
for (const auto & s : data_file_paths)
282+
{
283+
if (Poco::File f(s); f.exists())
284+
{
285+
f.remove();
286+
}
287+
}
288+
if likely (manifest_writer != nullptr)
289+
{
290+
manifest_writer->abort();
291+
}
292+
}
293+
271294
} // namespace DB::PS::V3

Diff for: dbms/src/Storages/Page/V3/CheckpointFile/CPFilesWriter.h

+2
Original file line numberDiff line numberDiff line change
@@ -101,6 +101,8 @@ class CPFilesWriter : private boost::noncopyable
101101
*/
102102
[[nodiscard]] std::vector<String> writeSuffix();
103103

104+
void abort();
105+
104106
#ifndef DBMS_PUBLIC_GTEST
105107
private:
106108
#else

Diff for: dbms/src/Storages/Page/V3/CheckpointFile/CPManifestFileWriter.cpp

+11
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@
1313
// limitations under the License.
1414

1515
#include <Common/Logger.h>
16+
#include <Poco/File.h>
1617
#include <Storages/Page/V3/CheckpointFile/CPManifestFileWriter.h>
1718
#include <Storages/Page/V3/CheckpointFile/ProtoHelper.h>
1819
#include <Storages/Page/V3/Universal/UniversalPageIdFormatImpl.h>
@@ -147,4 +148,14 @@ void CPManifestFileWriter::writeSuffix()
147148
write_stage = WriteStage::WritingFinished;
148149
}
149150

151+
void CPManifestFileWriter::abort()
152+
{
153+
if (options.file_path.empty())
154+
return;
155+
if (Poco::File f(options.file_path); f.exists())
156+
{
157+
f.remove();
158+
}
159+
}
160+
150161
} // namespace DB::PS::V3

Diff for: dbms/src/Storages/Page/V3/CheckpointFile/CPManifestFileWriter.h

+6-2
Original file line numberDiff line numberDiff line change
@@ -40,8 +40,9 @@ class CPManifestFileWriter : private boost::noncopyable
4040
return std::make_unique<CPManifestFileWriter>(std::move(options));
4141
}
4242

43-
explicit CPManifestFileWriter(Options options)
44-
: file_writer(std::make_unique<WriteBufferFromFile>(options.file_path))
43+
explicit CPManifestFileWriter(Options options_)
44+
: options(std::move(options_))
45+
, file_writer(std::make_unique<WriteBufferFromFile>(options.file_path))
4546
, compressed_writer(std::make_unique<CompressedWriteBuffer<true>>(*file_writer, CompressionSettings()))
4647
, max_edit_records_per_part(options.max_edit_records_per_part)
4748
{
@@ -65,6 +66,8 @@ class CPManifestFileWriter : private boost::noncopyable
6566

6667
void flush();
6768

69+
void abort();
70+
6871
private:
6972
void writeEditsPart(const universal::PageEntriesEdit & edit, UInt64 start, UInt64 limit);
7073

@@ -78,6 +81,7 @@ class CPManifestFileWriter : private boost::noncopyable
7881
WritingFinished,
7982
};
8083

84+
Options options;
8185
// compressed<plain_file>
8286
const std::unique_ptr<WriteBufferFromFile> file_writer;
8387
const WriteBufferPtr compressed_writer;

Diff for: dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.cpp

+52-22
Original file line numberDiff line numberDiff line change
@@ -36,9 +36,13 @@
3636

3737
#include <mutex>
3838

39-
4039
namespace DB
4140
{
41+
namespace FailPoints
42+
{
43+
extern const char force_checkpoint_dump_throw_datafile[];
44+
}
45+
4246
UniversalPageStoragePtr UniversalPageStorage::create(
4347
const String & name,
4448
PSDiskDelegatorPtr delegator,
@@ -495,7 +499,7 @@ bool UniversalPageStorage::canSkipCheckpoint() const
495499
return snap->sequence == last_checkpoint_sequence;
496500
}
497501

498-
PS::V3::CPDataDumpStats UniversalPageStorage::dumpIncrementalCheckpoint(
502+
std::optional<PS::V3::CPDataDumpStats> UniversalPageStorage::dumpIncrementalCheckpoint(
499503
const UniversalPageStorage::DumpCheckpointOptions & options)
500504
{
501505
std::scoped_lock lock(checkpoint_mu);
@@ -504,7 +508,7 @@ PS::V3::CPDataDumpStats UniversalPageStorage::dumpIncrementalCheckpoint(
504508
auto snap = page_directory->createSnapshot(/*tracing_id*/ "dumpIncrementalCheckpoint");
505509

506510
if (snap->sequence == last_checkpoint_sequence && !options.full_compact)
507-
return {.has_new_data = false};
511+
return PS::V3::CPDataDumpStats{.has_new_data = false};
508512

509513
// As a checkpoint, we write both entries (in manifest) and its data.
510514
// Some entries' data may be already written by a previous checkpoint. These data will not be written again.
@@ -542,23 +546,49 @@ PS::V3::CPDataDumpStats UniversalPageStorage::dumpIncrementalCheckpoint(
542546
.max_data_file_size = options.max_data_file_size,
543547
.max_edit_records_per_part = options.max_edit_records_per_part,
544548
});
545-
546-
writer->writePrefix({
547-
.writer = options.writer_info,
548-
.sequence = snap->sequence,
549-
.last_sequence = last_checkpoint_sequence,
550-
});
551-
PS::V3::CPFilesWriter::CompactOptions compact_opts = [&]() {
552-
if (options.full_compact)
553-
return PS::V3::CPFilesWriter::CompactOptions(true);
554-
if (options.compact_getter == nullptr)
555-
return PS::V3::CPFilesWriter::CompactOptions(false);
556-
return PS::V3::CPFilesWriter::CompactOptions(options.compact_getter());
549+
std::vector<String> data_file_paths;
550+
const auto checkpoint_dump_stats = [&]() -> std::optional<PS::V3::CPDataDumpStats> {
551+
try
552+
{
553+
writer->writePrefix({
554+
.writer = options.writer_info,
555+
.sequence = snap->sequence,
556+
.last_sequence = last_checkpoint_sequence,
557+
});
558+
const PS::V3::CPFilesWriter::CompactOptions compact_opts = [&]() {
559+
if (options.full_compact)
560+
return PS::V3::CPFilesWriter::CompactOptions(true);
561+
if (options.compact_getter == nullptr)
562+
return PS::V3::CPFilesWriter::CompactOptions(false);
563+
return PS::V3::CPFilesWriter::CompactOptions(options.compact_getter());
564+
}();
565+
// get the remote file ids that need to be compacted
566+
const auto checkpoint_dump_stats_inner
567+
= writer->writeEditsAndApplyCheckpointInfo(edit_from_mem, compact_opts, options.only_upload_manifest);
568+
data_file_paths = writer->writeSuffix();
569+
fiu_do_on(FailPoints::force_checkpoint_dump_throw_datafile, {
570+
throw Exception(ErrorCodes::LOGICAL_ERROR, "fake checkpoint write exception");
571+
});
572+
return checkpoint_dump_stats_inner;
573+
}
574+
catch (...)
575+
{
576+
// Could be #9406, which is a soft error.
577+
tryLogCurrentException(
578+
__PRETTY_FUNCTION__,
579+
fmt::format(
580+
"Error dumping incremental snapshot sequence={} manifest_file_path={} data_file_path_pattern={}",
581+
sequence,
582+
manifest_file_path,
583+
options.data_file_path_pattern));
584+
writer->abort();
585+
return std::nullopt;
586+
}
557587
}();
558-
// get the remote file ids that need to be compacted
559-
const auto checkpoint_dump_stats
560-
= writer->writeEditsAndApplyCheckpointInfo(edit_from_mem, compact_opts, options.only_upload_manifest);
561-
auto data_file_paths = writer->writeSuffix();
588+
589+
if (!checkpoint_dump_stats.has_value())
590+
return std::nullopt;
591+
562592
writer.reset();
563593
auto dump_data_seconds = sw.elapsedMillisecondsFromLastTime() / 1000.0;
564594

@@ -580,7 +610,7 @@ PS::V3::CPDataDumpStats UniversalPageStorage::dumpIncrementalCheckpoint(
580610
// TODO: Currently, even when has_new_data == false,
581611
// something will be written to DataFile (i.e., the file prefix).
582612
// This can be avoided, as its content is useless.
583-
if (checkpoint_dump_stats.has_new_data)
613+
if (checkpoint_dump_stats->has_new_data)
584614
{
585615
// Copy back the checkpoint info to the current PageStorage.
586616
// New checkpoint infos are attached in `writeEditsAndApplyCheckpointInfo`.
@@ -607,8 +637,8 @@ PS::V3::CPDataDumpStats UniversalPageStorage::dumpIncrementalCheckpoint(
607637
copy_checkpoint_info_seconds,
608638
sw.elapsedSeconds(),
609639
sequence,
610-
checkpoint_dump_stats);
611-
SetMetrics(checkpoint_dump_stats);
640+
*checkpoint_dump_stats);
641+
SetMetrics(*checkpoint_dump_stats);
612642
return checkpoint_dump_stats;
613643
}
614644

Diff for: dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.h

+1-1
Original file line numberDiff line numberDiff line change
@@ -227,7 +227,7 @@ class UniversalPageStorage final
227227
UInt64 max_edit_records_per_part = 100000;
228228
};
229229

230-
PS::V3::CPDataDumpStats dumpIncrementalCheckpoint(const DumpCheckpointOptions & options);
230+
std::optional<PS::V3::CPDataDumpStats> dumpIncrementalCheckpoint(const DumpCheckpointOptions & options);
231231

232232
PS::V3::CPDataFilesStatCache::CacheMap getRemoteDataFilesStatCache() const
233233
{

Diff for: dbms/src/Storages/Page/V3/Universal/UniversalPageStorageService.cpp

+14-10
Original file line numberDiff line numberDiff line change
@@ -282,16 +282,20 @@ bool UniversalPageStorageService::uploadCheckpointImpl(
282282
};
283283

284284
const auto write_stats = uni_page_storage->dumpIncrementalCheckpoint(opts);
285-
GET_METRIC(tiflash_storage_checkpoint_flow, type_incremental).Increment(write_stats.incremental_data_bytes);
286-
GET_METRIC(tiflash_storage_checkpoint_flow, type_compaction).Increment(write_stats.compact_data_bytes);
287-
288-
LOG_INFO(
289-
log,
290-
"Upload checkpoint success,{} upload_sequence={} incremental_bytes={} compact_bytes={}",
291-
force_sync_data ? " sync_all=true" : "",
292-
upload_info.upload_sequence,
293-
write_stats.incremental_data_bytes,
294-
write_stats.compact_data_bytes);
285+
if (write_stats.has_value())
286+
{
287+
GET_METRIC(tiflash_storage_checkpoint_flow, type_incremental)
288+
.Increment(write_stats.value().incremental_data_bytes);
289+
GET_METRIC(tiflash_storage_checkpoint_flow, type_compaction).Increment(write_stats.value().compact_data_bytes);
290+
291+
LOG_INFO(
292+
log,
293+
"Upload checkpoint success,{} upload_sequence={} incremental_bytes={} compact_bytes={}",
294+
force_sync_data ? " sync_all=true" : "",
295+
upload_info.upload_sequence,
296+
write_stats->incremental_data_bytes,
297+
write_stats->compact_data_bytes);
298+
}
295299

296300
return true;
297301
}

Diff for: dbms/src/Storages/S3/S3RandomAccessFile.cpp

+17-2
Original file line numberDiff line numberDiff line change
@@ -192,7 +192,15 @@ bool S3RandomAccessFile::initialize()
192192
auto outcome = client_ptr->GetObject(req);
193193
if (!outcome.IsSuccess())
194194
{
195-
LOG_ERROR(log, "S3 GetObject failed: {}, cur_retry={}", S3::S3ErrorMessage(outcome.GetError()), cur_retry);
195+
auto el = sw.elapsedSeconds();
196+
LOG_ERROR(
197+
log,
198+
"S3 GetObject failed: {}, cur_retry={}, key={}, elapsed{}={:.3f}s",
199+
S3::S3ErrorMessage(outcome.GetError()),
200+
cur_retry,
201+
req.GetKey(),
202+
el > 60.0 ? "(long)" : "",
203+
el);
196204
continue;
197205
}
198206

@@ -208,7 +216,14 @@ bool S3RandomAccessFile::initialize()
208216
}
209217
if (cur_retry >= max_retry && !request_succ)
210218
{
211-
LOG_INFO(log, "S3 GetObject timeout: {}, max_retry={}", remote_fname, max_retry);
219+
auto el = sw.elapsedSeconds();
220+
LOG_INFO(
221+
log,
222+
"S3 GetObject timeout: max_retry={}, key={}, elapsed{}={:.3f}s",
223+
max_retry,
224+
req.GetKey(),
225+
el > 60.0 ? "(long)" : "",
226+
el);
212227
}
213228
return request_succ;
214229
}

0 commit comments

Comments
 (0)