TiDB Bug List

Note:
This topic has been translated from a Chinese forum by GPT and might contain errors.

Original topic: TiDB Bug List=

| username: h5n1

[This post is used to record bugs encountered in TiDB for easy reference. The content is sourced from the TUG issue compilation. Please do not reply with non-bug related information. Content additions need to be maintained by everyone.]

1. bug-11217: Multi-key GC call causes GC to not work, leaving a large number of historical versions

2. bug-11903: Counter error causes GC delete range to not work after drop/truncate, reporting “gc work is too busy”

3. bug-12934: [Critical bug] Switching PD Leader or restarting PD may cause continuous SQL execution errors

4. bug-5687: [Critical bug] TiFlash may crash occasionally after enabling Profiling

5. bug-37238: [Critical bug] Outer join results may execute incorrectly

6. TiFlash 6.2 memory surge causing OOM without large queries

7. In version 6.x, after setting a new TiFlash replica, the progress synchronization does not update

8. In version 6.4, BR uses an incorrect partitionID after restoring a partitioned table

9. In versions v6.3.0-v6.5.0, using Multi-schema change to add a unique index causes data index inconsistency

| username: h5n1 | Original post link

bug-11217: Multi-key GC calls cause GC to not work, leaving a large number of historical versions

Affected Versions

  • v5.0.0-v5.0.4
  • v5.1.0-v5.1.2
  • v5.2.0-v5.2.2

issue: https://github.com/tikv/tikv/issues/11217
TiKV GcKeys task doesn’t work when called with multiple keys (at least in 5.1 but I think for everything)

Symptoms and Confirmation Methods

  • Some scan-related operations execute slowly, and there are many UPDATE/DELETE statements executed in the cluster
  • In the scan detail of EXPLAIN ANALYZE, key_skipped_count is much higher than total_process_keys or in the slow log, key_skipped_count is much higher than total_process_keys

Solution

  • Set gc.enable-compaction-filter: false to disable TiKV’s compaction filter GC and use the old GC mode for multi-version GC
  • Upgrade to a new version, choosing the latest release version as mentioned above
| username: h5n1 | Original post link

bug-11903: Counter error causes GC delete range to not work after drop/truncate, reporting “gc work is too busy”

Problem

During periods when TiKV GC worker CPU usage is at 100%, executing drop table or truncate table commands may result in the issue where TiKV space is not reclaimed after deleting the table. Even after the GC worker CPU usage decreases, subsequent drop table or truncate table commands still do not reclaim space.

Cause

TiDB’s drop table and truncate table commands send an unsafe destroy range request to TiKV to delete a range of data.

When the TiKV GC worker is busy, the number of pending tasks for the GC worker may reach its limit. If unsafe destroy range tasks are added during this time, the task counter is incorrectly incremented but not decremented.

After multiple such operations, the counter value permanently exceeds the busy threshold of the GC worker. Consequently, all subsequent unsafe destroy range requests are rejected by TiKV, causing the data deletion to fail after drop/truncate table operations.

GitHub issue: https://github.com/tikv/tikv/issues/11903
– False GcWorkerTooBusy caused by incorrect scheduled_tasks

Confirmation

  1. The TiDB monitoring of GC - Delete Range Failure OPM shows continuous send failures, as shown in the figure:

  1. TiDB logs confirm that the Delete Range error reason is “gc worker is too busy.”
  2. From the principle, confirm again that TiKV has experienced a situation where the GC worker CPU was continuously at 100%.

Solution

Bugfix PR: https://github.com/tikv/tikv/pull/11904

Fixed versions: 5.0.7, 5.1.4, 5.3.1, 5.4.0

Temporary Measures

  1. If the current TiKV GC worker CPU usage is not high, you can restart the TiKV instance to reset the incorrect counter and temporarily avoid the issue.
  2. Avoid executing drop table/truncate table operations when the TiKV GC worker CPU usage is high.
| username: Billmay表妹 | Original post link

bug12934: [Critical bug] Switching PD Leader or Restarting PD May Cause Continuous SQL Execution Errors

Product TiDB
Component PD TiKV
Version 5.4.2 5.3.2
Category Troubleshoot
Tag KnowIssue
Source

Issue

After performing a Transfer Leader or restart operation on PD, the cluster experiences continuous SQL execution errors.

This issue was discovered during 6.2.0 testing: Master: two tikv don't report region heartbeat after inject fault to pd leader · Issue #12934 · tikv/tikv · GitHub

Affected versions: v5.3.2, v5.4.2

Diagnostic Steps

  1. TiDB monitoring observes continuous SQL execution errors, such as Region Unavailable / Region Epoch not match, etc.

  2. TiKV monitoring in TiKV Details - PD - PD heartbeats shows a rapid increase in pending.

Resolution

Upgrade TiKV to a version that has fixed this bug.

Bug Fix PR: pd-client: pd client should update if the grpc stream sender failed. by bufferflies · Pull Request #13094 · tikv/tikv · GitHub

Expected fixed versions: v5.3.3, v5.4.3

| username: Billmay表妹 | Original post link

bug-5687: [Critical bug] TiFlash Occasionally Crashes After Enabling Profiling

Product TiDB
Component TiFlash
Version 6.1.0
Category Troubleshoot
Tags KnowIssue
Source

Issue

TiFlash occasionally encounters certain system calls (such as write()) returning illegal errno during operation. Since the program cannot handle illegal errno, it eventually leads to a process crash. This issue only reproduces during Profiling, so it is suspected to be related to Profiling.

Github issue: PageFile::Writer::closeFd raise Bad file descriptor error · Issue #5687 · pingcap/tiflash · GitHub

Diagnostic Steps

  1. Observe the logs and stack trace when TiFlash crashes to determine if they are caused by illegal errno.
  2. Check if there are any ongoing Profiling actions (including Continuous Profiling, Manual Profiling, or calling the /debug/pprof/profile interface).

Workaround

Disable Continuous Profiling in TiDB Dashboard and temporarily avoid performing Manual Profiling on TiFlash.

NOTE: Currently, only versions v6.1.0 and v6.2.0 have Continuous Profiling enabled by default. Starting from version v6.1.1, Continuous Profiling is disabled by default.

| username: Billmay表妹 | Original post link

bug-37238: [Critical bug] Outer join results may be incorrect

Product TiDB
Component TiDB
Version 6.1.0
Category Troubleshoot
Tags KnowIssue
Source

Issue

Root Cause

When tidb_enable_outer_join_reorder is set to true,

there is an error in handling the ON condition of the join during the join reorder process.

Diagnostic Steps

When there are multiple outer joins, and each ON condition is relatively simple, involving no more than two tables, and there are multiple outer joins sharing a common outer table, the result may be incorrect.

  • A left join B_1 on F_1 left join B_2 on F_2 left join B_3 on F_3 left join B_i on F_i
  • All join conditions F_i each involve only two tables. Among them, there are two join conditions F_i, F_j, where F_i involves tables A and B_i, and F_j involves tables A and B_j. Additionally, one of the join conditions in F_i and F_j involves only table A.
  • At this point, due to some improper handling in join reorder, the result may be incorrect.

Resolution

Fix PR: planner: fix outer join reorder will push down its outer join condition by AilinKid · Pull Request #37245 · pingcap/tidb · GitHub

Fixed in versions 6.1.1 and 6.2.1

| username: h5n1 | Original post link

Question
OOM occurs under non-large queries and low pressure after upgrading to TiFlash 6.2.

Affected Versions
v6.2 and later

Cause of the Issue:
Starting from version 6.2, TiFlash introduced PageStorage v3 as the underlying storage, which can significantly reduce write amplification and improve write performance, while also avoiding high CPU usage from background GC. However, it also introduced a bug where the deletion marks of pages in PageStorage during GC are not actually reclaimed, causing the WAL files to grow larger and larger, leading to increased memory usage by GC.
Related issues:
Minor compaction does not work as expected · Issue #6159 · pingcap/tiflash · GitHub
Compaction on PageStorage WAL causes OOM · Issue #6163 · pingcap/tiflash · GitHub

Verification Method:

  1. Memory surge leading to OOM in TiFlash under non-large query conditions.
  2. The log_xxxx_1 file in the <tiflash_data_dir>/data/page/log/wal/ directory continues to grow.

Temporary Solution
Use ALTER TABLE xxx COMPACT TIFLASH REPLICA to manually perform defragmentation and reduce memory usage.

Fixed Version
Expected to be fixed in version 6.4.

Reference:

| username: h5n1 | Original post link

Question
After setting a new TiFlash replica in version 6.x, the progress synchronization does not update.

Affected Versions
6.0 - 6.2

Cause of the Issue:
The upper limit value of region_id is int64, but TiFlash incorrectly parses region_id as int32. When region_id reaches the value of int32, subsequent new tables with TiFlash replicas cannot synchronize, but tables that already have TiFlash replicas are not affected.
Related issue:

Verification Method:

  1. Check if the region_id in PD monitoring or in information_schema.tikv_region_peers exceeds the maximum value of int32.

Temporary Solution
Apply the hot fix: If region id larger than MaxInt32, TiFlash replica progress can't be updated. · Issue #37862 · pingcap/tidb · GitHub. Upgrade to the fixed version.

Fixed Version
Expected to be fixed in versions 6.5 and 6.1.4

Reference:

| username: Billmay表妹 | Original post link

We have discovered a critical bug related to the “BR” component of TiDB. Details are as follows:

Product TiDB
Component BR
Version 6.4.0
Category Troubleshoot
Tags KnowIssue
Source

Issue

An optimization in BR v6.4.0 inadvertently introduced a bug (restore: data corrupt when restoring partition tables · Issue #40177 · pingcap/tidb · GitHub) that may cause issues with certain partition tables using problematic Partition IDs after restoration.

The “problematic” aspect is that these Partition IDs might later be used as Table IDs by other tables.

Root Cause

BR v6.4.0 introduced an optimization to try to keep the Table IDs of the old cluster unchanged during restoration. The basic process of this optimization is as follows:

  1. Find the largest ID among all Table IDs used in the backup files.
  2. Rebase TiDB’s Global ID (which can be understood as the auto-increment ID used for Table IDs) to this largest ID.

After performing step (2), theoretically, we can safely use the old cluster’s Table IDs (since these IDs will not be reused).

However, the current implementation inadvertently ignored partitions in step (1): Partition IDs also come from the Global ID and may be larger than Table IDs. If we only rebase the Global ID to Max(TableID), we cannot ensure that these Partition IDs can be safely used.

The issue arises when any Partition ID is greater than the largest Table ID.

Diagnostic Steps

Depending on how these “problematic” partitions are used later, the specific manifestations of this bug are numerous and mostly catastrophic, such as:

  • Partition table data being unexpectedly garbage collected (GC).
  • Partition table data being unexpectedly overwritten.
  • New tables containing records that should not exist.

Another method to more accurately determine if this bug has been triggered:

First, search for “registering the table IDs” in the BR logs, and you will get a log like this:

[INFO] [client.go:244] ["registering the table IDs"] [ids="ID:[79,153)"]

Here, ids indicates the range of Global IDs that BR considers “safe to use.” Next, use this SQL query:

SELECT T.`TIDB_TABLE_ID`, P.`TIDB_PARTITION_ID`, T.table_schema, T.`table_name`, partition_name FROM 
  INFORMATION_SCHEMA.`PARTITIONS` P INNER JOIN 
    INFORMATION_SCHEMA.`TABLES` T 
      ON (T.TABLE_NAME = P.TABLE_NAME and T.`TABLE_SCHEMA` = P.TABLE_SCHEMA) 
  WHERE T.`TIDB_TABLE_ID` BETWEEN @lower AND @higher;

Replace @lower and @higher with the range indicated by ids in the log.

If any TIDB_PARTITION_ID appears outside the ids range, this bug has likely been triggered. Please refer to the “Workaround” section below.

Resolution

We discovered and fixed this issue in v6.5.0 LTS. Now, BR will rebase the Global ID to Max(TableID ∪ PartitionID) to avoid this issue.

Workaround

It is recommended not to use BR v6.4.0 for partition table restoration. Instead, use v6.5.0 LTS.

If you must use v6.4.0 for restoration, you can workaround this bug by dropping all restored databases and tables and re-executing the restoration.

| username: Billmay表妹 | Original post link

[Critical bug] v6.3.0-v6.5.0 Using Multi-schema Change to Add Unique Index Causes Data Index Inconsistency

Product TiDB
Component TiDB
Version 6.3.0 6.4.0 6.5.0

Issue

After using multi-schema change to add a unique index, the status of the unique index was not set correctly, leading to incorrect insertion of duplicate rows when executing the INSERT IGNORE statement, thus violating the uniqueness constraint of the index. For example:

create table t (a int, b int);
insert into t values (1, 1);
insert into t values (2, 2);
alter table t add unique index idx(b), ...[any other schema changes];
insert ignore into t values (2, 2);
admin check table t;

ERROR 8223 (HY000): data inconsistency in table: t, index: idx, handle: 2, index-values:"handle: 3, values: [KindInt64 2]" != record-values:"handle: 2, values: [KindInt64 2]"

Currently, this type of uniqueness constraint violation bug has only been found with the INSERT IGNORE statement. Using INSERT, UPDATE, and REPLACE statements to insert duplicate values into the unique index will correctly report a “duplicate entry” error as expected.

Root Cause

Since version 6.3.0, TiDB supports adding indexes using the ingest mode (@@tidb_ddl_enable_fast_reorg). After the incremental data is merged back into the original index, the status change of the target index by the multi-schema change was not persisted to TiKV, and this change was discarded when executing the next schema change. After the entire multi-schema change DDL is completed, the target index remains in an incorrect state.

If the application subsequently executes the INSERT IGNORE statement, TiDB’s logic for determining whether the index value is duplicated is affected by the status of the index, causing the check to be ignored and duplicate values to be inserted (normally, duplicate values should not be inserted and a warning should be reported).

Diagnostic Steps

If data index inconsistency occurs and the following conditions are met, it can be confirmed as the same issue:

  • TiDB version is 6.3.0, 6.4.0, or 6.5.0.
  • The DDL history of the table shows that multi-schema change was used to add a unique index.
  • The data index inconsistency involves the same index added by the multi-schema change.
  • TiDB logs during the period when the unique index was added contain the keyword “[ddl-ingest]”.

Resolution

We will fix this issue in v6.5.1 LTS.

Workaround

It is recommended not to use multi-schema change to add unique indexes. If already added, it is best to rebuild the index to avoid future data index inconsistencies.

| username: Billmay表妹 | Original post link

[Critical Bug] The update could not read the latest data if read committed is used

Product TiDB
Component TiDB
Version 6.0.0 6.1.1 6.1.2 6.1.3 6.1.4

Issue

Root Cause

When TiDB uses RC isolation, the timestamp read by the point get executor is not refreshed as expected, so it cannot get the latest value. When using the RC isolation level, the executor does not acquire a pessimistic lock for non-existent keys, and updates may not take effect on newly inserted rows.

Diagnostic Steps

If the update result is unexpected, such as not updating to the latest value, try to confirm:

  • TiDB version is v6.0, v6.1.1 - v6.1.4
  • Using RC isolation level
  • Using concurrent insert and update transactions, the update/delete transaction expects to see the latest insert result using pointGetExecutor to read

Resolution

  • Avoid using versions
    • v6.0
    • v6.1.1 - v6.1.4
  • For v6.1.x versions, upgrade to v6.1.5

Workaround

  • Use select for update to lock the row key before update/delete statements
  • Disable RC isolation level and switch to the default RR
| username: TiDBer_yyy | Original post link

This post is very similar
Cluster version: 5.0.4

| username: 喵父666 | Original post link

:+1: :+1: :+1: :+1:

| username: Billmay表妹 | Original post link

[Critical bug] Log Backup may lose data after TiKV restarts

Hello everyone,

We have recently discovered a critical bug related to the “TiKV BR” component of TiDB. Details are as follows:

Product TiDB
Component TiKV BR
Version 6.5.4 7.1.0 7.1.1 7.4.0 7.2.0 7.3.0 7.1.2 7.5.0 6.5.6 7.1.3 6.5.5 7.1.4 7.5.1 7.6.0 6.5.7 6.5.8 8.0.0 6.5.9
Category Developer
Tags KnowIssue
Source

Issue

Root Cause

TiKV starts two parallel processes during log backup:

  1. One process pushes files to downstream, listening to changes in raftstore and caching them locally.
  2. Another process advances the backup progress by continuously obtaining the latest TSO from PD and using it to calculate the latest backup progress.

When process (1) accumulates enough files, it uploads the current cached files to external storage. After this, it gets the current progress from (2) and updates it to PD.

However, during the file upload to external storage by process (1), new writes continue to come in and are included in the progress calculation. The problem arises here: these new writes will not be uploaded to external storage in the current batch. Therefore, if process (2) calculates the latest progress while process (1) is uploading files, this progress is actually “ahead.”

Generally, these writes will be uploaded with the next batch, so this “ahead” progress does not cause serious consequences. However, if TiKV restarts at this time, this batch of data will be lost. If the global Checkpoint has already advanced, other TiKVs will start log backup from this “ahead” progress when they become the Leader. The data not backed up earlier will never be backed up.

[1,Flush] [2,calculate progress]

TS --------------|----------|-------->

^ ^

| ± …but the log backup believes we have backed up to here

± Writes were actually backed up to here…

Currently, there is no precise way to estimate the trigger probability, but overall, the trigger probability:

  • Increases as the time taken for Flush increases.
  • Decreases as the number of TiKVs increases (because with more TiKVs, the probability of the global checkpoint being stuck by a TiKV increases).

Considering that TiKV restarts are not common, and PiTR itself requires periodic full backups, the overall trigger probability of this bug is not high.

Diagnostic Steps

Currently, there is no good way to detect this issue in the backup cluster.

A noticeable characteristic of data lost due to this issue is that the lost data was written within a few hundred milliseconds before a progress update. You can check the timestamps of data rows or confirm through the TiDB HTTP API by querying MVCC versions.

Resolution

The incremental backup data will be lost after triggering, and it cannot be recovered.

It is recommended to refer to the workaround section or upgrade to an unaffected version (6.5.10, 7.1.5, 7.5.2, or higher LTS versions).

The following LTS versions are affected:

  • Release 6.5: 6.5.4~6.5.9
  • Release 7.1: 7.1.0~7.1.4
  • Release 7.5: 7.5.0~7.5.1

Workaround

In the upstream cluster, you can set log-backup.min-ts-interval (this configuration item is undocumented) to a value greater than the full backup interval (e.g., 365d) to resolve this issue; however, the downside is that RPO may be lower than expected if Leader Transfer is frequent.

| username: Billmay表妹 | Original post link

[Critical bug] Using restore to recover tables with auto random fields may cause data modification errors

Hello everyone,

We have recently discovered a critical bug related to the “BR” component of TiDB. Details are as follows:

Product TiDB
Component BR
Version 6.4.0 6.5.0 6.5.1 6.5.2 6.5.3 6.5.4 7.1.0 7.1.1 6.1.5 6.1.6 6.1.7 7.0.0 7.4.0 6.6.0 7.2.0 7.3.0 7.1.2 7.5.0 6.5.6 7.1.3 6.5.5 7.1.4 7.5.1 7.6.0 6.5.7 6.5.8 8.0.0
Category Troubleshoot
Tag KnowIssue
Source

Issue

When using br restore to recover a cluster, executing the insert on duplicate key update statement may result in incorrect target records being modified.

Root Cause

If the primary key of a table is a composite index that includes an auto random id, the auto random id field will not be rebase on the restored cluster. Subsequent insert on duplicate key update statements on this table will generate ids that duplicate existing records’ ids, causing unexpected updates to old records and resulting in data errors.

Refer to the Github Issue br doesn't rebase auto random if table's PKIsHandle is false · Issue #52255 · pingcap/tidb · GitHub for an example of this issue.

Diagnostic Steps

You can diagnose the existence of this issue through the following methods:

  1. Check if there are tables in the cluster whose primary key is a composite index containing an auto random id, and if the data of these tables were generated through BR restore. If not, skip; otherwise, proceed to step 2.

  2. For tables with composite primary keys containing an auto random id field, execute the show create table statement and observe whether the AUTO_RANDOM_BASE value in the output is normal.

Resolution

This issue has been fixed in the latest versions. The following LTS versions are affected by this issue. Consider upgrading to versions 6.1.8, 6.5.9, 7.1.5, 7.5.2, and later to avoid this issue.

  1. v6.1.5-v6.1.7

  2. v6.5.0-v6.5.8

  3. v7.1.0-v7.1.4

  4. v7.5.0-v7.5.1

Workaround

Refer to the source cluster and use the alter table auto_random_base statement to fix the affected tables.

| username: Billmay表妹 | Original post link

[Critical bug] Issue with TiFlash metadata corruption when upgrading clusters from versions below v6.5 to versions not lower than v6.5

Hello everyone,

We have recently discovered a critical bug related to the “TiFlash” component of TiDB. Details are as follows:

Product TiDB
Component TiFlash
Version 6.5.0 6.5.1 6.5.2 6.5.3 6.5.4 7.1.0 7.1.1 7.0.0 7.4.0 6.6.0 7.2.0 7.3.0 7.1.2 7.5.0 6.5.6 7.1.3 6.5.5 7.1.4 7.5.1 7.6.0 6.5.7 6.5.8 8.0.0 6.5.9
Category Upgrade
Tag kbBug
Source

Issue

When there are TiFlash replicas in the cluster, upgrading from versions below v6.5 to versions not lower than v6.5 may lead to TiFlash metadata corruption, causing it to fail to start.

Affected versions: Upgrading the cluster from below v6.5 to the following LTS versions may encounter this issue: v6.5.0~v6.5.9, v7.1.0~v7.1.5, v7.5.0~v7.5.1.

Root Cause

When upgrading a cluster from versions below v6.5 to versions not lower than v6.5, TiFlash internally upgrades the storage format to PageStorage V3 to improve write performance.

If the table data meets certain distribution criteria (no new data written to the Delta layer of the DeltaTree storage engine in TiFlash before the upgrade), the “allocated file ID” may get an incorrect value after the upgrade. This leads to erroneous metadata overwrites as new data is written during subsequent TiFlash operations. This issue may not manifest until the TiFlash instance is restarted. Upon the next restart, the corrupted metadata causes a check failure, resulting in the TiFlash instance failing to start.

Diagnostic Steps

Confirm if there has been a recent upgrade operation from a version below v6.5 in the cluster.

When startup fails, the TiFlash log error messages may look like:

["Code: 10017, e.displayText() = DB::Exception: try to create external version with invalid state [ver=23783.0] [state={type:VAR_REF, create_ver: 1653.0, is_deleted: true, delete_ver: 21767.0, ori_page_id: 22935.100, being_ref_count: 1, num_entries: 0}]: [type=PUT_EXTERNAL] [page_id=22935.106] [ver=23783.0] …

or

["Code: 10015, e.displayText() = DB::Exception: Invalid page id, entry not exist [page_id=69.2] [resolve_id=69.2]: keyspace=4294967295 physical_table_id=-1: (while preHandleSnapshot region_id=100, index=3246, term=7) …

Resolution

When upgrading the cluster from versions below v6.5 to the following LTS versions, TiFlash may encounter errors: v6.5.0~v6.5.9, v7.1.0~v7.1.5, v7.5.0~v7.5.1.

If you need to upgrade the cluster from below v6.5.0, it is recommended to choose at least v6.5.10/v7.1.6/v7.5.2/v8.1.0 as the target version. Additionally, if there are no TiFlash instances before the upgrade and TiFlash instances are deployed only after upgrading to v6.5, this issue will not occur as it does not involve upgrading the data format within TiFlash.

Workaround

[For TiFlash instances that have encountered this issue]

If TiFlash metadata corruption occurs due to this issue, take the affected TiFlash instance offline, redeploy the TiFlash instance, and wait for data to resynchronize from TiKV.

[How to avoid this issue before upgrading the cluster version]

If you must upgrade from a version below v6.5 to the affected versions, a workaround to avoid this issue is:

  1. Set the TiFlash replica count of the tables to 0 and scale down all TiFlash instances before the upgrade.
  2. Upgrade the cluster to the target version.
  3. Redeploy TiFlash instances and add TiFlash replicas.
| username: Billmay表妹 | Original post link

[Critical bug] TiFlash randomly crashes when async grpc server is enabled

Versions

6.1.0 6.1.1 6.1.2 6.1.3 6.1.4 6.1.5 6.1.6

Issue

Root Cause

There is a data race in the implementation of the async grpc server.

Diagnostic Steps

TiFlash encounters random crashes when the async grpc server is enabled. The crash stack contains information related to EstablishCallData. For example:
[2024/01/19 19:06:28.901 +08:00] [ERROR] [BaseDaemon.cpp:570] [“BaseDaemon:\n 0x1bf3ca4\tfaultSignalHandler(int, siginfo_t*, void*) [tiflash+29310116]\n \tlibs/libdaemon/src/BaseDaemon.cpp:221\n 0xfffc96c207c0\t [linux-vdso.so.1+1984]\n 0x648fc54\tDB::EstablishCallData::proceed() [tiflash+105446484]\n \tdbms/src/Flash/EstablishCall.cpp:151\n 0x1ab2280\tDB::handleRpcs(grpc_impl::ServerCompletionQueue*, Poco::Logger*) [tiflash+27992704]\n \tdbms/src/Server/Server.cpp:546\n 0x1afe6b0\tauto DB::wrapInvocable<std::__1::function<void ()> >(bool, std::__1::function<void ()>&&)::‘lambda’()::operator()() [tiflash+28305072]\n \tdbms/src/Common/wrapInvocable.h:36\n 0x1afe828\tstd::__1::packaged_task<void ()>::operator()() [tiflash+28305448]\n \t/usr/local/bin/…/include/c++/v1/future:2089\n 0x1af0ca8\tDB::DynamicThreadPool::executeTask(std::__1::unique_ptr<DB::IExecutableTask, std::__1::default_deleteDB::IExecutableTask >&) [tiflash+28249256]\n \tdbms/src/Common/DynamicThreadPool.cpp:101\n 0x1af0684\tDB::DynamicThreadPool::fixedWork(unsigned long) [tiflash+28247684]\n \tdbms/src/Common/DynamicThreadPool.cpp:115\n 0x1af1804\tauto std::__1::thread DB::ThreadFactory::newThread<void (DB::DynamicThreadPool::)(unsigned long), DB::DynamicThreadPool, unsigned long&>(bool, std::__1::basic_string<char, std::__1::char_traits, std::__1::allocator >, void (DB::DynamicThreadPool::&&)(unsigned long), DB::DynamicThreadPool&&, unsigned long&)::‘lambda’(auto&&…)::operator()<DB::DynamicThreadPool*, unsigned long>(auto&&…) const [tiflash+28252164]\n \tdbms/src/Common/ThreadFactory.h:47\n 0x1af1628\tvoid* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_deletestd::__1::__thread_struct >, std::__1::thread DB::ThreadFactory::newThread<void (DB::DynamicThreadPool::)(unsigned long), DB::DynamicThreadPool, unsigned long&>(bool, std::__1::basic_string<char, std::__1::char_traits, std::__1::allocator >, void (DB::DynamicThreadPool::&&)(unsigned long), DB::DynamicThreadPool&&, unsigned long&)::‘lambda’(auto&&…), DB::DynamicThreadPool*, unsigned long> >(void*) [tiflash+28251688]\n \t/usr/local/bin/…/include/c++/v1/thread:291\n 0xfffc939a88cc\t [libpthread.so.0+35020]”] [thread_id=3266]

Resolution

Upgrade to version v6.1.7 or above.

Workaround

Disable the TiFlash async grpc server.

For clusters deployed with TiUP, you can use the command:

tiup cluster edit-config cluster_name

to modify the cluster configuration. Add the following under server_configs for tiflash:

profiles.default.enable_async_server: false

Then use:

tiup cluster reload cluster_name -R tiflash

to reload the configuration and restart the TiFlash nodes.

For clusters not deployed with TiUP, you can directly edit the tiflash.toml file. Add the following under:

[profiles]
[profiles.default]
enable_async_server = false

Then restart the corresponding TiFlash server.

| username: TiDBer_yyy | Original post link

I encountered this problem before :rofl:
At that time, I rebuilt the TiFlash cluster
TiDB 7.1.0 2台 Tiflash oom之后无法启动 - TiDB 的问答社区.