[fix](group_commit) Need to wait wal to be deleted when creating MaterializedView (#30956)
This commit is contained in:
@ -218,6 +218,10 @@ public class MaterializedViewHandler extends AlterHandler {
|
||||
|
||||
olapTable.setState(OlapTableState.ROLLUP);
|
||||
|
||||
// wait wal delete
|
||||
Env.getCurrentEnv().getGroupCommitManager().blockTable(olapTable.getId());
|
||||
Env.getCurrentEnv().getGroupCommitManager().waitWalFinished(olapTable.getId());
|
||||
|
||||
Env.getCurrentEnv().getEditLog().logAlterJob(rollupJobV2);
|
||||
LOG.info("finished to create materialized view job: {}", rollupJobV2.getJobId());
|
||||
} finally {
|
||||
@ -300,6 +304,11 @@ public class MaterializedViewHandler extends AlterHandler {
|
||||
// ATTN: This order is not mandatory, because database lock will protect us,
|
||||
// but this order is more reasonable
|
||||
olapTable.setState(OlapTableState.ROLLUP);
|
||||
|
||||
// wait wal delete
|
||||
Env.getCurrentEnv().getGroupCommitManager().blockTable(olapTable.getId());
|
||||
Env.getCurrentEnv().getGroupCommitManager().waitWalFinished(olapTable.getId());
|
||||
|
||||
// 2 batch submit rollup job
|
||||
List<AlterJobV2> rollupJobV2List = new ArrayList<>(rollupNameJobMap.values());
|
||||
batchAddAlterJobV2(rollupJobV2List);
|
||||
@ -1169,6 +1178,7 @@ public class MaterializedViewHandler extends AlterHandler {
|
||||
private void onJobDone(AlterJobV2 alterJob) {
|
||||
removeJobFromRunningQueue(alterJob);
|
||||
if (removeAlterJobV2FromTableNotFinalStateJobMap(alterJob)) {
|
||||
Env.getCurrentEnv().getGroupCommitManager().unblockTable(alterJob.getTableId());
|
||||
changeTableStatus(alterJob.getDbId(), alterJob.getTableId(), OlapTableState.NORMAL);
|
||||
LOG.info("set table's state to NORMAL, table id: {}, job id: {}", alterJob.getTableId(),
|
||||
alterJob.getJobId());
|
||||
|
||||
@ -40,7 +40,6 @@ import org.apache.doris.catalog.Tablet;
|
||||
import org.apache.doris.catalog.TabletInvertedIndex;
|
||||
import org.apache.doris.catalog.TabletMeta;
|
||||
import org.apache.doris.common.AnalysisException;
|
||||
import org.apache.doris.common.Config;
|
||||
import org.apache.doris.common.FeConstants;
|
||||
import org.apache.doris.common.MarkedCountDownLatch;
|
||||
import org.apache.doris.common.MetaNotFoundException;
|
||||
@ -537,7 +536,9 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
|
||||
}
|
||||
return;
|
||||
}
|
||||
waitWalFinished();
|
||||
Env.getCurrentEnv().getGroupCommitManager().blockTable(tableId);
|
||||
Env.getCurrentEnv().getGroupCommitManager().waitWalFinished(tableId);
|
||||
Env.getCurrentEnv().getGroupCommitManager().unblockTable(tableId);
|
||||
/*
|
||||
* all tasks are finished. check the integrity.
|
||||
* we just check whether all new replicas are healthy.
|
||||
@ -600,34 +601,6 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
|
||||
LOG.info("set table's state to NORMAL, table id: {}, job id: {}", tableId, jobId);
|
||||
}
|
||||
|
||||
private void waitWalFinished() {
|
||||
// wait wal done here
|
||||
Env.getCurrentEnv().getGroupCommitManager().blockTable(tableId);
|
||||
LOG.info("block group commit for table={} when schema change", tableId);
|
||||
List<Long> aliveBeIds = Env.getCurrentSystemInfo().getAllBackendIds(true);
|
||||
long expireTime = System.currentTimeMillis() + Config.check_wal_queue_timeout_threshold;
|
||||
while (true) {
|
||||
LOG.info("wait for wal queue size to be empty");
|
||||
boolean walFinished = Env.getCurrentEnv().getGroupCommitManager()
|
||||
.isPreviousWalFinished(tableId, aliveBeIds);
|
||||
if (walFinished) {
|
||||
LOG.info("all wal is finished for table={}", tableId);
|
||||
break;
|
||||
} else if (System.currentTimeMillis() > expireTime) {
|
||||
LOG.warn("waitWalFinished time out for table={}", tableId);
|
||||
break;
|
||||
} else {
|
||||
try {
|
||||
Thread.sleep(100);
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.warn("failed to wait for wal for table={} when schema change", tableId, ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
Env.getCurrentEnv().getGroupCommitManager().unblockTable(tableId);
|
||||
LOG.info("unblock group commit for table={} when schema change", tableId);
|
||||
}
|
||||
|
||||
private void onFinished(OlapTable tbl) {
|
||||
// replace the origin index with shadow index, set index state as NORMAL
|
||||
for (Partition partition : tbl.getPartitions()) {
|
||||
|
||||
@ -45,11 +45,39 @@ public class GroupCommitManager {
|
||||
}
|
||||
|
||||
public void blockTable(long tableId) {
|
||||
LOG.info("block group commit for table={} when schema change", tableId);
|
||||
blockedTableIds.add(tableId);
|
||||
}
|
||||
|
||||
public void unblockTable(long tableId) {
|
||||
blockedTableIds.remove(tableId);
|
||||
LOG.info("unblock group commit for table={} when schema change", tableId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Waiting All WAL files to be deleted.
|
||||
*/
|
||||
public void waitWalFinished(long tableId) {
|
||||
List<Long> aliveBeIds = Env.getCurrentSystemInfo().getAllBackendIds(true);
|
||||
long expireTime = System.currentTimeMillis() + Config.check_wal_queue_timeout_threshold;
|
||||
while (true) {
|
||||
LOG.info("wait for wal queue size to be empty");
|
||||
boolean walFinished = Env.getCurrentEnv().getGroupCommitManager()
|
||||
.isPreviousWalFinished(tableId, aliveBeIds);
|
||||
if (walFinished) {
|
||||
LOG.info("all wal is finished for table={}", tableId);
|
||||
break;
|
||||
} else if (System.currentTimeMillis() > expireTime) {
|
||||
LOG.warn("waitWalFinished time out for table={}", tableId);
|
||||
break;
|
||||
} else {
|
||||
try {
|
||||
Thread.sleep(100);
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.warn("failed to wait for wal for table={} when schema change", tableId, ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
Reference in New Issue
Block a user