Sign In Try Free

TiDB 5.4 Release Notes

Release date: February 15, 2022

TiDB version: 5.4.0

In v5.4, the key new features or improvements are as follows:

  • Support the GBK character set
  • Support using Index Merge to access data, which merges the filtering results of indexes on multiple columns
  • Support reading stale data using a session variable
  • Support persisting the configuration for collecting statistics
  • Support using Raft Engine as the log storage engine of TiKV (experimental)
  • Optimize the impact of backup on the cluster
  • Support using Azure Blob storage as the backup storage
  • Continuously improve the stability and performance of TiFlash and the MPP engine
  • Add a switch in TiDB Lightning to determine whether to allow importing to an existing table with data
  • Optimize the Continuous Profiling feature (experimental)
  • TiSpark supports user identification and authentication

Compatibility changes

System variables

Variable name Change type Description
tidb_enable_column_tracking Newly added Controls whether to allow TiDB to collectPREDICATE COLUMNS. The default value isOFF.
tidb_enable_paging Newly added Controls whether to use the method of paging to send coprocessor requests inIndexLookUpoperator. The default value isOFF.
For read queries that useIndexLookupandLimitand thatLimitcannot be pushed down toIndexScan, there might be high latency for the read queries and high CPU usage for TiKV'sunified read pool. In such cases, because theLimitoperator only requires a small set of data, if you settidb_enable_pagingtoON, TiDB processes less data, which reduces query latency and resource consumption.
tidb_enable_top_sql Newly added Controls whether to enable the Top SQL feature. The default value isOFF.
tidb_persist_analyze_options Newly added Controls whether to enable theANALYZE configuration persistencefeature. The default value isON.
tidb_read_staleness Newly added Controls the range of historical data that can be read in the current session. The default value is0.
tidb_regard_null_as_point Newly added 控制优化器可以使用查询是否反对dition including null equivalence as a prefix condition for index access.
tidb_stats_load_sync_wait Newly added Controls whether to enable the synchronously loading statistics feature. The default value0means that the feature is disabled and that the statistics is asynchronously loaded. When the feature is enabled, this variable controls the maximum time that SQL optimization can wait for synchronously loading statistics before timeout.
tidb_stats_load_pseudo_timeout Newly added Controls when synchronously loading statistics reaches timeout, whether SQL fails (OFF) or falls back to using pseudo statistics (ON). The default value isOFF.
tidb_backoff_lock_fast Modified The default value is changed from100to10.
tidb_enable_index_merge Modified The default value is changed fromOFFtoON.
  • If you upgrade a TiDB cluster from versions earlier than v4.0.0 to v5.4.0 or later, this variable isOFFby default.
  • 如果你从v4.0.0 TiDB集群升级或更高版本to v5.4.0 or later, this variable remains the same as before the upgrade.
  • For the newly created TiDB clusters of v5.4.0 and later, this variable isONby default.
tidb_store_limit Modified Before v5.4.0, this variable can be configured at instance level and globally. Starting from v5.4.0, this variable only supports global configuration.

Configuration file parameters

Configuration file Configuration Change type Description
TiDB stats-load-concurrency Newly added Controls the maximum number of columns that the TiDB synchronously loading statistics feature can process concurrently. The default value is5.
TiDB stats-load-queue-size Newly added Controls the maximum number of column requests that the TiDB synchronously loading statistics feature can cache. The default value is1000.
TiKV snap-generator-pool-size Newly added The size of thesnap-generatorthread pool. The default value is2.
TiKV log.file.max-size,log.file.max-days,log.file.max-backups Newly added For details, seeTiKV Configuration File -log.file.
TiKV raft-engine Newly added Includesenable,dir,batch-compression-threshold,bytes-per-sync,target-file-size,purge-threshold,recovery-mode,recovery-read-block-size,recovery-read-block-size, andrecovery-threads. For details, seeTiKV Configuration File -raft-engine.
TiKV backup.enable-auto-tune Newly added In v5.3.0, the default value isfalse. Since v5.4.0, the default value is changed totrue. This parameter controls whether to limit the resources used by backup tasks to reduce the impact on the cluster when the cluster resource utilization is high. In the default configuration, the speed of backup tasks might slow down.
TiKV log-level,日志格式,log-file,log-rotation-size Modified The names of TiKV log parameters are replaced with the names that are consistent with TiDB log parameters, which arelog.level,log.format,log.file.filename, andlog.enable-timestamp. If you only set the old parameters, and their values are set to non-default values, the old parameters remain compatible with the new parameters. If both old and new parameters are set, the new parameters take effect. For details, seeTiKV Configuration File - log.
TiKV log-rotation-timespan Deleted The timespan between log rotations. After this timespan passes, a log file is rotated, which means a timestamp is appended to the file name of the current log file, and a new log file is created.
TiKV allow-remove-leader Deleted Determines whether to allow deleting the main switch.
TiKV raft-msg-flush-interval Deleted Determines the interval at which Raft messages are sent in batches. The Raft messages are sent in batches at every interval specified by this configuration item.
PD log.level Modified The default value is changed from "INFO" to "info", guaranteed to be case-insensitive.
TiFlash profile.default.enable_elastic_threadpool Newly added Determines whether to enable or disable the elastic thread pool function. Enabling this configuration item can significantly improve TiFlash CPU utilization in high concurrency scenarios. The default value isfalse.
TiFlash storage.format_version Newly added Specifies the version of DTFile. The default value is2, under which hashes are embedded in the data file. You can also set the value to3. When it is3, the data file contains metadata and token data checksum, and supports multiple hash algorithms.
TiFlash logger.count Modified The default value is changed to10.
TiFlash status.metrics_port Modified The default value is changed to8234.
TiFlash raftstore.apply-pool-size Newly added The allowable number of threads in the pool that flushes Raft data to storage. The default value is4.
TiFlash raftstore.store-pool-size Newly added The allowable number of threads that process Raft, which is the size of the Raftstore thread pool. The default value is4.
TiDB Data Migration (DM) collation_compatible Newly added The mode to sync the default collation inCREATESQL statements. The value options are "loose" (by default) and "strict".
TiCDC max-message-bytes Modified Change the default value ofmax-message-bytesin Kafka sink to104857601(10MB)
TiCDC partition-num Modified Change the default value ofpartition-numin Kafka Sink from4to3. It makes TiCDC send messages to Kafaka partitions more evenly.
TiDB Lightning meta-schema-name Modified Specifies the schema name for the metadata in the target TiDB. From v5.4.0, this schema is created only if you have enabledparallel import(the corresponding parameter istikv-importer.incremental-import = true).
TiDB Lightning task-info-schema-name Newly added Specifies the name of the database where duplicated data is stored when TiDB Lightning detects conflicts. By default, the value is "lightning_task_info". Specify this parameter only if you have enabled the "duplicate-resolution" feature.
TiDB Lightning incremental-import Newly added Determines whether to allow importing data to tables where data already exists. The default value isfalse.

Others

  • An interface is added between TiDB and PD. When you use theinformation_schema.TIDB_HOT_REGIONS_HISTORYsystem table, TiDB needs to use PD in the corresponding version.
  • TiDB Server, PD Server, and TiKV Server start using a unified naming method for the log-related parameters to manage log names, output formats, and the rules for rotation and expiration. For details, seeTiKV configuration file - log.
  • Since v5.4.0, if you create a SQL binding for an execution plan that has been cached via Plan Cache, the binding invalidates the plan already cached for the corresponding query. The new binding does not affect execution plans cached before v5.4.0.
  • In v5.3 and earlier versions,TiDB Data Migration (DM)documentation is independent of TiDB documentation. Since v5.4, DM documentation is integrated into TiDB documentation with the same version. You can directly readDM documentationwithout accessing the DM documentation site.
  • Remove the experimental feature of Point-in-time recovery (PITR) along with cdclog. Since v5.4.0, cdclog-based PITR and cdclog are no longer supported.
  • Make the behavior of setting system variables to the "DEFAULT" more MySQL-compatible#29680
  • Set the system variablelc_time_namesto read-only# 30084
  • Set the scope oftidb_store_limitfrom INSTANCE or GLOBAL to GLOBAL# 30756
  • Forbid converting the integer type column to the time type column when the column contains zero#25728
  • 修复的问题,没有错误的报道InforNANvalue when inserting floating-point values# 30148
  • 修复issue that theREPLACEstatement incorrectly changes other rows when the auto ID is out of range# 30301

New features

SQL

  • TiDB supports the GBK character set since v5.4.0

    Before v5.4.0, TiDB supportsascii,binary,latin1,utf8, andutf8mb4character sets.

    To better support Chinese users, TiDB supports the GBK character set since v5.4.0. After enabling thenew_collations_enabled_on_first_bootstrapoption in the TiDB configuration file when initializing a TiDB cluster for the first time, the TiDB GBK character set supports bothgbk_binandgbk_chinese_cicollations.

    When using the GBK character set, you need to pay attention to the compatibility restrictions. For details, seeCharacter Set and Collation - GBK.

Security

  • TiSpark supports user authentication and authorization

    Since TiSpark 2.5.0, TiSpark supports both database user authentication and read/write authorization at a database or table level. After enabling this feature, you can prevent the business from running unauthorized batch tasks such as draws to obtain data, which improves the stability and data security of online clusters.

    This feature is disabled by default. When it is enabled, if a user operating through TiSpark does not have the needed permissions, the user gets an exception from TiSpark.

    User document

  • TiUP supports generating an initial password for the root user

    An--initparameter is introduced to the command for starting a cluster. With this parameter, in a TiDB cluster deployed using TiUP, TiUP generates an initial strong password for the database root user. This avoids security risks in using a root user with an empty password and ensures the security of databases.

    User document

Performance

  • Continue improving the stability and performance of the columnar storage engine TiFlash and the computing engine MPP

    • Support pusing down more functions to the MPP engine:

      • String functions:LPAD(),RPAD(),STRCMP()
      • Date functions:ADDDATE(string, real),DATE_ADD(string, real),DATE_SUB(string, real),SUBDATE(string, real),QUARTER()
    • Introduce the elastic thread pool feature to improve resource utilization (experimental)

    • Improve the efficiency of converting data from row-based storage format to column-based storage format when replicating data from TiKV, which brings 50% improvement in the overall performance of data replication

    • Improve TiFlash performance and stability by tuning the default values of some configuration items. In an HTAP hybrid load, the performance of simple queries on a single table improves up to 20%.

      User documents:Supported push-down calculations,Configure the tiflash.toml file

  • Read historical data within a specified time range through a session variable

    TiDB is a multi-replica distributed database based on the Raft consensus algorithm. In face of high-concurrency and high-throughput application scenarios, TiDB can scale out its read performance through follower replicas, separating read and write requests.

    For different application scenarios, TiDB provides two modes of follower read: strongly consistent read and weakly consistent history read. The strongly consistent read mode is suitable for application scenarios that require real-time data. However, in this mode, because of the data replication latency between leaders and followers and the reduced throughput, the read request might have high latency, especially for geo-distributed deployments.

    For the application scenarios that have less strict requirements on real-time data, the history read mode is recommended. This mode can reduce latency and improve throughput. TiDB currently supports reading historical data by the following methods: Use SQL statements to read data from a time point in the past, or start a read-only transaction based on a time point in the past. Both methods support reading the historical data of a specific point in time or within a specified time range. For details, refer toRead Historical Data Using theAS OF TIMESTAMPClause.

    Since v5.4.0, TiDB improves the usability of the history read mode by supporting reading historical data within a specified time range through a session variable. This mode serves low-latency, high-throughput read requests in quasi-real-time scenarios. You can set the variable as follows:

    
                    
    set@@tidb_replica_read=leader_and_followerset@@tidb_read_staleness="-5"

    By this setting, TiDB can select the nearest leader or follower node and read the latest historical data within 5 seconds.

    User document

  • GA for Index Merge

    Index Merge is introduced in TiDB v4.0 as an experimental feature for SQL optimization. This method greatly accelerates condition filtering when a query requires scanning of multiple columns of data. Take the following query as an example. In theWHEREstatement, if the filtering conditions connected byORhave their respective indexes on columnskey1andkey2, the Index Merge feature filters the respective indexes at the same time, merges the query results, and returns the merged result.

    
                    
    SELECT * FROM table WHEREkey1<= 100 ORkey2= 200;

    Before TiDB v4.0, a query on a table supports using only one index for filtering at one time. If you want to query multiple columns of data, you can enable Index Merge to get the exact query result in a short time by using the indexes in individual columns. Index Merge avoids unnecessary full table scans and does not require establishing a large number of composite indexes.

    In v5.4.0, Index Merge becomes GA. However, you still need to pay attention to the following restrictions:

    • Index Merge supports only disjunctive normal form (X1⋁ X2⋁ …Xn). That is, this feature only works when filtering conditions in aWHEREclause are connected byOR.

    • For newly deployed TiDB clusters of v5.4.0 or later, this feature is enabled by default. For v5.4.0 or later TiDB clusters upgraded from earlier versions, this feature inherits the setting before the upgrade and you can change the setting as required (for TiDB clusters earlier than v4.0, this feature does not exist and is disabled by default).

      User document

  • Support Raft Engine (experimental)

    Support usingRaft Engineas the log storage engine in TiKV. Compared with RocksDB, Raft Engine can reduce TiKV I/O write traffic by up to 40% and CPU usage by 10%, while improving foreground throughput by about 5% and reducing tail latency by 20% under certain loads. In addition, Raft Engine improves the efficiency of log recycling and fixes the issue of log accumulation in extreme conditions.

    Raft Engine is still an experimental feature and is disabled by default. Note that the data format of Raft Engine in v5.4.0 is not compatible with previous versions. Before upgrading the cluster, you need to make sure that Raft Engine on all TiKV nodes is disabled. It is recommended to use Raft Engine only in v5.4.0 or a later version.

    User document

  • Support collecting statistics onPREDICATE COLUMNS(experimental)

    In most cases, when executing SQL statements, the optimizer only uses statistics of some columns (such as columns in theWHERE,JOIN,ORDER BY, andGROUP BYstatements). These used columns are calledPREDICATE COLUMNS.

    Since v5.4.0, you can set the value of thetidb_enable_column_trackingsystem variable toONto enable TiDB to collectPREDICATE COLUMNS.

    设置后,TiDB写道PREDICATE COLUMNSinformation to themysql.column_stats_usagesystem table every 100 *stats-lease. When the query pattern of your business is stable, you can use theANALYZE TABLE TableName PREDICATE COLUMNSsyntax to collect statistics on thePREDICATE COLUMNScolumns only, which can greatly reduce the overhead of collecting statistics.

    User document

  • Support synchronously loading statistics (experimental)

    Since v5.4.0, TiDB introduces the synchronously loading statistics feature. This feature is disabled by default. After enabling the feature, TiDB can synchronously load large-sized statistics (such as histograms, TopN, and Count-Min Sketch statistics) into memory when you execute SQL statements, which improves the completeness of statistics for SQL optimization.

    User document

Stability

  • Support persisting ANALYZE configurations

    统计数据是一种类型t的基本信息hat the optimizer refers to when generating execution plans. The accuracy of the statistics directly affects whether the generated execution plans are reasonable. To ensure the accuracy of the statistics, sometimes it is necessary to set different collection configurations for different tables, partitions, and indexes.

    Since v5.4.0, TiDB supports persisting someANALYZEconfigurations. With this feature, the existing configurations can be easily reused for future statistics collection.

    TheANALYZEconfiguration persistence feature is enabled by default (the system variabletidb_analyze_versionis2andtidb_persist_analyze_optionsisONby default). You can use this feature to record the persistence configurations specified in theANALYZEstatement when executing the statement manually. Once recorded, the next time TiDB automatically updates statistics or you manually collect statistics without specifying these configurations, TiDB will collect statistics according to the recorded configurations.

    User document

High availability and disaster recovery

  • Reduce the impact of backup tasks on the cluster

    Backup & Restore (BR) introduces the auto-tune feature (enabled by default). This feature can reduce the impact of backup tasks on the cluster by monitoring the cluster resource usage and adjusting the number of threads used by the backup tasks. In some cases, if you increase the cluster hardware resource for backup and enable the auto-tune feature, it can limit the impact of backup tasks on the cluster to 10% or less.

    User document

  • Support Azure Blob Storage as a target storage for backup

    Backup & Restore (BR) supports Azure Blob Storage as a remote backup storage. If you deploy TiDB in Azure Cloud, now you can back up the cluster data to the Azure Blob Storage service.

    User document

Data migration

  • TiDB Lightning introduces a new feature to determine whether to allow importing data to tables with data

    TiDB Lightning introduces a configuration itemincremental-import. It determines whether to allow importing data to tables with data. The default value isfalse. When using the parallel import mode, you must set the configuration totrue.

    User document

  • TiDB Lightning introduces the schema name that stores the meta information for parallel import

    TiDB Lightning introduces themeta-schema-nameconfiguration item. In parallel import mode, this parameter specifies the schema name that stores the meta information for each TiDB Lightning instance in the target cluster. By default, the value is "lightning_metadata". The value set for this parameter must be the same for each TiDB Lightning instance that participates in the same parallel import; otherwise, the correctness of the imported data cannot be ensured.

    User document

  • TiDB Lightning introduces duplicate resolution

    In Local-backend mode, TiDB Lightning outputs duplicated data before the data import is completed, and then removes that duplicated data from the database. You can resolve the duplicated data after the import is completed and select suitable data to insert according to application rules. It is recommended to clean upstream data sources based on duplicated data to avoid data inconsistency caused by duplicated data encountered in the subsequent incremental data migration phase.

    User document

  • Optimize the usage of relay log in TiDB Data Migration (DM)

    • 恢复enable-relayswitch in thesourceconfiguration.

    • Support dynamically enabling and disabling relay log using thestart-relayandstop-relaycommands.

    • Bind the status of relay log tosource.sourcekeeps its original status of being enabled or disabled after it is migrated to any DM-worker.

    • Move the storage path of relay log to the DM-worker configuration file.

      User document

  • Optimize the processing ofcollationin DM

    Add thecollation_compatibleconfiguration item. The value options areloose(default) andstrict:

    • If your application does not have strict requirements on collation, and the collation of query results can be different between the upstream and downstream, you can use the defaultloosemode to avoid reporting errors.

    • If your application has strict requirements on collation, and the collation must be consistent between the upstream and downstream, you can use thestrictmode. However, if the downstream does not support the upstream's default collation, the data replication might report errors.

      User document

  • Optimizetransfer sourcein DM to support running replication tasks smoothly

    When the DM-worker nodes have an unbalanced load, thetransfer sourcecommand can be used to manually transfer the configuration of asourceto another load. After the optimization, thetransfer sourcecommand simplifies the manual operation. You can smoothly transfer the source instead of pausing all related tasks, because DM completes other operations internally.

  • DM OpenAPI becomes generally available (GA)

    DM supports daily management via API, including adding data sources and managing tasks. In v5.4.0, DM OpenAPI becomes GA.

    User document

Diagnostic efficiency

  • Top SQL (experimental feature)

    A new experimental feature, Top SQL (disabled by default), is introduced to help you easily find source-consuming queries.

    User document

TiDB data share subscription

  • Optimize the impact of TiCDC on clusters

    Significantly reduces the impact on the performance of TiDB clusters when you use TiCDC. In the test environment, the performance impact of TiCDC on TiDB can be reduced to less than 5%.

Deployment and maintenance

  • Enhance Continuous Profiling (experimental)

    • More components supported: Besides TiDB, PD, and TiKV, TiDB v5.4.0 also supports CPU profiling of TiFlash.

    • More forms of profiling display: Supports showing CPU profiling and Goroutine results on flame charts.

    • More deployment environments supported: Continuous Profiling can also be used for clusters deployed using TiDB Operator.

      Continuous Profiling is disabled by default and can be enabled on TiDB Dashboard.

      Continuous Profiling is applicable to clusters deployed or upgraded using TiUP of v1.9.0 or later or TiDB Operator of v1.3.0 or later.

      User document

Improvements

  • TiDB

    • Support theADMIN {SESSION | INSTANCE | GLOBAL} PLAN_CACHEsyntax to clear the cached query plan# 30370
  • TiKV

    • Coprocessor supports paging API to process requests in a stream-like way#11448
    • Supportread-through-lockso that read operations do not need to wait for secondary locks to be resolved#11402
    • Add a disk protection mechanism to avoid panic caused by disk space drainage#10537
    • Support archiving and rotating logs#11651
    • Reduce the system call by the Raft client and increase CPU efficiency#11309
    • Coprocessor supports pushing down substring to TiKV#11495
    • Improve the scan performance by skip reading locks in the Read Committed isolation level#11485
    • Reduce the default thread pool size used by backup operations and limit the use of thread pool when the stress is high#11000
    • Support dynamically adjusting the sizes of the Apply thread pool and the Store thread pool#11159
    • Support configuring the size of thesnap-generatorthread pool#11247
    • Optimize the issue of global lock race that occurs when there are many files with frequent reads and writes#250
  • PD

    • Record the historic hotspot information by default#25281
    • Add signature for the HTTP component to identify the request source#4490
    • Update TiDB Dashboard to v2021.12.31#4257
  • TiFlash

    • Optimize the communication of local operators
    • Increase the non-temporary thread count of gRPC to avoid the frequent creation or destruction of threads
  • Tools

    • Backup & Restore (BR)

      • Add a validity check for the key when BR performs encrypted backup#29794
    • TiCDC

      • Reduce the count of "EventFeed retry rate limited" logs#4006
      • Reduce the replication latency when replicating many tables# 3900
      • Reduce the time for the KV client to recover when a TiKV store is down# 3191
    • TiDB Data Migration (DM)

      • Lower the usage rate of CPU when the relay is enabled#2214
    • TiDB Lightning

      • Use optimistic transactions by default to write data to improve performance in TiDB-backend mode# 30953
    • Dumpling

      • Improve compatibility when Dumpling checks the database version#29500
      • Add the default collation when dumpingCREATE DATABASEandCREATE TABLE# 3420

Bug fixes

  • TiDB

    • 修复issue of thetidb_analyze_versionvalue change that occurs when upgrading the cluster from v4.x to v5.x#25422
    • 修复issue of the wrong result that occurs when using different collations in a subquery# 30748
    • 修复issue that the result ofconcat(ifnull(time(3))in TiDB is different from that in MySQL#29498
    • 修复issue of potential data index inconsistency in optimistic transaction mode# 30410
    • 修复issue that the query execution plan of IndexMerge is wrong when an expression cannot be pushed down to TiKV# 30200
    • 修复issue that concurrent column type change causes inconsistency between the schema and the data# 31048
    • 修复issue that the IndexMerge query result is wrong when there is a subquery# 30913
    • 修复panic issue that occurs when the FetchSize is set too large in the client# 30896
    • 修复issue that LEFT JOIN might be mistakenly converted to INNER JOIN#20510
    • 修复issue that panic might occur when theCASE-WHENexpression and collation are used together# 30245
    • 修复issue of wrong query result that occurs when theINvalue contains a binary constant# 31261
    • 修复issue of wrong query result that occurs when CTE has a subquery# 31255
    • 修复issue that executing theINSERT ... SELECT ... ON DUPLICATE KEY UPDATEstatement gets panic#28078
    • 修复issue that INDEX HASH JOIN returns thesend on closed channelerror# 31129
  • TiKV

    • 修复issue that the MVCC deletion records are not cleared by GC#11217
    • 修复issue that retrying prewrite requests in the pessimistic transaction mode might cause the risk of data inconsistency in rare cases#11187
    • 修复issue that GC scan causes memory overflow#11410
    • 修复issue that RocksDB flush or compaction causes panic when the disk capacity is full#11224
  • PD

    • 修复issue that Region statistics are not affected byflow-round-by-digit#4295
    • 修复issue that the scheduling operator cannot fail fast because the target store is down# 3353
    • 修复issue that Regions on offline stores cannot be merged#4119
    • 修复issue that the cold hotspot data cannot be deleted from the hotspot statistics#4390
  • TiFlash

    • 修复issue that TiFlash might panic when an MPP query is stopped
    • 修复issue that queries with thewhere clause return wrong results
    • 修复potential issue of data inconsistency that might occur when setting the column type of an integer primary key to a larger range
    • 修复issue that when an input time is earlier than 1970-01-01 00:00:01 UTC, the behavior ofunix_timestampis inconsistent with that of TiDB or MySQL
    • 修复issue that TiFlash might return theEstablishMPPConnectionerror after it is restarted
    • 修复issue that theCastStringAsDecimalbehavior is inconsistent in TiFlash and in TiDB/TiKV
    • 修复issue that theDB::Exception: Encode type of coprocessor response is not CHBlockerror is returned in the query result
    • 修复issue that thecastStringAsRealbehavior is inconsistent in TiFlash and in TiDB/TiKV
    • 修复issue that the returned result of thedate_add_string_xxxfunction in TiFlash is inconsistent with that in MySQL
  • Tools

    • Backup & Restore (BR)

      • 修复potential issue that Region distribution might be uneven after a restore operation is finished# 30425
      • 修复issue that'/'cannot be specified in endpoint whenminiois used as the backup storage# 30104
      • 修复issue that system tables cannot be restored because concurrently backing up system tables makes the table name fail to update#29710
    • TiCDC

      • 修复issue that replication cannot be performed whenmin.insync.replicasis smaller thanreplication-factor# 3994
      • 修复issue that thecached regionmonitoring metric is negative#4300
      • 修复issue thatmq sink write rowdoes not have monitoring data# 3431
      • 修复compatibility issue ofsql mode# 3810
      • 修复potential panic issue that occurs when a replication task is removed# 3128
      • 修复issue of panic and data inconsistency that occurs when outputting the default column value# 3929
      • 修复issue that default values cannot be replicated# 3793
      • 修复potential issue that the deadlock causes a replication task to get stuck#4055
      • 修复issue that no log is output when the disk is fully written# 3362
      • 修复issue that special comments in DDL statements cause the replication task to stop# 3755
      • 修复issue that the service cannot be started because of a timezone issue in the RHEL release# 3584
      • 修复issue of potential data loss caused by inaccurate checkpoint# 3545
      • 修复OOM issue in the container environment#1798
      • 修复issue of replication stop caused by the incorrect configuration ofconfig.Metadata.Timeout# 3352
    • TiDB Data Migration (DM)

      • 修复issue that theCREATE VIEWstatement interrupts data replication#4173
      • 修复issue the schema needs to be reset after a DDL statement is skipped#4177
      • 修复的问题表检查点不是乌利希期刊指南ated in time after a DDL statement is skipped#4184
      • Fix a compatibility issue of the TiDB version with the Parser version#4298
      • 修复issue that syncer metrics are updated only when querying the status#4281
    • TiDB Lightning

      • 修复issue of wrong import result that occurs when TiDB Lightning does not have the privilege to access themysql.tidbtable# 31088
      • 修复issue that some checks are skipped when TiDB Lightning is restarted# 30772
      • 修复issue that TiDB Lightning fails to report the error when the S3 path does not exist# 30674
    • TiDB Binlog

      • 修复issue that Drainer fails because it is incompatible with theCREATE PLACEMENT POLICYstatement#1118
Download PDF Request docs changes Ask questions on Discord
Playground
New
One-stop & interactive experience of TiDB's capabilities WITHOUT registration.
Was this page helpful?
Products
TiDB
TiDB Dedicated
TiDB Serverless
Pricing
Get Demo
Get Started
©2023PingCAP. All Rights Reserved.