From ea8738de3dc89711e630fb9ca362836861d3b422 Mon Sep 17 00:00:00 2001 From: yujun777 Date: Tue, 10 Oct 2023 16:38:48 +0800 Subject: [PATCH 1/3] impr fe sync version with be --- .../org/apache/doris/catalog/Replica.java | 36 +++- .../doris/catalog/TabletInvertedIndex.java | 26 ++- .../apache/doris/clone/TabletSchedCtx.java | 7 + .../org/apache/doris/master/MasterImpl.java | 9 +- .../apache/doris/master/ReportHandler.java | 41 +++- .../apache/doris/clone/RepairVersionTest.java | 177 ++++++++++++++++++ .../doris/utframe/TestWithFeService.java | 2 +- 7 files changed, 276 insertions(+), 22 deletions(-) create mode 100644 fe/fe-core/src/test/java/org/apache/doris/clone/RepairVersionTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java index e55eab8939216f..8ee6c61e6ba590 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java @@ -20,6 +20,7 @@ import org.apache.doris.common.Config; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; +import org.apache.doris.common.util.DebugPointUtil; import org.apache.doris.thrift.TUniqueId; import com.google.gson.annotations.SerializedName; @@ -114,6 +115,14 @@ public enum ReplicaStatus { private TUniqueId cooldownMetaId; private long cooldownTerm = -1; + // A replica version should increase monotonically, + // but backend may missing some versions due to disk failure or bugs. + // FE should found these and mark the replica as missing versions. + // If backend's report version < fe version, record the backend's report version as `regressiveVersion`, + // and if time exceed 5min, fe should mark this replica miss versions. + private long regressiveVersion = -1; + private long regressiveVersionTimestamp = 0; + /* * This can happen when this replica is created by a balance clone task, and * when task finished, the version of this replica is behind the partition's visible version. @@ -435,9 +444,9 @@ private void updateReplicaInfo(long newVersion, if (lastFailedVersion != this.lastFailedVersion) { // Case 2: - if (lastFailedVersion > this.lastFailedVersion) { + if (lastFailedVersion > this.lastFailedVersion || lastFailedVersion < 0) { this.lastFailedVersion = lastFailedVersion; - this.lastFailedTimestamp = System.currentTimeMillis(); + this.lastFailedTimestamp = lastFailedVersion > 0 ? System.currentTimeMillis() : -1L; } this.lastSuccessVersion = this.version; @@ -506,10 +515,6 @@ public boolean checkVersionCatchUp(long expectedVersion, boolean ignoreAlter) { return true; } - public void setLastFailedVersion(long lastFailedVersion) { - this.lastFailedVersion = lastFailedVersion; - } - public void setState(ReplicaState replicaState) { this.state = replicaState; } @@ -534,6 +539,25 @@ public void setVersionCount(long versionCount) { this.versionCount = versionCount; } + public boolean checkVersionRegressive(long newVersion) { + if (newVersion >= version) { + regressiveVersion = -1; + regressiveVersionTimestamp = -1; + return false; + } + + if (DebugPointUtil.isEnable("Replica.regressive_version_immediately")) { + return true; + } + + if (newVersion != regressiveVersion) { + regressiveVersion = newVersion; + regressiveVersionTimestamp = System.currentTimeMillis(); + } + + return System.currentTimeMillis() - regressiveVersionTimestamp >= 5 * 60 * 1000L; + } + @Override public String toString() { StringBuilder strBuffer = new StringBuilder("[replicaId="); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java index 2b601f9f0306a4..52ed4a28f216e6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java @@ -390,10 +390,22 @@ private boolean needSync(Replica replicaInFe, TTabletInfo backendTabletInfo) { if (backendTabletInfo.getVersion() > versionInFe) { // backend replica's version is larger or newer than replica in FE, sync it. return true; - } else if (versionInFe == backendTabletInfo.getVersion() && replicaInFe.isBad()) { - // backend replica's version is equal to replica in FE, but replica in FE is bad, - // while backend replica is good, sync it - return true; + } else if (versionInFe == backendTabletInfo.getVersion()) { + // backend replica's version is equal to replica in FE, but replica in FE is bad, + // while backend replica is good, sync it + if (replicaInFe.isBad()) { + return true; + } + + // FE' s replica last failed version > partition's committed version + // this can be occur when be report miss version, fe will set last failed version = visible version + 1 + // then last failed version may greater than partition's committed version + // + // But here cannot got variable partition, we just check lastFailedVersion = version + 1, + // In ReportHandler.sync, we will check if last failed version > partition's committed version again. + if (replicaInFe.getLastFailedVersion() == versionInFe + 1) { + return true; + } } return false; @@ -501,6 +513,12 @@ private boolean needRecover(Replica replicaInFe, int schemaHashInFe, TTabletInfo // so we only return true if version_miss is true. return true; } + + // backend versions regressive due to bugs + if (replicaInFe.checkVersionRegressive(backendTabletInfo.getVersion())) { + return true; + } + return false; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java index c9671cd2db77b4..c57bdc7762a939 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java @@ -1074,6 +1074,13 @@ public void finishCloneTask(CloneTask cloneTask, TFinishTaskRequest request) replica.updateVersionInfo(reportedTablet.getVersion(), reportedTablet.getDataSize(), reportedTablet.getDataSize(), reportedTablet.getRowCount()); + if (replica.getLastFailedVersion() > partition.getCommittedVersion() + && reportedTablet.getVersion() >= partition.getCommittedVersion() + //&& !(reportedTablet.isSetVersionMiss() && reportedTablet.isVersionMiss() + && !(reportedTablet.isSetUsed() && !reportedTablet.isUsed())) { + LOG.info("change replica {} of tablet {} 's last failed version to -1", replica, tabletId); + replica.updateLastFailedVersion(-1L); + } if (reportedTablet.isSetPathHash()) { replica.setPathHash(reportedTablet.getPathHash()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java b/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java index 2833eff5f3d0e8..64b771663b2465 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java @@ -192,7 +192,7 @@ public TMasterResult finishTask(TFinishTaskRequest request) { finishRecoverTablet(task); break; case ALTER: - finishAlterTask(task); + finishAlterTask(task, request); break; case ALTER_INVERTED_INDEX: finishAlterInvertedIndexTask(task, request); @@ -575,7 +575,7 @@ public TMasterResult report(TReportRequest request) throws TException { return reportHandler.handleReport(request); } - private void finishAlterTask(AgentTask task) { + private void finishAlterTask(AgentTask task, TFinishTaskRequest request) { AlterReplicaTask alterTask = (AlterReplicaTask) task; try { if (alterTask.getJobType() == JobType.ROLLUP) { @@ -584,6 +584,11 @@ private void finishAlterTask(AgentTask task) { Env.getCurrentEnv().getSchemaChangeHandler().handleFinishAlterTask(alterTask); } alterTask.setFinished(true); + if (request.isSetReportVersion()) { + long reportVersion = request.getReportVersion(); + Env.getCurrentSystemInfo().updateBackendReportVersion( + task.getBackendId(), reportVersion, task.getDbId(), task.getTableId()); + } } catch (MetaNotFoundException e) { LOG.warn("failed to handle finish alter task: {}, {}", task.getSignature(), e.getMessage()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java index 4461ba19473a02..530a284bc93c67 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java @@ -403,7 +403,8 @@ private static void diffResource(List storageResourcesInBe, Li } } - private static void tabletReport(long backendId, Map backendTablets, long backendReportVersion) { + // public for fe ut + public static void tabletReport(long backendId, Map backendTablets, long backendReportVersion) { long start = System.currentTimeMillis(); LOG.info("backend[{}] reports {} tablet(s). report version: {}", backendId, backendTablets.size(), backendReportVersion); @@ -607,6 +608,11 @@ private static void sync(Map backendTablets, ListMultimap backendTablets, ListMultimap= partition.getCommittedVersion() + && replica.getLastFailedVersion() > partition.getCommittedVersion()) { + LOG.info("sync replica {} of tablet {} in backend {} in db {}. replica last failed version" + + " change to -1 because last failed version > replica's committed version {}", + replica, tabletId, backendId, dbId, partition.getCommittedVersion()); + replica.updateLastFailedVersion(-1L); + needSync = true; + } + } + if (needSync) { // happens when // 1. PUSH finished in BE but failed or not yet report to FE // 2. repair for VERSION_INCOMPLETE finished in BE, but failed or not yet report to FE @@ -1048,18 +1064,25 @@ private static void handleRecoverTablet(ListMultimap tabletRecoveryM break; } - if (tTabletInfo.isSetVersionMiss() && tTabletInfo.isVersionMiss()) { + if ((tTabletInfo.isSetVersionMiss() && tTabletInfo.isVersionMiss()) + || replica.checkVersionRegressive(tTabletInfo.getVersion())) { // If the origin last failed version is larger than 0, not change it. // Otherwise, we set last failed version to replica'version + 1. // Because last failed version should always larger than replica's version. long newLastFailedVersion = replica.getLastFailedVersion(); if (newLastFailedVersion < 0) { newLastFailedVersion = replica.getVersion() + 1; + replica.updateLastFailedVersion(newLastFailedVersion); + LOG.warn("set missing version for replica {} of tablet {} on backend {}, " + + "version in fe {}, version in be {}, be missing {}", + replica.getId(), tabletId, backendId, replica.getVersion(), + tTabletInfo.getVersion(), tTabletInfo.isVersionMiss()); } - replica.updateLastFailedVersion(newLastFailedVersion); backendReplicasInfo.addMissingVersionReplica(tabletId, newLastFailedVersion); break; } + + break; } } } finally { diff --git a/fe/fe-core/src/test/java/org/apache/doris/clone/RepairVersionTest.java b/fe/fe-core/src/test/java/org/apache/doris/clone/RepairVersionTest.java new file mode 100644 index 00000000000000..3064c546cca5ae --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/clone/RepairVersionTest.java @@ -0,0 +1,177 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.clone; + +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.Tablet; +import org.apache.doris.common.Config; +import org.apache.doris.common.FeConstants; +import org.apache.doris.common.util.DebugPointUtil; +import org.apache.doris.master.ReportHandler; +import org.apache.doris.thrift.TTablet; +import org.apache.doris.thrift.TTabletInfo; +import org.apache.doris.utframe.TestWithFeService; + +import com.google.common.collect.Maps; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Map; + +public class RepairVersionTest extends TestWithFeService { + private class TableInfo { + Partition partition; + Tablet tablet; + Replica replica; + } + + @Override + protected void beforeCreatingConnectContext() throws Exception { + needCleanDir = false; + Config.enable_debug_points = true; + Config.disable_balance = true; + Config.disable_tablet_scheduler = true; + Config.allow_replica_on_same_host = true; + FeConstants.tablet_checker_interval_ms = 100; + FeConstants.tablet_schedule_interval_ms = 100; + } + + @Override + protected void runBeforeAll() throws Exception { + createDatabase("test"); + } + + @Override + protected int backendNum() { + return 2; + } + + @Test + public void testRepairLastFailedVersionByClone() throws Exception { + TableInfo info = prepareTableForTest("tbl_repair_last_fail_version_by_clone"); + Partition partition = info.partition; + Replica replica = info.replica; + + replica.updateLastFailedVersion(replica.getVersion() + 1); + Assertions.assertEquals(partition.getCommittedVersion() + 1, replica.getLastFailedVersion()); + + Config.disable_tablet_scheduler = false; + Thread.sleep(1000); + Config.disable_tablet_scheduler = true; + + Assertions.assertEquals(partition.getVisibleVersion(), replica.getVersion()); + Assertions.assertEquals(-1L, replica.getLastFailedVersion()); + } + + @Test + public void testRepairLastFailedVersionByReport() throws Exception { + TableInfo info = prepareTableForTest("tbl_repair_last_fail_version_by_report"); + Partition partition = info.partition; + Tablet tablet = info.tablet; + Replica replica = info.replica; + + replica.updateLastFailedVersion(replica.getVersion() + 1); + Assertions.assertEquals(partition.getCommittedVersion() + 1, replica.getLastFailedVersion()); + + TTabletInfo tTabletInfo = new TTabletInfo(); + tTabletInfo.setTabletId(tablet.getId()); + tTabletInfo.setSchemaHash(replica.getSchemaHash()); + tTabletInfo.setVersion(replica.getVersion()); + tTabletInfo.setPathHash(replica.getPathHash()); + tTabletInfo.setPartitionId(partition.getId()); + tTabletInfo.setReplicaId(replica.getId()); + + TTablet tTablet = new TTablet(); + tTablet.addToTabletInfos(tTabletInfo); + Map tablets = Maps.newHashMap(); + tablets.put(tablet.getId(), tTablet); + Assertions.assertEquals(partition.getVisibleVersion(), replica.getVersion()); + + ReportHandler.tabletReport(replica.getBackendId(), tablets, 100L); + + Assertions.assertEquals(partition.getVisibleVersion(), replica.getVersion()); + Assertions.assertEquals(-1L, replica.getLastFailedVersion()); + } + + @Test + public void testVersionRegressive() throws Exception { + TableInfo info = prepareTableForTest("tbl_version_regressive"); + Partition partition = info.partition; + Tablet tablet = info.tablet; + Replica replica = info.replica; + + Assertions.assertEquals(partition.getVisibleVersion(), replica.getVersion()); + Assertions.assertEquals(-1L, replica.getLastFailedVersion()); + Assertions.assertTrue(replica.getVersion() > 1L); + + TTabletInfo tTabletInfo = new TTabletInfo(); + tTabletInfo.setTabletId(tablet.getId()); + tTabletInfo.setSchemaHash(replica.getSchemaHash()); + tTabletInfo.setVersion(1L); // be report version = 1 which less than fe version + tTabletInfo.setPathHash(replica.getPathHash()); + tTabletInfo.setPartitionId(partition.getId()); + tTabletInfo.setReplicaId(replica.getId()); + + TTablet tTablet = new TTablet(); + tTablet.addToTabletInfos(tTabletInfo); + Map tablets = Maps.newHashMap(); + tablets.put(tablet.getId(), tTablet); + + ReportHandler.tabletReport(replica.getBackendId(), tablets, 100L); + Assertions.assertEquals(-1L, replica.getLastFailedVersion()); + + DebugPointUtil.addDebugPoint("Replica.regressive_version_immediately", -1, -1); + ReportHandler.tabletReport(replica.getBackendId(), tablets, 100L); + Assertions.assertEquals(replica.getVersion() + 1, replica.getLastFailedVersion()); + + Assertions.assertEquals(partition.getVisibleVersion(), replica.getVersion()); + } + + private TableInfo prepareTableForTest(String tableName) throws Exception { + createTable("CREATE TABLE test." + tableName + " (k INT) DISTRIBUTED BY HASH(k) " + + " BUCKETS 1 PROPERTIES ( \"replication_num\" = \"2\" )"); + + Database db = Env.getCurrentInternalCatalog().getDbOrMetaException("default_cluster:test"); + OlapTable tbl = (OlapTable) db.getTableOrMetaException(tableName); + Assertions.assertNotNull(tbl); + Partition partition = tbl.getPartitions().iterator().next(); + Tablet tablet = partition.getMaterializedIndices(MaterializedIndex.IndexExtState.ALL).iterator().next() + .getTablets().iterator().next(); + + long visibleVersion = 2L; + partition.updateVisibleVersion(visibleVersion); + partition.setNextVersion(visibleVersion + 1); + tablet.getReplicas().forEach(replica -> replica.updateVersionInfo(visibleVersion, 1L, 1L, 1L)); + + Replica replica = tablet.getReplicas().iterator().next(); + Assertions.assertEquals(visibleVersion, replica.getVersion()); + Assertions.assertEquals(-1L, replica.getLastFailedVersion()); + + TableInfo info = new TableInfo(); + info.partition = partition; + info.tablet = tablet; + info.replica = replica; + + return info; + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java index 8861112624f06f..3aa3a464c7302a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java +++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java @@ -392,7 +392,7 @@ protected void createDorisCluster(String runningDir, int backendNum) InterruptedException { int feRpcPort = startFEServer(runningDir); List bes = Lists.newArrayList(); - System.out.println("start create backend"); + System.out.println("start create backend, backend num " + backendNum); for (int i = 0; i < backendNum; i++) { bes.add(createBackend("127.0.0.1", feRpcPort)); } From e4ef1385f8059f6671325bde09794501b6cab033 Mon Sep 17 00:00:00 2001 From: yujun777 Date: Wed, 11 Oct 2023 09:33:05 +0800 Subject: [PATCH 2/3] fix compile --- .../org/apache/doris/catalog/Replica.java | 2 +- .../doris/catalog/TabletInvertedIndex.java | 28 +++++++++---------- .../apache/doris/master/ReportHandler.java | 5 ++-- 3 files changed, 18 insertions(+), 17 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java index 8ee6c61e6ba590..631f2ebaf3ba05 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java @@ -119,7 +119,7 @@ public enum ReplicaStatus { // but backend may missing some versions due to disk failure or bugs. // FE should found these and mark the replica as missing versions. // If backend's report version < fe version, record the backend's report version as `regressiveVersion`, - // and if time exceed 5min, fe should mark this replica miss versions. + // and if time exceed 5min, fe should mark this replica as missing versions. private long regressiveVersion = -1; private long regressiveVersionTimestamp = 0; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java index 52ed4a28f216e6..a2d5983aac41c9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java @@ -391,21 +391,21 @@ private boolean needSync(Replica replicaInFe, TTabletInfo backendTabletInfo) { // backend replica's version is larger or newer than replica in FE, sync it. return true; } else if (versionInFe == backendTabletInfo.getVersion()) { - // backend replica's version is equal to replica in FE, but replica in FE is bad, - // while backend replica is good, sync it - if (replicaInFe.isBad()) { - return true; - } + // backend replica's version is equal to replica in FE, but replica in FE is bad, + // while backend replica is good, sync it + if (replicaInFe.isBad()) { + return true; + } - // FE' s replica last failed version > partition's committed version - // this can be occur when be report miss version, fe will set last failed version = visible version + 1 - // then last failed version may greater than partition's committed version - // - // But here cannot got variable partition, we just check lastFailedVersion = version + 1, - // In ReportHandler.sync, we will check if last failed version > partition's committed version again. - if (replicaInFe.getLastFailedVersion() == versionInFe + 1) { - return true; - } + // FE' s replica last failed version > partition's committed version + // this can be occur when be report miss version, fe will set last failed version = visible version + 1 + // then last failed version may greater than partition's committed version + // + // But here cannot got variable partition, we just check lastFailedVersion = version + 1, + // In ReportHandler.sync, we will check if last failed version > partition's committed version again. + if (replicaInFe.getLastFailedVersion() == versionInFe + 1) { + return true; + } } return false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java index 530a284bc93c67..5b781d7c5f0ec1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java @@ -675,8 +675,9 @@ private static void sync(Map backendTablets, ListMultimap= partition.getCommittedVersion() && replica.getLastFailedVersion() > partition.getCommittedVersion()) { - LOG.info("sync replica {} of tablet {} in backend {} in db {}. replica last failed version" - + " change to -1 because last failed version > replica's committed version {}", + LOG.info("sync replica {} of tablet {} in backend {} in db {}. replica last failed" + + " version change to -1 because last failed version > replica's committed" + + " version {}", replica, tabletId, backendId, dbId, partition.getCommittedVersion()); replica.updateLastFailedVersion(-1L); needSync = true; From 3e4de8a26feecee74d1765078094bd23c2dd7063 Mon Sep 17 00:00:00 2001 From: yujun777 Date: Wed, 11 Oct 2023 21:42:59 +0800 Subject: [PATCH 3/3] update master --- .../java/org/apache/doris/clone/RepairVersionTest.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/fe/fe-core/src/test/java/org/apache/doris/clone/RepairVersionTest.java b/fe/fe-core/src/test/java/org/apache/doris/clone/RepairVersionTest.java index 3064c546cca5ae..7539548583c502 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/clone/RepairVersionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/clone/RepairVersionTest.java @@ -25,8 +25,8 @@ import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.Tablet; import org.apache.doris.common.Config; -import org.apache.doris.common.FeConstants; import org.apache.doris.common.util.DebugPointUtil; +import org.apache.doris.common.util.DebugPointUtil.DebugPoint; import org.apache.doris.master.ReportHandler; import org.apache.doris.thrift.TTablet; import org.apache.doris.thrift.TTabletInfo; @@ -47,13 +47,12 @@ private class TableInfo { @Override protected void beforeCreatingConnectContext() throws Exception { - needCleanDir = false; Config.enable_debug_points = true; Config.disable_balance = true; Config.disable_tablet_scheduler = true; Config.allow_replica_on_same_host = true; - FeConstants.tablet_checker_interval_ms = 100; - FeConstants.tablet_schedule_interval_ms = 100; + Config.tablet_checker_interval_ms = 100; + Config.tablet_schedule_interval_ms = 100; } @Override @@ -140,7 +139,7 @@ public void testVersionRegressive() throws Exception { ReportHandler.tabletReport(replica.getBackendId(), tablets, 100L); Assertions.assertEquals(-1L, replica.getLastFailedVersion()); - DebugPointUtil.addDebugPoint("Replica.regressive_version_immediately", -1, -1); + DebugPointUtil.addDebugPoint("Replica.regressive_version_immediately", new DebugPoint()); ReportHandler.tabletReport(replica.getBackendId(), tablets, 100L); Assertions.assertEquals(replica.getVersion() + 1, replica.getLastFailedVersion());