From 1be06659ba65813fa836e5f3b8970430ac981253 Mon Sep 17 00:00:00 2001 From: guoleiyi Date: Sat, 12 Feb 2022 22:31:03 +0800 Subject: [PATCH 1/7] Remove old rollup job schema change job decomission job --- .../org/apache/doris/alter/AlterHandler.java | 221 --- .../java/org/apache/doris/alter/AlterJob.java | 360 ----- .../org/apache/doris/alter/AlterJobV2.java | 2 +- .../doris/alter/DecommissionBackendJob.java | 369 ----- .../apache/doris/alter/DecommissionType.java | 6 + .../doris/alter/MaterializedViewHandler.java | 176 +-- .../org/apache/doris/alter/RollupJob.java | 1180 -------------- .../doris/alter/SchemaChangeHandler.java | 183 +-- .../apache/doris/alter/SchemaChangeJob.java | 1385 ----------------- .../org/apache/doris/alter/SystemHandler.java | 36 - .../analysis/DecommissionBackendClause.java | 2 +- .../org/apache/doris/catalog/Catalog.java | 91 +- .../org/apache/doris/load/LoadChecker.java | 24 +- .../org/apache/doris/metric/MetricRepo.java | 2 +- .../org/apache/doris/persist/EditLog.java | 92 -- .../org/apache/doris/catalog/CatalogTest.java | 50 +- .../org/apache/doris/catalog/FakeEditLog.java | 24 - 17 files changed, 31 insertions(+), 4172 deletions(-) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/alter/AlterJob.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/alter/DecommissionBackendJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/alter/DecommissionType.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java index 9aefdbbd6da330..909d6784a2da30 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java @@ -17,7 +17,6 @@ package org.apache.doris.alter; -import org.apache.doris.alter.AlterJob.JobState; import org.apache.doris.analysis.AlterClause; import org.apache.doris.analysis.CancelStmt; import org.apache.doris.catalog.Catalog; @@ -28,7 +27,6 @@ import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Tablet; -import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.FeConstants; import org.apache.doris.common.MetaNotFoundException; @@ -37,9 +35,7 @@ import org.apache.doris.common.util.TimeUtils; import org.apache.doris.persist.RemoveAlterJobV2OperationLog; import org.apache.doris.persist.ReplicaPersistInfo; -import org.apache.doris.task.AgentTask; import org.apache.doris.task.AlterReplicaTask; -import org.apache.doris.thrift.TTabletInfo; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; @@ -51,20 +47,12 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.locks.ReentrantLock; public abstract class AlterHandler extends MasterDaemon { private static final Logger LOG = LogManager.getLogger(AlterHandler.class); - // tableId -> AlterJob - @Deprecated - protected ConcurrentHashMap alterJobs = new ConcurrentHashMap<>(); - @Deprecated - protected ConcurrentLinkedQueue finishedOrCancelledAlterJobs = new ConcurrentLinkedQueue<>(); - // queue of alter job v2 protected ConcurrentMap alterJobsV2 = Maps.newConcurrentMap(); @@ -123,22 +111,6 @@ public Map getAlterJobsV2() { return this.alterJobsV2; } - // should be removed in version 0.13 - @Deprecated - private void clearExpireFinishedOrCancelledAlterJobs() { - long curTime = System.currentTimeMillis(); - // clean history job - Iterator iter = finishedOrCancelledAlterJobs.iterator(); - while (iter.hasNext()) { - AlterJob historyJob = iter.next(); - if ((curTime - historyJob.getCreateTimeMs()) / 1000 > Config.history_job_keep_max_second) { - iter.remove(); - LOG.info("remove history {} job[{}]. finish at {}", historyJob.getType(), - historyJob.getTableId(), TimeUtils.longToTimeString(historyJob.getFinishedTime())); - } - } - } - private void clearExpireFinishedOrCancelledAlterJobsV2() { Iterator> iterator = alterJobsV2.entrySet().iterator(); while (iterator.hasNext()) { @@ -162,61 +134,6 @@ public void replayRemoveAlterJobV2(RemoveAlterJobV2OperationLog log) { } } - @Deprecated - protected void addAlterJob(AlterJob alterJob) { - this.alterJobs.put(alterJob.getTableId(), alterJob); - LOG.info("add {} job[{}]", alterJob.getType(), alterJob.getTableId()); - } - - @Deprecated - public AlterJob getAlterJob(long tableId) { - return this.alterJobs.get(tableId); - } - - @Deprecated - public boolean hasUnfinishedAlterJob(long tableId) { - return this.alterJobs.containsKey(tableId); - } - - @Deprecated - public int getAlterJobNum(JobState state, long dbId) { - int jobNum = 0; - if (state == JobState.PENDING || state == JobState.RUNNING || state == JobState.FINISHING) { - for (AlterJob alterJob : alterJobs.values()) { - if (alterJob.getState() == state && alterJob.getDbId() == dbId) { - ++jobNum; - } - } - } else if (state == JobState.FINISHED) { - // lock to perform atomically - lock(); - try { - for (AlterJob alterJob : alterJobs.values()) { - if (alterJob.getState() == JobState.FINISHED && alterJob.getDbId() == dbId) { - ++jobNum; - } - } - - for (AlterJob alterJob : finishedOrCancelledAlterJobs) { - if (alterJob.getState() == JobState.FINISHED && alterJob.getDbId() == dbId) { - ++jobNum; - } - } - } finally { - unlock(); - } - - } else if (state == JobState.CANCELLED) { - for (AlterJob alterJob : finishedOrCancelledAlterJobs) { - if (alterJob.getState() == JobState.CANCELLED && alterJob.getDbId() == dbId) { - ++jobNum; - } - } - } - - return jobNum; - } - public Long getAlterJobV2Num(org.apache.doris.alter.AlterJobV2.JobState state, long dbId) { return alterJobsV2.values().stream().filter(e -> e.getJobState() == state && e.getDbId() == dbId).count(); } @@ -225,135 +142,8 @@ public Long getAlterJobV2Num(org.apache.doris.alter.AlterJobV2.JobState state) { return alterJobsV2.values().stream().filter(e -> e.getJobState() == state).count(); } - @Deprecated - public Map unprotectedGetAlterJobs() { - return this.alterJobs; - } - - @Deprecated - public ConcurrentLinkedQueue unprotectedGetFinishedOrCancelledAlterJobs() { - return this.finishedOrCancelledAlterJobs; - } - - @Deprecated - public void addFinishedOrCancelledAlterJob(AlterJob alterJob) { - alterJob.clear(); - LOG.info("add {} job[{}] to finished or cancel list", alterJob.getType(), alterJob.getTableId()); - this.finishedOrCancelledAlterJobs.add(alterJob); - } - - @Deprecated - protected AlterJob removeAlterJob(long tableId) { - return this.alterJobs.remove(tableId); - } - - @Deprecated - public void removeDbAlterJob(long dbId) { - Iterator> iterator = alterJobs.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - AlterJob alterJob = entry.getValue(); - if (alterJob.getDbId() == dbId) { - iterator.remove(); - } - } - } - - /* - * handle task report - * reportVersion is used in schema change job. - */ - @Deprecated - public void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo, long reportVersion) - throws MetaNotFoundException { - long tableId = task.getTableId(); - - AlterJob alterJob = getAlterJob(tableId); - if (alterJob == null) { - throw new MetaNotFoundException("Cannot find " + task.getTaskType().name() + " job[" + tableId + "]"); - } - alterJob.handleFinishedReplica(task, finishTabletInfo, reportVersion); - } - - protected void cancelInternal(AlterJob alterJob, OlapTable olapTable, String msg) { - // cancel - if (olapTable != null) { - olapTable.writeLock(); - } - try { - alterJob.cancel(olapTable, msg); - } finally { - if (olapTable != null) { - olapTable.writeUnlock(); - } - } - jobDone(alterJob); - } - - protected void jobDone(AlterJob alterJob) { - lock(); - try { - // remove job - AlterJob alterJobRemoved = removeAlterJob(alterJob.getTableId()); - // add to finishedOrCancelledAlterJobs - if (alterJobRemoved != null) { - // add alterJob not alterJobRemoved, because the alterJob maybe a new object - // deserialized from journal, and the finished state is set to the new object - addFinishedOrCancelledAlterJob(alterJob); - } - } finally { - unlock(); - } - } - - public void replayInitJob(AlterJob alterJob, Catalog catalog) throws MetaNotFoundException { - try { - Database db = catalog.getDbOrMetaException(alterJob.getDbId()); - alterJob.replayInitJob(db); - } finally { - // add rollup job - addAlterJob(alterJob); - } - } - - public void replayFinishing(AlterJob alterJob, Catalog catalog) throws MetaNotFoundException { - try { - Database db = catalog.getDbOrMetaException(alterJob.getDbId()); - alterJob.replayFinishing(db); - } finally { - alterJob.setState(JobState.FINISHING); - // !!! the alter job should add to the cache again, because the alter job is deserialized from journal - // it is a different object compared to the cache - addAlterJob(alterJob); - } - } - - public void replayFinish(AlterJob alterJob, Catalog catalog) throws MetaNotFoundException { - try { - Database db = catalog.getDbOrMetaException(alterJob.getDbId()); - alterJob.replayFinish(db); - } finally { - alterJob.setState(JobState.FINISHED); - jobDone(alterJob); - } - } - - public void replayCancel(AlterJob alterJob, Catalog catalog) throws MetaNotFoundException { - removeAlterJob(alterJob.getTableId()); - alterJob.setState(JobState.CANCELLED); - try { - // we log rollup job cancelled even if db is dropped. - // so check db != null here - Database db = catalog.getDbOrMetaException(alterJob.getDbId()); - alterJob.replayCancel(db); - } finally { - addFinishedOrCancelledAlterJob(alterJob); - } - } - @Override protected void runAfterCatalogReady() { - clearExpireFinishedOrCancelledAlterJobs(); clearExpireFinishedOrCancelledAlterJobsV2(); } @@ -387,17 +177,6 @@ public void processExternalTable(List alterClauses, Database db, Ta */ public abstract void cancel(CancelStmt stmt) throws DdlException; - @Deprecated - public Integer getAlterJobNumByState(JobState state) { - int jobNum = 0; - for (AlterJob alterJob : alterJobs.values()) { - if (alterJob.getState() == state) { - ++jobNum; - } - } - return jobNum; - } - /* * Handle the finish report of alter task. * If task is success, which means the history data before specified version has been transformed successfully. diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJob.java deleted file mode 100644 index 5b1d5e46d25671..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJob.java +++ /dev/null @@ -1,360 +0,0 @@ -// 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.alter; - -import org.apache.doris.catalog.Catalog; -import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.Replica; -import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.Config; -import org.apache.doris.common.FeMetaVersion; -import org.apache.doris.common.MetaNotFoundException; -import org.apache.doris.common.io.Text; -import org.apache.doris.common.io.Writable; -import org.apache.doris.system.Backend; -import org.apache.doris.task.AgentBatchTask; -import org.apache.doris.task.AgentTask; -import org.apache.doris.thrift.TResourceInfo; -import org.apache.doris.thrift.TTabletInfo; - -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import com.google.common.collect.Multimap; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.List; - -public abstract class AlterJob implements Writable { - private static final Logger LOG = LogManager.getLogger(AlterJob.class); - - public enum JobState { - PENDING, - RUNNING, - FINISHING, - FINISHED, - CANCELLED - } - - public enum JobType { - ROLLUP, - SCHEMA_CHANGE, - DECOMMISSION_BACKEND; - - @Override - public String toString() { - switch (this) { - case ROLLUP: - return "rollup"; - case SCHEMA_CHANGE: - return "schema change"; - case DECOMMISSION_BACKEND: - return "decommission backend"; - default: - Preconditions.checkState(false); - return "invalid"; - } - } - } - - protected final JobType type; - protected JobState state; - - protected long dbId; - protected long tableId; - protected long transactionId = -1; - // not serialize it - protected boolean isPreviousLoadFinished = false; - protected AgentBatchTask batchClearAlterTask = null; - - protected long createTime; - protected long finishedTime; - - protected String cancelMsg; - - protected TResourceInfo resourceInfo; - - // backendId -> replicaIds - // this map show which replica is still alive - // if backend is down, replica is not reachable in BE, remove replica from this map - protected Multimap backendIdToReplicaIds; - - public AlterJob(JobType type) { - // for persist - this.type = type; - - this.state = JobState.PENDING; - - this.createTime = System.currentTimeMillis(); - this.finishedTime = -1L; - // this.backendIdToReplicaIds = HashMultimap.create(); - } - - public AlterJob(JobType type, long dbId, long tableId, TResourceInfo resourceInfo) { - this.type = type; - this.state = JobState.PENDING; - - this.dbId = dbId; - this.tableId = tableId; - this.resourceInfo = resourceInfo; - - this.createTime = System.currentTimeMillis(); - this.finishedTime = -1L; - - this.cancelMsg = ""; - // this.backendIdToReplicaIds = HashMultimap.create(); - } - - public final JobType getType() { - return this.type; - } - - public void setState(JobState state) { - this.state = state; - } - - public synchronized JobState getState() { - return this.state; - } - - public final long getDbId() { - return dbId; - } - - public final long getTableId() { - return tableId; - } - - public final long getTransactionId() { - return transactionId; - } - - public final long getCreateTimeMs() { - return this.createTime; - } - - public final synchronized long getFinishedTime() { - return this.finishedTime; - } - - public synchronized void setMsg(String msg) { - this.cancelMsg = msg; - } - - public final synchronized String getMsg() { - return this.cancelMsg; - } - - public boolean isTimeout() { - // 0 means never timeout - if (Config.alter_table_timeout_second == 0 - || System.currentTimeMillis() - this.createTime < Config.alter_table_timeout_second * 1000L) { - return false; - } - setMsg("timeout"); - LOG.info("{} job[{}] timeout. cancel it.", type, tableId); - return true; - } - - /** - * this should be call in each round. - * otherwise, - * alter job will not perceived backend's down event during job created and first handle round. - */ - protected boolean checkBackendState(Replica replica) { - LOG.debug("check backend[{}] state for replica[{}]", replica.getBackendId(), replica.getId()); - Backend backend = Catalog.getCurrentSystemInfo().getBackend(replica.getBackendId()); - // not send event to event bus because there is a dead lock, job --> check state --> bus lock --> handle backend down - // backend down --> bus lock --> handle backend down --> job.lock - if (backend == null) { - return false; - } else if (!backend.isAlive()) { - long currentTime = System.currentTimeMillis(); - // If this backend is done for a long time and not restart automatically. - // we consider it as dead and return false. - return backend.getLastUpdateMs() <= 0 - || currentTime - backend.getLastUpdateMs() <= Config.max_backend_down_time_second * 1000; - } else { - return !backend.isDecommissioned(); - } - - } - - public static AlterJob read(DataInput in) throws IOException { - JobType type = JobType.valueOf(Text.readString(in)); - switch (type) { - case ROLLUP: - return RollupJob.read(in); - case SCHEMA_CHANGE: - return SchemaChangeJob.read(in); - case DECOMMISSION_BACKEND: - return DecommissionBackendJob.read(in); - default: - Preconditions.checkState(false); - return null; - } - } - - /* - * abstract methods - */ - - /** - * add replicas which need to be handled in this job - */ - public abstract void addReplicaId(long parentId, long replicaId, long backendId); - - /** - * set replicas as finished when replica task report success - */ - public abstract void setReplicaFinished(long parentId, long replicaId); - - /** - * send tasks to backends - */ - public abstract boolean sendTasks(); - - /** - * cancel job - */ - public abstract void cancel(OlapTable olapTable, String msg); - - /** - * remove replica related tasks in some failure situation - */ - public abstract void removeReplicaRelatedTask(long parentId, long tabletId, long replicaId, long backendId); - - /** - * handle replica finish task report - */ - public abstract void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo, long reportVersion) - throws MetaNotFoundException; - - /** - * return - * -1: need cancel - * 0: waiting next poll - * 1: finishing - */ - public abstract int tryFinishJob(); - - /** - * clear some date structure in this job to save memory - */ - public abstract void clear(); - - /** - * do something when state transferring from FINISHING to FINISHED. - * eg: - * set table's state to NORMAL - */ - public abstract void finishJob(); - - /** - * replay methods - * corresponding to start/finished/cancelled - */ - public abstract void replayInitJob(Database db); - - public abstract void replayFinishing(Database db); - - public abstract void replayFinish(Database db); - - public abstract void replayCancel(Database db); - - public abstract void getJobInfo(List> jobInfos, OlapTable tbl); - - // return true if all previous load job has been finished. - // return false if not - public boolean isPreviousLoadFinished() { - if (isPreviousLoadFinished) { - return true; - } else { - try { - isPreviousLoadFinished = Catalog.getCurrentGlobalTransactionMgr() - .isPreviousTransactionsFinished(transactionId, dbId, Lists.newArrayList(tableId)); - } catch (AnalysisException e) { - // this is a deprecated method, so just return true to make the compilation happy. - LOG.warn("failed to check previous load status for db: {}, tbl: {}, {}", - dbId, tableId, e.getMessage()); - return true; - } - return isPreviousLoadFinished; - } - } - - public synchronized void readFields(DataInput in) throws IOException { - // read common members as write in AlterJob.write(). - // except 'type' member, which is read in AlterJob.read() - if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_4) { - state = JobState.valueOf(Text.readString(in)); - } - - dbId = in.readLong(); - tableId = in.readLong(); - - createTime = in.readLong(); - finishedTime = in.readLong(); - - cancelMsg = Text.readString(in); - - // resource - boolean hasResourceInfo = in.readBoolean(); - if (hasResourceInfo) { - String user = Text.readString(in); - String group = Text.readString(in); - resourceInfo = new TResourceInfo(user, group); - } - if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_45) { - transactionId = in.readLong(); - } - } - - @Override - public synchronized void write(DataOutput out) throws IOException { - // write common members - Text.writeString(out, type.name()); - - Text.writeString(out, state.name()); - - out.writeLong(dbId); - out.writeLong(tableId); - - out.writeLong(createTime); - out.writeLong(finishedTime); - - Text.writeString(out, cancelMsg); - - // resourceInfo - if (resourceInfo == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - Text.writeString(out, resourceInfo.getUser()); - Text.writeString(out, resourceInfo.getGroup()); - } - - out.writeLong(transactionId); - - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java index c8c8815921fab9..157a36370d0291 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java @@ -61,7 +61,7 @@ public boolean isFinalState() { } public enum JobType { - ROLLUP, SCHEMA_CHANGE + ROLLUP, SCHEMA_CHANGE, DECOMMISSION_BACKEND } @SerializedName(value = "type") diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/DecommissionBackendJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/DecommissionBackendJob.java deleted file mode 100644 index aa72d838985690..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/DecommissionBackendJob.java +++ /dev/null @@ -1,369 +0,0 @@ -// 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.alter; - -import org.apache.doris.catalog.Catalog; -import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.TabletInvertedIndex; -import org.apache.doris.cluster.Cluster; -import org.apache.doris.common.DdlException; -import org.apache.doris.common.FeMetaVersion; -import org.apache.doris.common.MetaNotFoundException; -import org.apache.doris.common.io.Text; -import org.apache.doris.persist.BackendIdsUpdateInfo; -import org.apache.doris.system.Backend; -import org.apache.doris.system.Backend.BackendState; -import org.apache.doris.system.SystemInfoService; -import org.apache.doris.task.AgentTask; -import org.apache.doris.thrift.TTabletInfo; - -import com.google.common.base.Joiner; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; - -import org.apache.commons.lang.NotImplementedException; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; - -@Deprecated -public class DecommissionBackendJob extends AlterJob { - - public enum DecommissionType { - SystemDecommission, // after finished system decommission, the backend will be removed from Palo. - ClusterDecommission // after finished cluster decommission, the backend will be removed from cluster. - } - - private static final Logger LOG = LogManager.getLogger(DecommissionBackendJob.class); - - private static final Joiner JOINER = Joiner.on("; "); - - // all backends need to be decommissioned - private Map> clusterBackendsMap; - private Set allClusterBackendIds; - - // add backendId to 'finishedBackendIds' only if no tablets exist in that - // backend - private Set finishedBackendIds; - - private DecommissionType decommissionType; - - public DecommissionBackendJob() { - // for persist - super(JobType.DECOMMISSION_BACKEND); - - clusterBackendsMap = Maps.newHashMap(); - allClusterBackendIds = Sets.newHashSet(); - - finishedBackendIds = Sets.newHashSet(); - decommissionType = DecommissionType.SystemDecommission; - } - - public DecommissionBackendJob(long jobId, Map> backendIds) { - super(JobType.DECOMMISSION_BACKEND, -1L, jobId, null); - - clusterBackendsMap = backendIds; - allClusterBackendIds = Sets.newHashSet(); - for (Map backends : clusterBackendsMap.values()) { - allClusterBackendIds.addAll(backends.keySet()); - } - - finishedBackendIds = Sets.newHashSet(); - decommissionType = DecommissionType.SystemDecommission; - } - - /** - * in Multi-Tenancy example "clusterA:1,2,3;clusterB:4,5,6" - * - * @return - */ - public String getBackendIdsString() { - final Joiner joiner = Joiner.on(","); - final Set clusterBackendsSet = new HashSet(); - for (String cluster : clusterBackendsMap.keySet()) { - final Map backends = clusterBackendsMap.get(cluster); - final String backendStr = joiner.join(backends.keySet()); - final StringBuilder builder = new StringBuilder(cluster); - builder.append(":").append(backendStr); - clusterBackendsSet.add(builder.toString()); - } - String res = JOINER.join(clusterBackendsSet); - return res; - } - - public Set getBackendIds(String name) { - return clusterBackendsMap.get(name).keySet(); - } - - public DecommissionType getDecommissionType() { - return decommissionType; - } - - public void setDecommissionType(DecommissionType decommissionType) { - this.decommissionType = decommissionType; - } - - @Override - public void addReplicaId(long parentId, long replicaId, long backendId) { - throw new NotImplementedException(); - } - - @Override - public void setReplicaFinished(long parentId, long replicaId) { - throw new NotImplementedException(); - } - - @Override - public synchronized boolean sendTasks() { - // do nothing. - // In previous implementation, we send clone task actively. - // But now, TabletChecker will do all the things, here we just skip PENDING phase. - this.state = JobState.RUNNING; - return true; - } - - @Override - public synchronized void cancel(OlapTable olapTable, String msg) { - // set state - this.state = JobState.CANCELLED; - if (msg != null) { - this.cancelMsg = msg; - } - - this.finishedTime = System.currentTimeMillis(); - - // no need to log - LOG.info("finished cancel decommission backend"); - } - - @Override - public void removeReplicaRelatedTask(long parentId, long tabletId, long replicaId, long backendId) { - throw new NotImplementedException(); - } - - @Override - public synchronized void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo, long reportVersion) - throws MetaNotFoundException { - throw new NotImplementedException(); - } - - @Override - public synchronized int tryFinishJob() { - TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); - SystemInfoService systemInfo = Catalog.getCurrentSystemInfo(); - - LOG.debug("start try finish decommission backend job: {}", getBackendIdsString()); - for (String cluster : clusterBackendsMap.keySet()) { - final Map backends = clusterBackendsMap.get(cluster); - // check if tablets in one backend has full replicas - Iterator backendIter = backends.keySet().iterator(); - while (backendIter.hasNext()) { - long backendId = backendIter.next(); - Backend backend = systemInfo.getBackend(backendId); - if (backend == null || !backend.isDecommissioned()) { - backendIter.remove(); - LOG.info("backend[{}] is not decommissioned. remove from decommission jobs"); - continue; - } - - if (finishedBackendIds.contains(backendId)) { - continue; - } - - List backendTabletIds = invertedIndex.getTabletIdsByBackendId(backendId); - if (backendTabletIds.isEmpty()) { - LOG.info("no tablet in {}", backend); - finishedBackendIds.add(backendId); - continue; - } - - LOG.info("{} lefts {} replicas to migrate: {}", backend, backendTabletIds.size(), - backendTabletIds.size() <= 20 ? backendTabletIds : "too many"); - } // end for backends - } - - if (finishedBackendIds.size() >= allClusterBackendIds.size()) { - // use '>=' not '==', because backend may be removed from backendIds - // after it finished. - // drop backend - if (decommissionType == DecommissionType.SystemDecommission) { - for (long backendId : allClusterBackendIds) { - try { - systemInfo.dropBackend(backendId); - } catch (DdlException e) { - // it's ok, backend has already been dropped - LOG.info("drop backend[{}] failed. cause: {}", backendId, e.getMessage()); - } - } - } else { - // Shrinking capacity in cluster - if (decommissionType == DecommissionType.ClusterDecommission) { - for (String clusterName : clusterBackendsMap.keySet()) { - final Map idToBackend = clusterBackendsMap.get(clusterName); - final Cluster cluster = Catalog.getCurrentCatalog().getCluster(clusterName); - List backendList = Lists.newArrayList(); - for (long id : idToBackend.keySet()) { - final Backend backend = idToBackend.get(id); - backend.clearClusterName(); - backend.setBackendState(BackendState.free); - backend.setDecommissioned(false); - backendList.add(id); - cluster.removeBackend(id); - } - BackendIdsUpdateInfo updateInfo = new BackendIdsUpdateInfo(backendList); - Catalog.getCurrentCatalog().getEditLog().logUpdateClusterAndBackendState(updateInfo); - } - } - } - - this.finishedTime = System.currentTimeMillis(); - this.state = JobState.FINISHED; - - Catalog.getCurrentCatalog().getEditLog().logFinishDecommissionBackend(this); - - LOG.info("finished {} decommission {} backends: {}", decommissionType.toString(), - allClusterBackendIds.size(), getBackendIdsString()); - return 1; - } else { - Set unfinishedBackendIds = Sets.newHashSet(); - for (Long backendId : allClusterBackendIds) { - if (!finishedBackendIds.contains(backendId)) { - unfinishedBackendIds.add(backendId); - } - } - LOG.info("waiting {} backends to finish tablets migration: {}", unfinishedBackendIds.size(), - unfinishedBackendIds); - return 0; - } - } - - @Override - public synchronized void clear() { - finishedBackendIds.clear(); - } - - @Override - public void replayInitJob(Database db) { - // do nothing - } - - @Override - public void replayFinishing(Database db) { - // do nothing - } - - @Override - public void replayFinish(Database db) { - // do nothing - } - - @Override - public void replayCancel(Database db) { - // do nothing - } - - @Override - public void getJobInfo(List> jobInfos, OlapTable tbl) { - // do nothing - } - - /** - * to Backward compatibility - * - * @param in - * @throws IOException - */ - public void readFields(DataInput in) throws IOException { - super.readFields(in); - - if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_30) { - long clusterMapSize = in.readLong(); - while (clusterMapSize-- > 0) { - final String cluster = Text.readString(in); - long backendMspSize = in.readLong(); - Map backends = Maps.newHashMap(); - while (backendMspSize-- > 0) { - final long id = in.readLong(); - final Backend backend = Catalog.getCurrentSystemInfo().getBackend(id); - backends.put(id, backend); - allClusterBackendIds.add(id); - } - clusterBackendsMap.put(cluster, backends); - } - - if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_33) { - String str = Text.readString(in); - // this is only for rectify misspellings... - if (str.equals("SystemDecomission")) { - str = "SystemDecommission"; - } else if (str.equals("ClusterDecomission")) { - str = "ClusterDecommission"; - } - decommissionType = DecommissionType.valueOf(str); - } - } else { - int backendNum = in.readInt(); - Map backends = Maps.newHashMap(); - for (int i = 0; i < backendNum; i++) { - final long backendId = in.readLong(); - allClusterBackendIds.add(backendId); - final Backend backend = Catalog.getCurrentSystemInfo().getBackend(backendId); - backends.put(backendId, backend); - } - clusterBackendsMap.put(SystemInfoService.DEFAULT_CLUSTER, backends); - } - } - - @Override - public void write(DataOutput out) throws IOException { - super.write(out); - - out.writeLong(clusterBackendsMap.keySet().size()); - for (String cluster : clusterBackendsMap.keySet()) { - final Map backends = clusterBackendsMap.get(cluster); - Text.writeString(out, cluster); - out.writeLong(backends.keySet().size()); - for (Long id : backends.keySet()) { - out.writeLong(id); - } - } - - Text.writeString(out, decommissionType.toString()); - } - - public static DecommissionBackendJob read(DataInput in) throws IOException { - DecommissionBackendJob decommissionBackendJob = new DecommissionBackendJob(); - decommissionBackendJob.readFields(in); - return decommissionBackendJob; - } - - @Override - public void finishJob() { - // do nothing - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/DecommissionType.java b/fe/fe-core/src/main/java/org/apache/doris/alter/DecommissionType.java new file mode 100644 index 00000000000000..64437575c05c2e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/DecommissionType.java @@ -0,0 +1,6 @@ +package org.apache.doris.alter; + +public enum DecommissionType { + SystemDecommission, // after finished system decommission, the backend will be removed from Palo. + ClusterDecommission // after finished cluster decommission, the backend will be removed from cluster. +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java index af07dbba6d14d2..e59bb390c73cdf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java @@ -17,7 +17,6 @@ package org.apache.doris.alter; -import org.apache.doris.alter.AlterJob.JobState; import org.apache.doris.analysis.AddRollupClause; import org.apache.doris.analysis.AlterClause; import org.apache.doris.analysis.CancelAlterTableStmt; @@ -860,7 +859,6 @@ public void replayDropRollup(DropInfo dropInfo, Catalog catalog) throws MetaNotF @Override protected void runAfterCatalogReady() { super.runAfterCatalogReady(); - runOldAlterJob(); runAlterJobV2(); } @@ -982,131 +980,10 @@ private void onJobDone(AlterJobV2 alterJob) { } } - @Deprecated - private void runOldAlterJob() { - List cancelledJobs = Lists.newArrayList(); - List finishedJobs = Lists.newArrayList(); - - for (AlterJob alterJob : alterJobs.values()) { - RollupJob rollupJob = (RollupJob) alterJob; - if (rollupJob.getState() != JobState.FINISHING - && rollupJob.getState() != JobState.FINISHED - && rollupJob.getState() != JobState.CANCELLED) { - // cancel the old alter table job - cancelledJobs.add(rollupJob); - continue; - } - - if (rollupJob.getTransactionId() < 0) { - // it means this is an old type job and current version is real time load version - // then kill this job - cancelledJobs.add(rollupJob); - continue; - } - JobState state = rollupJob.getState(); - switch (state) { - case PENDING: { - // if rollup job's status is PENDING, we need to send tasks. - if (!rollupJob.sendTasks()) { - cancelledJobs.add(rollupJob); - LOG.warn("sending rollup job[" + rollupJob.getTableId() + "] tasks failed. cancel it."); - } - break; - } - case RUNNING: { - if (rollupJob.isTimeout()) { - cancelledJobs.add(rollupJob); - } else { - int res = rollupJob.tryFinishJob(); - if (res == -1) { - // cancel rollup - cancelledJobs.add(rollupJob); - LOG.warn("cancel rollup[{}] cause bad rollup job[{}]", - ((RollupJob) rollupJob).getRollupIndexName(), rollupJob.getTableId()); - } - } - break; - } - case FINISHING: { - // check previous load job finished - if (rollupJob.isPreviousLoadFinished()) { - // if all previous load job finished, then send clear alter tasks to all related be - LOG.info("previous txn finished, try to send clear txn task"); - int res = rollupJob.checkOrResendClearTasks(); - if (res != 0) { - LOG.info("send clear txn task return {}", res); - if (res == -1) { - LOG.warn("rollup job is in finishing state, but could not finished, " - + "just finish it, maybe a fatal error {}", rollupJob); - } - finishedJobs.add(rollupJob); - } - } else { - LOG.info("previous load jobs are not finished. can not finish rollup job: {}", - rollupJob.getTableId()); - } - break; - } - case FINISHED: { - break; - } - case CANCELLED: { - // the alter job could be cancelled in 3 ways - // 1. the table or db is dropped - // 2. user cancels the job - // 3. the job meets errors when running - // for the previous 2 scenarios, user will call jobdone to finish the job and set its state to cancelled - // so that there exists alter job whose state is cancelled - // for the third scenario, the thread will add to cancelled job list and will be dealt by call jobdone - // Preconditions.checkState(false); - break; - } - default: - Preconditions.checkState(false); - break; - } - } // end for jobs - - // handle cancelled rollup jobs - for (AlterJob rollupJob : cancelledJobs) { - Database db = Catalog.getCurrentCatalog().getDbNullable(rollupJob.getDbId()); - if (db == null) { - cancelInternal(rollupJob, null, null); - continue; - } - - OlapTable olapTable = (OlapTable) db.getTableNullable(rollupJob.getTableId()); - if (olapTable != null) { - olapTable.writeLock(); - } - try { - rollupJob.cancel(olapTable, "cancelled"); - } finally { - if (olapTable != null) { - olapTable.writeUnlock(); - } - } - jobDone(rollupJob); - } - - // handle finished rollup jobs - for (AlterJob alterJob : finishedJobs) { - alterJob.setState(JobState.FINISHED); - // remove from alterJobs. - // has to remove here, because the job maybe finished and it still in alter job list, - // then user could submit schema change task, and auto load to two table flag will be set false. - // then schema change job will be failed. - alterJob.finishJob(); - jobDone(alterJob); - Catalog.getCurrentCatalog().getEditLog().logFinishRollup((RollupJob) alterJob); - } - } - @Override public List> getAlterJobInfosByDb(Database db) { List> rollupJobInfos = new LinkedList>(); - getOldAlterJobInfos(db, rollupJobInfos); getAlterJobV2Infos(db, rollupJobInfos); // sort by @@ -1132,43 +1009,6 @@ private void getAlterJobV2Infos(Database db, List> rollupJobInf } } - @Deprecated - private void getOldAlterJobInfos(Database db, List> rollupJobInfos) { - List jobs = Lists.newArrayList(); - // lock to perform atomically - lock(); - try { - for (AlterJob alterJob : this.alterJobs.values()) { - if (alterJob.getDbId() == db.getId()) { - jobs.add(alterJob); - } - } - - for (AlterJob alterJob : this.finishedOrCancelledAlterJobs) { - if (alterJob.getDbId() == db.getId()) { - jobs.add(alterJob); - } - } - } finally { - unlock(); - } - - - for (AlterJob selectedJob : jobs) { - try { - OlapTable olapTable = db.getTableOrMetaException(selectedJob.getTableId(), Table.TableType.OLAP); - olapTable.readLock(); - try { - selectedJob.getJobInfo(rollupJobInfos, olapTable); - } finally { - olapTable.readUnlock(); - } - } catch (MetaNotFoundException ignored) { - } - - } - } - @Override public void process(List alterClauses, String clusterName, Database db, OlapTable olapTable) throws DdlException, AnalysisException, MetaNotFoundException { @@ -1195,7 +1035,6 @@ public void cancel(CancelStmt stmt) throws DdlException { Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName); - AlterJob rollupJob = null; List rollupJobV2List = new ArrayList<>(); OlapTable olapTable; try { @@ -1222,14 +1061,8 @@ public void cancel(CancelStmt stmt) throws DdlException { rollupJobV2List = getUnfinishedAlterJobV2ByTableId(olapTable.getId()); } if (rollupJobV2List.size() == 0) { - rollupJob = getAlterJob(olapTable.getId()); - Preconditions.checkNotNull(rollupJob, "Table[" + tableName + "] is not under ROLLUP. "); - if (rollupJob.getState() == JobState.FINISHED - || rollupJob.getState() == JobState.FINISHING - || rollupJob.getState() == JobState.CANCELLED) { - throw new DdlException("job is already " + rollupJob.getState().name() + ", can not cancel it"); - } - rollupJob.cancel(olapTable, "user cancelled"); + // Alter job v1 is not supported, delete related code + throw new DdlException("Table[" + tableName + "] is not under ROLLUP. Maybe it has old alter job"); } } finally { olapTable.writeUnlock(); @@ -1245,11 +1078,6 @@ public void cancel(CancelStmt stmt) throws DdlException { } return; } - - // handle old alter job - if (rollupJob != null && rollupJob.getState() == JobState.CANCELLED) { - jobDone(rollupJob); - } } // just for ut diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java deleted file mode 100644 index 1ab6cd214e2365..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java +++ /dev/null @@ -1,1180 +0,0 @@ -// 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.alter; - -import org.apache.doris.catalog.Catalog; -import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.KeysType; -import org.apache.doris.catalog.MaterializedIndex; -import org.apache.doris.catalog.MaterializedIndex.IndexState; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.OlapTable.OlapTableState; -import org.apache.doris.catalog.Partition; -import org.apache.doris.catalog.Partition.PartitionState; -import org.apache.doris.catalog.Replica; -import org.apache.doris.catalog.Replica.ReplicaState; -import org.apache.doris.catalog.Table; -import org.apache.doris.catalog.Tablet; -import org.apache.doris.catalog.TabletInvertedIndex; -import org.apache.doris.catalog.TabletMeta; -import org.apache.doris.common.Config; -import org.apache.doris.common.FeConstants; -import org.apache.doris.common.FeMetaVersion; -import org.apache.doris.common.MetaNotFoundException; -import org.apache.doris.common.io.Text; -import org.apache.doris.common.util.ListComparator; -import org.apache.doris.common.util.TimeUtils; -import org.apache.doris.persist.ReplicaPersistInfo; -import org.apache.doris.task.AgentBatchTask; -import org.apache.doris.task.AgentTask; -import org.apache.doris.task.AgentTaskQueue; -import org.apache.doris.task.ClearAlterTask; -import org.apache.doris.task.CreateRollupTask; -import org.apache.doris.thrift.TKeysType; -import org.apache.doris.thrift.TResourceInfo; -import org.apache.doris.thrift.TStorageMedium; -import org.apache.doris.thrift.TStorageType; -import org.apache.doris.thrift.TTabletInfo; -import org.apache.doris.thrift.TTaskType; - -import com.google.common.base.Preconditions; -import com.google.common.base.Strings; -import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.LinkedHashMultimap; -import com.google.common.collect.Lists; -import com.google.common.collect.Multimap; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; - -public class RollupJob extends AlterJob { - private static final Logger LOG = LogManager.getLogger(RollupJob.class); - - // for replica statistic - private Multimap partitionIdToUnfinishedReplicaIds; - private int totalReplicaNum; - - // partition id -> (rollup tablet id -> base tablet id) - private Map> partitionIdToBaseRollupTabletIdMap; - private Map partitionIdToRollupIndex; - // Record new replica info for catalog restore - // partition id -> list - private Multimap partitionIdToReplicaInfos; - - private Set finishedPartitionIds; - - // rollup and base schema info - private long baseIndexId; - private long rollupIndexId; - private String baseIndexName; - private String rollupIndexName; - - private List rollupSchema; - private int baseSchemaHash; - private int rollupSchemaHash; - - private TStorageType rollupStorageType; - private TKeysType rollupKeysType; - private short rollupShortKeyColumnCount; - - private RollupJob() { - super(JobType.ROLLUP); - - this.partitionIdToUnfinishedReplicaIds = HashMultimap.create(); - this.totalReplicaNum = 0; - - this.partitionIdToBaseRollupTabletIdMap = new HashMap>(); - this.partitionIdToRollupIndex = new HashMap(); - this.partitionIdToReplicaInfos = LinkedHashMultimap.create(); - - this.rollupSchema = new LinkedList(); - - this.finishedPartitionIds = new HashSet(); - } - - // yiguolei: every job has a transactionid to identify the current time, for example - // a load job's transactionid is 10 and a rollup job's transaction id is 12, then we could - // find load job is occurred before rollup job - public RollupJob(long dbId, long tableId, long baseIndexId, long rollupIndexId, - String baseIndexName, String rollupIndexName, List rollupSchema, - int baseSchemaHash, int rollupSchemaHash, TStorageType rollupStorageType, - short rollupShortKeyColumnCount, TResourceInfo resourceInfo, TKeysType rollupKeysType, - long transactionId) { - super(JobType.ROLLUP, dbId, tableId, resourceInfo); - - // rollup and base info - this.baseIndexId = baseIndexId; - this.rollupIndexId = rollupIndexId; - this.baseIndexName = baseIndexName; - this.rollupIndexName = rollupIndexName; - - this.rollupSchema = rollupSchema; - this.baseSchemaHash = baseSchemaHash; - this.rollupSchemaHash = rollupSchemaHash; - - this.rollupStorageType = rollupStorageType; - this.rollupKeysType = rollupKeysType; - this.rollupShortKeyColumnCount = rollupShortKeyColumnCount; - - // init other data struct - this.partitionIdToUnfinishedReplicaIds = ArrayListMultimap.create(); - this.totalReplicaNum = 0; - - this.partitionIdToBaseRollupTabletIdMap = new HashMap>(); - this.partitionIdToRollupIndex = new HashMap(); - this.partitionIdToReplicaInfos = LinkedHashMultimap.create(); - - this.finishedPartitionIds = new HashSet(); - - this.transactionId = transactionId; - } - - public final long getBaseIndexId() { - return baseIndexId; - } - - public final long getRollupIndexId() { - return rollupIndexId; - } - - public final String getBaseIndexName() { - return baseIndexName; - } - - public final String getRollupIndexName() { - return rollupIndexName; - } - - public final int getBaseSchemaHash() { - return baseSchemaHash; - } - - public final int getRollupSchemaHash() { - return rollupSchemaHash; - } - - public final TStorageType getRollupStorageType() { - return rollupStorageType; - } - - public final TKeysType getRollupKeysType() { - return rollupKeysType; - } - - public final short getRollupShortKeyColumnCount() { - return rollupShortKeyColumnCount; - } - - public final synchronized int getTotalReplicaNum() { - return this.totalReplicaNum; - } - - public synchronized int getUnfinishedReplicaNum() { - return partitionIdToUnfinishedReplicaIds.values().size(); - } - - public void setTabletIdMap(long partitionId, long rollupTabletId, long baseTabletId) { - Map tabletIdMap = partitionIdToBaseRollupTabletIdMap.get(partitionId); - if (tabletIdMap == null) { - tabletIdMap = new HashMap(); - partitionIdToBaseRollupTabletIdMap.put(partitionId, tabletIdMap); - } - tabletIdMap.put(rollupTabletId, baseTabletId); - } - - public void addRollupIndex(long partitionId, MaterializedIndex rollupIndex) { - this.partitionIdToRollupIndex.put(partitionId, rollupIndex); - } - - public synchronized void updateRollupReplicaInfo(long partitionId, long indexId, long tabletId, long backendId, - int schemaHash, long version, long versionHash, long rowCount, - long dataSize) throws MetaNotFoundException { - MaterializedIndex rollupIndex = this.partitionIdToRollupIndex.get(partitionId); - if (rollupIndex == null || rollupIndex.getId() != indexId) { - throw new MetaNotFoundException("Cannot find rollup index[" + indexId + "]"); - } - - Tablet tablet = rollupIndex.getTablet(tabletId); - if (tablet == null) { - throw new MetaNotFoundException("Cannot find tablet[" + tabletId + "]"); - } - - // update replica info - Replica replica = tablet.getReplicaByBackendId(backendId); - if (replica == null) { - throw new MetaNotFoundException("cannot find replica in tablet[" + tabletId + "], backend[" + backendId - + "]"); - } - replica.updateVersionInfo(version, versionHash, dataSize, rowCount); - LOG.debug("rollup replica[{}] info updated. schemaHash:{}", replica.getId(), schemaHash); - } - - public synchronized List> getInfos() { - List> rollupJobInfos = new LinkedList>(); - for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { - long partitionId = entry.getKey(); - MaterializedIndex rollupIndex = entry.getValue(); - - List partitionInfo = new ArrayList(); - - // partition id - partitionInfo.add(Long.valueOf(partitionId)); - // rollup index id - partitionInfo.add(Long.valueOf(rollupIndex.getId())); - // table state - partitionInfo.add(rollupIndex.getState().name()); - - rollupJobInfos.add(partitionInfo); - } - - // sort by - // "PartitionId", "IndexState" - ListComparator> comparator = new ListComparator>(0, 1); - Collections.sort(rollupJobInfos, comparator); - - return rollupJobInfos; - } - - public synchronized List> getRollupIndexInfo(long partitionId) { - List> tabletInfos = new ArrayList>(); - - MaterializedIndex index = this.partitionIdToRollupIndex.get(partitionId); - if (index == null) { - return tabletInfos; - } - - for (Tablet tablet : index.getTablets()) { - long tabletId = tablet.getId(); - for (Replica replica : tablet.getReplicas()) { - List tabletInfo = new ArrayList(); - // tabletId -- replicaId -- backendId -- version -- versionHash -- dataSize -- rowCount -- state - tabletInfo.add(tabletId); - tabletInfo.add(replica.getId()); - tabletInfo.add(replica.getBackendId()); - tabletInfo.add(replica.getVersion()); - tabletInfo.add(replica.getVersionHash()); - tabletInfo.add(replica.getDataSize()); - tabletInfo.add(replica.getRowCount()); - tabletInfo.add(replica.getState()); - tabletInfos.add(tabletInfo); - } - } - - return tabletInfos; - } - - public synchronized MaterializedIndex getRollupIndex(long partitionId) { - MaterializedIndex index = this.partitionIdToRollupIndex.get(partitionId); - return index; - } - - @Override - public synchronized void addReplicaId(long parentId, long replicaId, long backendId) { - this.partitionIdToUnfinishedReplicaIds.put(parentId, replicaId); - ++this.totalReplicaNum; - } - - @Override - public synchronized void setReplicaFinished(long parentId, long replicaId) { - // parent id is partitionId here - if (parentId == -1L) { - for (long onePartitionId : partitionIdToUnfinishedReplicaIds.keySet()) { - Collection replicaIds = partitionIdToUnfinishedReplicaIds.get(onePartitionId); - replicaIds.remove(replicaId); - } - } else { - this.partitionIdToUnfinishedReplicaIds.get(parentId).remove(replicaId); - } - } - - /* - * return - * 0: sending clear tasks - * 1: all clear tasks are finished, the job is done normally. - * -1: job meet some fatal error, like db or table is missing. - */ - public int checkOrResendClearTasks() { - Preconditions.checkState(this.state == JobState.FINISHING); - // 1. check if all task finished - boolean clearFailed = false; - if (batchClearAlterTask != null) { - List allTasks = batchClearAlterTask.getAllTasks(); - for (AgentTask oneClearAlterTask : allTasks) { - ClearAlterTask clearAlterTask = (ClearAlterTask) oneClearAlterTask; - if (!clearAlterTask.isFinished()) { - clearFailed = true; - } - AgentTaskQueue.removeTask(clearAlterTask.getBackendId(), - TTaskType.CLEAR_ALTER_TASK, clearAlterTask.getSignature()); - } - } - if (!clearFailed && batchClearAlterTask != null) { - return 1; - } - Database db = Catalog.getCurrentCatalog().getDbNullable(dbId); - if (db == null) { - cancelMsg = "db[" + dbId + "] does not exist"; - LOG.warn(cancelMsg); - return -1; - } - - batchClearAlterTask = new AgentBatchTask(); - - synchronized (this) { - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn(e.getMessage()); - return -1; - } - olapTable.readLock(); - try { - boolean allAddSuccess = true; - LOG.info("sending clear rollup job tasks for table [{}]", tableId); - for (Partition partition : olapTable.getPartitions()) { - long partitionId = partition.getId(); - // has to use rollup base index, could not use partition.getBaseIndex() - // because the rollup index could be created based on another rollup index - MaterializedIndex baseIndex = partition.getIndex(this.getBaseIndexId()); - for (Tablet baseTablet : baseIndex.getTablets()) { - long baseTabletId = baseTablet.getId(); - List baseReplicas = baseTablet.getReplicas(); - for (Replica baseReplica : baseReplicas) { - long backendId = baseReplica.getBackendId(); - ClearAlterTask clearRollupTask = new ClearAlterTask(backendId, dbId, tableId, - partitionId, baseIndexId, baseTabletId, baseSchemaHash); - if (AgentTaskQueue.addTask(clearRollupTask)) { - batchClearAlterTask.addTask(clearRollupTask); - } else { - allAddSuccess = false; - break; - } - } // end for rollupReplicas - if (!allAddSuccess) { - break; - } - } // end for rollupTablets - if (!allAddSuccess) { - break; - } - } - if (!allAddSuccess) { - for (AgentTask task : batchClearAlterTask.getAllTasks()) { - AgentTaskQueue.removeTask(task.getBackendId(), task.getTaskType(), task.getSignature()); - } - batchClearAlterTask = null; - } - - } finally { - olapTable.readUnlock(); - } - } - LOG.info("successfully sending clear rollup job[{}]", tableId); - return 0; - } - - @Override - public boolean sendTasks() { - Preconditions.checkState(this.state == JobState.PENDING); - // here we just rejoin tasks to AgentTaskQueue. - // task report process will later resend these tasks - - Database db = Catalog.getCurrentCatalog().getDbNullable(dbId); - if (db == null) { - cancelMsg = "db[" + dbId + "] does not exist"; - LOG.warn(cancelMsg); - return false; - } - - - synchronized (this) { - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn(e.getMessage()); - return false; - } - - LOG.info("sending rollup job[{}] tasks.", tableId); - // in palo 3.2, the rollup keys type is not serialized, when a fe follower change to fe master - // the rollup keys type == null, so that send tasks will report error - if (rollupKeysType == null) { - rollupKeysType = olapTable.getKeysType().toThrift(); - } - olapTable.readLock(); - try { - for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { - long partitionId = entry.getKey(); - Partition partition = olapTable.getPartition(partitionId); - if (partition == null) { - continue; - } - MaterializedIndex rollupIndex = entry.getValue(); - - Map tabletIdMap = this.partitionIdToBaseRollupTabletIdMap.get(partitionId); - for (Tablet rollupTablet : rollupIndex.getTablets()) { - long rollupTabletId = rollupTablet.getId(); - List rollupReplicas = rollupTablet.getReplicas(); - for (Replica rollupReplica : rollupReplicas) { - long backendId = rollupReplica.getBackendId(); - long rollupReplicaId = rollupReplica.getId(); - Preconditions.checkNotNull(tabletIdMap.get(rollupTabletId)); // baseTabletId - CreateRollupTask createRollupTask = - new CreateRollupTask(resourceInfo, backendId, dbId, tableId, - partitionId, rollupIndexId, baseIndexId, - rollupTabletId, tabletIdMap.get(rollupTabletId), - rollupReplicaId, - rollupShortKeyColumnCount, - rollupSchemaHash, baseSchemaHash, - rollupStorageType, rollupSchema, - olapTable.getCopiedBfColumns(), olapTable.getBfFpp(), - rollupKeysType); - AgentTaskQueue.addTask(createRollupTask); - - addReplicaId(partitionId, rollupReplicaId, backendId); - } // end for rollupReplicas - } // end for rollupTablets - } - - this.state = JobState.RUNNING; - } finally { - olapTable.readUnlock(); - } - - } - - Preconditions.checkState(this.state == JobState.RUNNING); - LOG.info("successfully sending rollup job[{}]", tableId); - return true; - } - - @Override - public synchronized void cancel(OlapTable olapTable, String msg) { - // remove tasks - for (MaterializedIndex rollupIndex : this.partitionIdToRollupIndex.values()) { - for (Tablet rollupTablet : rollupIndex.getTablets()) { - long rollupTabletId = rollupTablet.getId(); - List rollupReplicas = rollupTablet.getReplicas(); - for (Replica rollupReplica : rollupReplicas) { - long backendId = rollupReplica.getBackendId(); - AgentTaskQueue.removeTask(backendId, TTaskType.ROLLUP, rollupTabletId); - } - - Catalog.getCurrentInvertedIndex().deleteTablet(rollupTabletId); - } - LOG.debug("rollup job[{}]'s all tasks removed", tableId); - } - - // set state - if (olapTable != null) { - Preconditions.checkState(olapTable.getId() == tableId); - for (Partition partition : olapTable.getPartitions()) { - if (partition.getState() == PartitionState.ROLLUP) { - partition.setState(PartitionState.NORMAL); - } - } - - if (olapTable.getState() == OlapTableState.ROLLUP) { - olapTable.setState(OlapTableState.NORMAL); - } - } - - this.state = JobState.CANCELLED; - if (Strings.isNullOrEmpty(cancelMsg) && !Strings.isNullOrEmpty(msg)) { - this.cancelMsg = msg; - } - - this.finishedTime = System.currentTimeMillis(); - - // log - Catalog.getCurrentCatalog().getEditLog().logCancelRollup(this); - LOG.debug("cancel rollup job[{}] finished. because: {}", tableId, cancelMsg); - } - - /* - * this is called when base replica is deleted from meta. - * we have to find the corresponding rollup replica and mark it as finished - */ - @Override - public synchronized void removeReplicaRelatedTask(long parentId, long baseTabletId, - long baseReplicaId, long backendId) { - // parent id here is partition id - // baseReplicaId is unused here - - // 1. find rollup index - MaterializedIndex rollupIndex = this.partitionIdToRollupIndex.get(parentId); - - // 2. find rollup tablet - long rollupTabletId = -1L; - Map tabletIdMap = this.partitionIdToBaseRollupTabletIdMap.get(parentId); - for (Map.Entry entry : tabletIdMap.entrySet()) { - if (entry.getValue() == baseTabletId) { - rollupTabletId = entry.getKey(); - } - } - Preconditions.checkState(rollupTabletId != -1L); - Tablet rollupTablet = rollupIndex.getTablet(rollupTabletId); - Preconditions.checkNotNull(rollupIndex); - - // 3. find rollup replica - Replica rollupReplica = rollupTablet.getReplicaByBackendId(backendId); - if (rollupReplica == null) { - LOG.debug("can not find rollup replica in rollup tablet[{}]. backend[{}]", rollupTabletId, backendId); - return; - } - LOG.debug("remove replica {} from backend {}", rollupReplica, backendId, new Exception()); - setReplicaFinished(parentId, rollupReplica.getId()); - - // 4. remove task - AgentTaskQueue.removeTask(backendId, TTaskType.ROLLUP, rollupTabletId); - } - - @Override - public synchronized void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo, long reportVersion) - throws MetaNotFoundException { - // report version is unused here - - Preconditions.checkArgument(task instanceof CreateRollupTask); - CreateRollupTask createRollupTask = (CreateRollupTask) task; - - long partitionId = createRollupTask.getPartitionId(); - long rollupIndexId = createRollupTask.getIndexId(); - long rollupTabletId = createRollupTask.getTabletId(); - long rollupReplicaId = createRollupTask.getRollupReplicaId(); - - MaterializedIndex rollupIndex = this.partitionIdToRollupIndex.get(partitionId); - if (rollupIndex == null || rollupIndex.getId() != rollupIndexId) { - throw new MetaNotFoundException("Cannot find rollup index[" + rollupIndexId + "]"); - } - Preconditions.checkState(rollupIndex.getState() == IndexState.ROLLUP); - - Preconditions.checkArgument(finishTabletInfo.getTabletId() == rollupTabletId); - Tablet rollupTablet = rollupIndex.getTablet(rollupTabletId); - if (rollupTablet == null) { - throw new MetaNotFoundException("Cannot find rollup tablet[" + rollupTabletId + "]"); - } - - Replica rollupReplica = rollupTablet.getReplicaById(rollupReplicaId); - if (rollupReplica == null) { - throw new MetaNotFoundException("Cannot find rollup replica[" + rollupReplicaId + "]"); - } - if (rollupReplica.getState() == ReplicaState.NORMAL) { - // FIXME(cmy): still don't know why this can happen. add log to observe - LOG.warn("rollup replica[{}]' state is already set to NORMAL. tablet[{}]. backend[{}]", - rollupReplicaId, rollupTabletId, task.getBackendId()); - } - - long version = finishTabletInfo.getVersion(); - long versionHash = finishTabletInfo.getVersionHash(); - long dataSize = finishTabletInfo.getDataSize(); - long rowCount = finishTabletInfo.getRowCount(); - // yiguolei: not check version here because the replica's first version will be set by rollup job - // the version is not set now - rollupReplica.updateVersionInfo(version, versionHash, dataSize, rowCount); - - if (finishTabletInfo.isSetPathHash()) { - rollupReplica.setPathHash(finishTabletInfo.getPathHash()); - } - - setReplicaFinished(partitionId, rollupReplicaId); - rollupReplica.setState(ReplicaState.NORMAL); - - LOG.info("finished rollup replica[{}]. index[{}]. tablet[{}]. backend[{}], version: {}-{}", - rollupReplicaId, rollupIndexId, rollupTabletId, task.getBackendId(), version, versionHash); - } - - /* - * we make the rollup visible, but keep table's state as ROLLUP. - * 1. Make the rollup visible, because we want that the following load jobs will load data to the new - * rollup, too. - * 2. keep the table's state in ROLLUP, because we don't want another alter job being processed. - */ - @Override - public int tryFinishJob() { - if (this.state != JobState.RUNNING) { - LOG.info("rollup job[{}] is not running or finishing.", tableId); - return 0; - } - - Database db = Catalog.getCurrentCatalog().getDbNullable(dbId); - if (db == null) { - cancelMsg = "Db[" + dbId + "] does not exist"; - LOG.warn(cancelMsg); - return -1; - } - - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn(e.getMessage()); - return -1; - } - - if (!olapTable.writeLockIfExist()) { - LOG.warn("unknown table, tableName=" + olapTable.getName()); - return -1; - } - try { - // if all previous transaction has finished, then check base and rollup replica num - synchronized (this) { - for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { - long partitionId = entry.getKey(); - Partition partition = olapTable.getPartition(partitionId); - if (partition == null) { - LOG.warn("partition[{}] does not exist", partitionId); - continue; - } - - short expectReplicationNum = olapTable.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum(); - MaterializedIndex rollupIndex = entry.getValue(); - for (Tablet rollupTablet : rollupIndex.getTablets()) { - // yiguolei: the rollup tablet only contains the replica that is healthy at rollup time - List replicas = rollupTablet.getReplicas(); - List errorReplicas = Lists.newArrayList(); - for (Replica replica : replicas) { - if (!checkBackendState(replica)) { - LOG.warn("backend {} state is abnormal, set replica {} as bad", replica.getBackendId(), - replica.getId()); - errorReplicas.add(replica); - } else if (replica.getLastFailedVersion() > 0 - && !partitionIdToUnfinishedReplicaIds.get(partitionId).contains(replica.getId())) { - // if the replica has finished converting history data, - // but failed during load, then it is a abnormal. - // remove it from replica set - // have to use delete replica, it will remove it from tablet inverted index - LOG.warn("replica [{}] last failed version > 0 and have finished history rollup job," - + " its a bad replica, remove it from rollup tablet", replica); - errorReplicas.add(replica); - } - } - - for (Replica errorReplica : errorReplicas) { - rollupTablet.deleteReplica(errorReplica); - setReplicaFinished(partitionId, errorReplica.getId()); - AgentTaskQueue.removeTask(errorReplica.getBackendId(), TTaskType.ROLLUP, rollupTablet.getId()); - } - - if (rollupTablet.getReplicas().size() < (expectReplicationNum / 2 + 1)) { - cancelMsg = String.format( - "rollup job[%d] cancelled. rollup tablet[%d] has few health replica." - + " num: %d", tableId, rollupTablet.getId(), replicas.size()); - LOG.warn(cancelMsg); - return -1; - } - } // end for tablets - - // check if partition is finished - if (!this.partitionIdToUnfinishedReplicaIds.get(partitionId).isEmpty()) { - LOG.debug("partition[{}] has unfinished rollup replica: {}", - partitionId, this.partitionIdToUnfinishedReplicaIds.get(partitionId).size()); - return 0; - } - - // partition is finished - if (!this.finishedPartitionIds.contains(partitionId)) { - Preconditions.checkState(rollupIndex.getState() == IndexState.ROLLUP); - rollupIndex.setState(IndexState.NORMAL); - this.finishedPartitionIds.add(partitionId); - - // remove task for safety - // task may be left if some backends are down during schema change - for (Tablet tablet : rollupIndex.getTablets()) { - for (Replica replica : tablet.getReplicas()) { - AgentTaskQueue.removeTask(replica.getBackendId(), TTaskType.ROLLUP, - tablet.getId()); - } - } - - LOG.info("create rollup index[{}] finished in partition[{}]", - rollupIndex.getId(), partitionId); - } - } // end for partitions - - // all partition is finished rollup - // add rollup index to each partition - for (Partition partition : olapTable.getPartitions()) { - long partitionId = partition.getId(); - MaterializedIndex rollupIndex = this.partitionIdToRollupIndex.get(partitionId); - Preconditions.checkNotNull(rollupIndex); - - // 1. record replica info - for (Tablet tablet : rollupIndex.getTablets()) { - long tabletId = tablet.getId(); - for (Replica replica : tablet.getReplicas()) { - ReplicaPersistInfo replicaInfo = - ReplicaPersistInfo.createForRollup(rollupIndexId, tabletId, replica.getBackendId(), - replica.getVersion(), replica.getVersionHash(), - rollupSchemaHash, - replica.getDataSize(), replica.getRowCount(), - replica.getLastFailedVersion(), - replica.getLastFailedVersionHash(), - replica.getLastSuccessVersion(), - replica.getLastSuccessVersionHash()); - this.partitionIdToReplicaInfos.put(partitionId, replicaInfo); - } - } // end for tablets - - // 2. add to partition - partition.createRollupIndex(rollupIndex); - - // 3. add rollup finished version to base index - MaterializedIndex baseIndex = partition.getIndex(baseIndexId); - if (baseIndex != null) { - baseIndex.setRollupIndexInfo(rollupIndexId, partition.getVisibleVersion()); - } - Preconditions.checkState(partition.getState() == PartitionState.ROLLUP); - partition.setState(PartitionState.NORMAL); - } // end for partitions - - // set index's info - olapTable.setIndexMeta(rollupIndexId, rollupIndexName, rollupSchema, 0, rollupSchemaHash, - rollupShortKeyColumnCount, rollupStorageType, KeysType.fromThrift(rollupKeysType)); - Preconditions.checkState(olapTable.getState() == OlapTableState.ROLLUP); - - this.state = JobState.FINISHING; - this.transactionId = Catalog.getCurrentGlobalTransactionMgr().getTransactionIDGenerator().getNextTransactionId(); - } - } finally { - olapTable.writeUnlock(); - } - - Catalog.getCurrentCatalog().getEditLog().logFinishingRollup(this); - LOG.info("rollup job[{}] is finishing.", this.getTableId()); - - return 1; - } - - @Override - public synchronized void clear() { - this.resourceInfo = null; - this.partitionIdToUnfinishedReplicaIds = null; - this.partitionIdToBaseRollupTabletIdMap = null; - // do not set to null. show proc use it - this.partitionIdToRollupIndex.clear(); - this.finishedPartitionIds = null; - this.partitionIdToReplicaInfos = null; - this.rollupSchema = null; - } - - @Override - public void replayInitJob(Database db) { - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn("[INCONSISTENT META] replay init rollup job failed", e); - return; - } - olapTable.writeLock(); - try { - // set state - TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); - - for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { - Partition partition = olapTable.getPartition(entry.getKey()); - partition.setState(PartitionState.ROLLUP); - TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty( - partition.getId()).getStorageMedium(); - - if (!Catalog.isCheckpointThread()) { - MaterializedIndex rollupIndex = entry.getValue(); - TabletMeta tabletMeta = new TabletMeta(dbId, tableId, entry.getKey(), rollupIndexId, - rollupSchemaHash, medium); - for (Tablet tablet : rollupIndex.getTablets()) { - long tabletId = tablet.getId(); - invertedIndex.addTablet(tabletId, tabletMeta); - for (Replica replica : tablet.getReplicas()) { - invertedIndex.addReplica(tabletId, replica); - } - } - } - } // end for partitions - olapTable.setState(OlapTableState.ROLLUP); - - // reset status to PENDING for resending the tasks in polling thread - this.state = JobState.PENDING; - } finally { - olapTable.writeUnlock(); - } - } - - @Override - public void replayFinishing(Database db) { - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn("[INCONSISTENT META] replay finishing rollup job failed", e); - return; - } - TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); - olapTable.writeLock(); - try { - for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { - long partitionId = entry.getKey(); - MaterializedIndex rollupIndex = entry.getValue(); - Partition partition = olapTable.getPartition(partitionId); - - if (!Catalog.isCheckpointThread()) { - // Here we have to use replicas in inverted index to rebuild the rollupIndex's tablet. - // Because the rollupIndex here is read from edit log, so the replicas in it are - // not the same objects as in inverted index. - // And checkpoint thread is no need to handle inverted index - for (Tablet tablet : rollupIndex.getTablets()) { - List copiedReplicas = Lists.newArrayList(tablet.getReplicas()); - tablet.clearReplica(); - for (Replica copiedReplica : copiedReplicas) { - Replica replica = invertedIndex.getReplica(tablet.getId(), copiedReplica.getBackendId()); - tablet.addReplica(replica, true); - } - } - } - - long rollupRowCount = 0L; - for (Tablet tablet : rollupIndex.getTablets()) { - for (Replica replica : tablet.getReplicas()) { - replica.setState(ReplicaState.NORMAL); - } - - // calculate rollup index row count - long tabletRowCount = 0L; - for (Replica replica : tablet.getReplicas()) { - long replicaRowCount = replica.getRowCount(); - if (replicaRowCount > tabletRowCount) { - tabletRowCount = replicaRowCount; - } - } - rollupRowCount += tabletRowCount; - } - - rollupIndex.setRowCount(rollupRowCount); - rollupIndex.setState(IndexState.NORMAL); - - MaterializedIndex baseIndex = partition.getIndex(baseIndexId); - if (baseIndex != null) { - baseIndex.setRollupIndexInfo(rollupIndexId, partition.getVisibleVersion()); - } - - partition.createRollupIndex(rollupIndex); - partition.setState(PartitionState.NORMAL); - - // Update database information - Collection replicaInfos = partitionIdToReplicaInfos.get(partitionId); - if (replicaInfos != null) { - for (ReplicaPersistInfo info : replicaInfos) { - MaterializedIndex mIndex = partition.getIndex(info.getIndexId()); - Tablet tablet = mIndex.getTablet(info.getTabletId()); - Replica replica = tablet.getReplicaByBackendId(info.getBackendId()); - replica.updateVersionInfo(info.getVersion(), info.getVersionHash(), - info.getLastFailedVersion(), - info.getLastFailedVersionHash(), - info.getLastSuccessVersion(), - info.getLastSuccessVersionHash()); - } - } - } - - olapTable.setIndexMeta(rollupIndexId, rollupIndexName, rollupSchema, 0, rollupSchemaHash, - rollupShortKeyColumnCount, rollupStorageType, KeysType.fromThrift(rollupKeysType)); - } finally { - olapTable.writeUnlock(); - } - - LOG.info("replay finishing the rollup job: {}", tableId); - } - - @Override - public void replayFinish(Database db) { - // if this is an old job, then should also update table or replica's state - if (transactionId < 0) { - replayFinishing(db); - } - - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn("[INCONSISTENT META] replay finish rollup job failed", e); - return; - } - olapTable.writeLock(); - try { - olapTable.setState(OlapTableState.NORMAL); - } finally { - olapTable.writeUnlock(); - } - } - - @Override - public void replayCancel(Database db) { - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn("[INCONSISTENT META] replay cancel rollup job failed", e); - return; - } - olapTable.writeLock(); - try{ - if (!Catalog.isCheckpointThread()) { - // remove from inverted index - for (MaterializedIndex rollupIndex : partitionIdToRollupIndex.values()) { - for (Tablet tablet : rollupIndex.getTablets()) { - Catalog.getCurrentInvertedIndex().deleteTablet(tablet.getId()); - } - } - } - - // set state - for (Partition partition : olapTable.getPartitions()) { - partition.setState(PartitionState.NORMAL); - } - olapTable.setState(OlapTableState.NORMAL); - } finally { - olapTable.writeUnlock(); - } - } - - @Override - public void finishJob() { - Database db = Catalog.getCurrentCatalog().getDbNullable(dbId); - if (db == null) { - cancelMsg = String.format("database %d does not exist", dbId); - LOG.warn(cancelMsg); - return; - } - - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn(e.getMessage()); - return; - } - - olapTable.writeLock(); - try { - olapTable.setState(OlapTableState.NORMAL); - } finally { - olapTable.writeUnlock(); - } - - this.finishedTime = System.currentTimeMillis(); - LOG.info("finished rollup job: {}", tableId); - } - - @Override - public void getJobInfo(List> jobInfos, OlapTable tbl) { - List jobInfo = new ArrayList(); - - // job id - jobInfo.add(tableId); - - // table name - jobInfo.add(tbl.getName()); - - // create time - jobInfo.add(TimeUtils.longToTimeString(createTime)); - - jobInfo.add(TimeUtils.longToTimeString(finishedTime)); - - // base index and rollup index name - jobInfo.add(baseIndexName); - jobInfo.add(rollupIndexName); - - // rollup id - jobInfo.add(rollupIndexId); - - // transaction id - jobInfo.add(transactionId); - - // job state - jobInfo.add(state.name()); - - // msg - jobInfo.add(cancelMsg); - - // progress - if (state == JobState.RUNNING) { - int unfinishedReplicaNum = getUnfinishedReplicaNum(); - int totalReplicaNum = getTotalReplicaNum(); - Preconditions.checkState(unfinishedReplicaNum <= totalReplicaNum); - jobInfo.add(((totalReplicaNum - unfinishedReplicaNum) * 100 / totalReplicaNum) + "%"); - } else { - jobInfo.add(FeConstants.null_string); - } - jobInfo.add(Config.alter_table_timeout_second); - - jobInfos.add(jobInfo); - } - - @Override - public synchronized void write(DataOutput out) throws IOException { - super.write(out); - - // 'partitionIdToUnfinishedReplicaIds' and 'totalReplicaNum' don't need persist - // build them when resendTasks - - out.writeInt(partitionIdToRollupIndex.size()); - for (long partitionId : partitionIdToRollupIndex.keySet()) { - out.writeLong(partitionId); - - out.writeInt(partitionIdToBaseRollupTabletIdMap.get(partitionId).size()); - for (Map.Entry entry : partitionIdToBaseRollupTabletIdMap.get(partitionId).entrySet()) { - out.writeLong(entry.getKey()); - out.writeLong(entry.getValue()); - } - - MaterializedIndex rollupIndex = partitionIdToRollupIndex.get(partitionId); - rollupIndex.write(out); - - out.writeInt(partitionIdToReplicaInfos.get(partitionId).size()); - for (ReplicaPersistInfo info : partitionIdToReplicaInfos.get(partitionId)) { - info.write(out); - } - } - - // backendIdToReplicaIds don't need persisit - - out.writeLong(baseIndexId); - out.writeLong(rollupIndexId); - Text.writeString(out, baseIndexName); - Text.writeString(out, rollupIndexName); - - // schema - if (rollupSchema != null) { - out.writeBoolean(true); - int count = rollupSchema.size(); - out.writeInt(count); - for (Column column : rollupSchema) { - column.write(out); - } - } else { - out.writeBoolean(false); - } - - out.writeInt(baseSchemaHash); - out.writeInt(rollupSchemaHash); - - out.writeShort(rollupShortKeyColumnCount); - Text.writeString(out, rollupStorageType.name()); - // when upgrade from 3.2, rollupKeysType == null - if (rollupKeysType != null) { - out.writeBoolean(true); - Text.writeString(out, rollupKeysType.name()); - } else { - out.writeBoolean(false); - } - } - - public void readFields(DataInput in) throws IOException { - super.readFields(in); - - int partitionNum = in.readInt(); - for (int i = 0; i < partitionNum; i++) { - long partitionId = in.readLong(); - - // tablet map - int count = in.readInt(); - Map tabletMap = new HashMap(); - for (int j = 0; j < count; j++) { - long rollupTabletId = in.readLong(); - long baseTabletId = in.readLong(); - tabletMap.put(rollupTabletId, baseTabletId); - } - partitionIdToBaseRollupTabletIdMap.put(partitionId, tabletMap); - - // rollup index - MaterializedIndex rollupIndex = MaterializedIndex.read(in); - partitionIdToRollupIndex.put(partitionId, rollupIndex); - - // replica infos - count = in.readInt(); - for (int j = 0; j < count; j++) { - ReplicaPersistInfo replicaInfo = ReplicaPersistInfo.read(in); - partitionIdToReplicaInfos.put(partitionId, replicaInfo); - } - } - - baseIndexId = in.readLong(); - rollupIndexId = in.readLong(); - baseIndexName = Text.readString(in); - rollupIndexName = Text.readString(in); - - // schema - boolean hasSchema = in.readBoolean(); - if (hasSchema) { - int count = in.readInt(); - for (int i = 0; i < count; i++) { - Column column = Column.read(in); - rollupSchema.add(column); - } - } - - baseSchemaHash = in.readInt(); - rollupSchemaHash = in.readInt(); - - rollupShortKeyColumnCount = in.readShort(); - rollupStorageType = TStorageType.valueOf(Text.readString(in)); - if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_45) { - boolean hasRollKeysType = in.readBoolean(); - if (hasRollKeysType) { - rollupKeysType = TKeysType.valueOf(Text.readString(in)); - } - } - } - - public static RollupJob read(DataInput in) throws IOException { - RollupJob rollupJob = new RollupJob(); - rollupJob.readFields(in); - return rollupJob; - } - - @Override - public boolean equals(Object obj) { - return true; - } - - @Override - public String toString() { - return "RollupJob [baseIndexId=" + baseIndexId + ", rollupIndexId=" + rollupIndexId + ", baseIndexName=" - + baseIndexName + ", rollupIndexName=" + rollupIndexName + ", rollupSchema=" + rollupSchema - + ", baseSchemaHash=" + baseSchemaHash + ", rollupSchemaHash=" + rollupSchemaHash + ", type=" + type - + ", state=" + state + ", dbId=" + dbId + ", tableId=" + tableId + ", transactionId=" + transactionId - + ", isPreviousLoadFinished=" + isPreviousLoadFinished + ", createTime=" + createTime - + ", finishedTime=" + finishedTime + "]"; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index db90d307a1e1e8..ad748864a4e139 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -17,7 +17,6 @@ package org.apache.doris.alter; -import org.apache.doris.alter.AlterJob.JobState; import org.apache.doris.analysis.AddColumnClause; import org.apache.doris.analysis.AddColumnsClause; import org.apache.doris.analysis.AlterClause; @@ -991,10 +990,6 @@ private void createJob(long dbId, OlapTable olapTable, Map cancelledJobs = Lists.newArrayList(); - List finishedJobs = Lists.newArrayList(); - - for (AlterJob alterJob : alterJobs.values()) { - SchemaChangeJob schemaChangeJob = (SchemaChangeJob) alterJob; - if (schemaChangeJob.getState() != JobState.FINISHING - && schemaChangeJob.getState() != JobState.FINISHED - && schemaChangeJob.getState() != JobState.CANCELLED) { - // cancel the old alter table job - cancelledJobs.add(schemaChangeJob); - continue; - } - // it means this is an old type job and current version is real time load version - // then kill this job - if (alterJob.getTransactionId() < 0) { - cancelledJobs.add(alterJob); - continue; - } - JobState state = alterJob.getState(); - switch (state) { - case PENDING: { - if (!alterJob.sendTasks()) { - cancelledJobs.add(alterJob); - LOG.warn("sending schema change job {} tasks failed. cancel it.", alterJob.getTableId()); - } - break; - } - case RUNNING: { - if (alterJob.isTimeout()) { - cancelledJobs.add(alterJob); - } else { - int res = alterJob.tryFinishJob(); - if (res == -1) { - cancelledJobs.add(alterJob); - LOG.warn("cancel bad schema change job[{}]", alterJob.getTableId()); - } - } - break; - } - case FINISHING: { - // check if previous load job finished - if (alterJob.isPreviousLoadFinished()) { - LOG.info("schema change job has finished, send clear tasks to all be {}", alterJob); - // if all previous load job finished, then send clear alter tasks to all related be - int res = schemaChangeJob.checkOrResendClearTasks(); - if (res != 0) { - if (res == -1) { - LOG.warn("schema change job is in finishing state,but could not finished, " - + "just finish it, maybe a fatal error {}", alterJob); - } else { - LOG.info("send clear tasks to all be for job [{}] successfully, " - + "set status to finished", alterJob); - } - - finishedJobs.add(alterJob); - } - } else { - LOG.info("previous load jobs are not finished. can not finish schema change job: {}", - alterJob.getTableId()); - } - break; - } - case FINISHED: { - break; - } - case CANCELLED: { - // the alter job could be cancelled in 3 ways - // 1. the table or db is dropped - // 2. user cancels the job - // 3. the job meets errors when running - // for the previous 2 scenarios, user will call jobdone to finish the job and set its state to cancelled - // so that there exists alter job whose state is cancelled - // for the third scenario, the thread will add to cancelled job list and will be dealt by call jobdone - // Preconditions.checkState(false); - break; - } - default: - Preconditions.checkState(false); - break; - } - } // end for jobs - - // handle cancelled schema change jobs - for (AlterJob alterJob : cancelledJobs) { - Database db = Catalog.getCurrentCatalog().getDbNullable(alterJob.getDbId()); - if (db == null) { - cancelInternal(alterJob, null, null); - continue; - } - - OlapTable olapTable = (OlapTable) db.getTableNullable(alterJob.getTableId()); - if (olapTable != null) { - olapTable.writeLock(); - } - try { - alterJob.cancel(olapTable, "cancelled"); - } finally { - if (olapTable != null) { - olapTable.writeUnlock(); - } - } - jobDone(alterJob); - } - - // handle finished schema change jobs - for (AlterJob alterJob : finishedJobs) { - alterJob.setState(JobState.FINISHED); - // has to remove here, because check is running every interval, it maybe finished but also in job list - // some check will failed - ((SchemaChangeJob) alterJob).deleteAllTableHistorySchema(); - alterJob.finishJob(); - jobDone(alterJob); - Catalog.getCurrentCatalog().getEditLog().logFinishSchemaChange((SchemaChangeJob) alterJob); - } - } - @Override public List> getAlterJobInfosByDb(Database db) { List> schemaChangeJobInfos = new LinkedList<>(); - getOldAlterJobInfos(db, schemaChangeJobInfos); getAlterJobV2Infos(db, schemaChangeJobInfos); // sort by "JobId", "PartitionName", "CreateTime", "FinishTime", "IndexName", "IndexState" @@ -1578,46 +1453,6 @@ private void getAlterJobV2Infos(Database db, List> schemaChange getAlterJobV2Infos(db, ImmutableList.copyOf(alterJobsV2.values()), schemaChangeJobInfos); } - @Deprecated - private void getOldAlterJobInfos(Database db, List> schemaChangeJobInfos) { - List selectedJobs = Lists.newArrayList(); - - lock(); - try { - // init or running - for (AlterJob alterJob : this.alterJobs.values()) { - if (alterJob.getDbId() == db.getId()) { - selectedJobs.add(alterJob); - } - } - - // finished or cancelled - for (AlterJob alterJob : this.finishedOrCancelledAlterJobs) { - if (alterJob.getDbId() == db.getId()) { - selectedJobs.add(alterJob); - } - } - - } finally { - unlock(); - } - - for (AlterJob selectedJob : selectedJobs) { - OlapTable olapTable = (OlapTable) db.getTableNullable(selectedJob.getTableId()); - if (olapTable == null) { - continue; - } - olapTable.readLock(); - try { - selectedJob.getJobInfo(schemaChangeJobInfos, olapTable); - } finally { - olapTable.readUnlock(); - } - - } - } - - @Override public void process(List alterClauses, String clusterName, Database db, OlapTable olapTable) throws UserException { @@ -1927,8 +1762,6 @@ public void cancel(CancelStmt stmt) throws DdlException { Preconditions.checkState(!Strings.isNullOrEmpty(tableName)); Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName); - - AlterJob schemaChangeJob = null; AlterJobV2 schemaChangeJobV2 = null; OlapTable olapTable = db.getOlapTableOrDdlException(tableName); @@ -1943,16 +1776,7 @@ public void cancel(CancelStmt stmt) throws DdlException { List schemaChangeJobV2List = getUnfinishedAlterJobV2ByTableId(olapTable.getId()); // current schemaChangeJob job doesn't support batch operation,so just need to get one job schemaChangeJobV2 = schemaChangeJobV2List.size() == 0 ? null : Iterables.getOnlyElement(schemaChangeJobV2List); - if (schemaChangeJobV2 == null) { - schemaChangeJob = getAlterJob(olapTable.getId()); - Preconditions.checkNotNull(schemaChangeJob, "Table[" + tableName + "] is not under SCHEMA_CHANGE."); - if (schemaChangeJob.getState() == JobState.FINISHING - || schemaChangeJob.getState() == JobState.FINISHED - || schemaChangeJob.getState() == JobState.CANCELLED) { - throw new DdlException("job is already " + schemaChangeJob.getState().name() + ", can not cancel it"); - } - schemaChangeJob.cancel(olapTable, "user cancelled"); - } + Preconditions.checkNotNull(schemaChangeJobV2, "Table[" + tableName + "] is not under SCHEMA_CHANGE."); } finally { olapTable.writeUnlock(); } @@ -1964,11 +1788,6 @@ public void cancel(CancelStmt stmt) throws DdlException { } return; } - - // handle old alter job - if (schemaChangeJob != null && schemaChangeJob.getState() == JobState.CANCELLED) { - jobDone(schemaChangeJob); - } } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java deleted file mode 100644 index f2df913864d34f..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java +++ /dev/null @@ -1,1385 +0,0 @@ -// 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.alter; - -import org.apache.doris.catalog.Catalog; -import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.KeysType; -import org.apache.doris.catalog.MaterializedIndex; -import org.apache.doris.catalog.MaterializedIndex.IndexExtState; -import org.apache.doris.catalog.MaterializedIndex.IndexState; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.OlapTable.OlapTableState; -import org.apache.doris.catalog.Partition; -import org.apache.doris.catalog.Partition.PartitionState; -import org.apache.doris.catalog.Replica; -import org.apache.doris.catalog.Replica.ReplicaState; -import org.apache.doris.catalog.Table; -import org.apache.doris.catalog.Tablet; -import org.apache.doris.common.Config; -import org.apache.doris.common.FeConstants; -import org.apache.doris.common.FeMetaVersion; -import org.apache.doris.common.MetaNotFoundException; -import org.apache.doris.common.io.Text; -import org.apache.doris.common.util.TimeUtils; -import org.apache.doris.persist.ReplicaPersistInfo; -import org.apache.doris.persist.ReplicaPersistInfo.ReplicaOperationType; -import org.apache.doris.task.AgentBatchTask; -import org.apache.doris.task.AgentTask; -import org.apache.doris.task.AgentTaskExecutor; -import org.apache.doris.task.AgentTaskQueue; -import org.apache.doris.task.ClearAlterTask; -import org.apache.doris.task.SchemaChangeTask; -import org.apache.doris.thrift.TKeysType; -import org.apache.doris.thrift.TResourceInfo; -import org.apache.doris.thrift.TStorageType; -import org.apache.doris.thrift.TTabletInfo; -import org.apache.doris.thrift.TTaskType; - -import com.google.common.base.Preconditions; -import com.google.common.base.Strings; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.HashMultiset; -import com.google.common.collect.LinkedHashMultimap; -import com.google.common.collect.Lists; -import com.google.common.collect.Multimap; -import com.google.common.collect.Multiset; -import com.google.common.collect.Sets; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - -public class SchemaChangeJob extends AlterJob { - private static final Logger LOG = LogManager.getLogger(SchemaChangeJob.class); - - private String tableName; - - // indexId -> changedColumns - private Map> changedIndexIdToSchema; - // indexId -> new schema version - private Map changedIndexIdToSchemaVersion; - // indexId -> new schema hash - private Map changedIndexIdToSchemaHash; - // indexId -> new short key column count - private Map changedIndexIdToShortKeyColumnCount; - - private TResourceInfo resourceInfo; - - // partition id -> list of replica infos - private Multimap replicaInfos; - - /* - * infos bellow do not need to persist - */ - // total unfinished replicas - private List unfinishedReplicaIds; - // for calculating process of schemaChange of an index - // indexId -> totalReplicaNum - private Multimap indexIdToTotalReplicaNum; - // use index id to count finished replica num - private Multiset indexIdToFinishedReplicaNum; - - private Multimap partitionIdToFinishedIndexIds; - - // bloom filter columns - private boolean hasBfChange; - private Set bfColumns; - private double bfFpp; - - // Init as null, to be compatible with former schema change job. - // If this is set to null, storage type will remain what it was. - // This can only set to COLUMN - private TStorageType newStorageType = null; - - private SchemaChangeJob() { - this(-1, -1, null, null, -1); - } - - public SchemaChangeJob(long dbId, long tableId, TResourceInfo resourceInfo, String tableName, long transactionId) { - super(JobType.SCHEMA_CHANGE, dbId, tableId, resourceInfo); - - this.tableName = tableName; - - this.changedIndexIdToSchema = new HashMap>(); - this.changedIndexIdToSchemaVersion = new HashMap(); - this.changedIndexIdToSchemaHash = new HashMap(); - this.changedIndexIdToShortKeyColumnCount = new HashMap(); - - this.replicaInfos = LinkedHashMultimap.create(); - - this.unfinishedReplicaIds = new LinkedList(); - this.indexIdToTotalReplicaNum = HashMultimap.create(); - this.indexIdToFinishedReplicaNum = HashMultiset.create(); - - this.partitionIdToFinishedIndexIds = HashMultimap.create(); - - this.hasBfChange = false; - this.bfColumns = null; - this.bfFpp = 0; - - this.transactionId = transactionId; - } - - public final String getTableName() { - return tableName; - } - - public synchronized int getTotalReplicaNumByIndexId(long indexId) { - return this.indexIdToTotalReplicaNum.get(indexId).size(); - } - - // schema - public void putToChangedIndexSchemaMap(long indexId, List alterSchema) { - this.changedIndexIdToSchema.put(indexId, alterSchema); - } - - public Map> getChangedIndexToSchema() { - return changedIndexIdToSchema; - } - - // schema info - public void setNewSchemaInfo(long indexId, int newSchemaVersion, int newSchemaHash, - short newShortKeyColumnCount) { - this.changedIndexIdToSchemaVersion.put(indexId, newSchemaVersion); - this.changedIndexIdToSchemaHash.put(indexId, newSchemaHash); - this.changedIndexIdToShortKeyColumnCount.put(indexId, newShortKeyColumnCount); - } - - // schema version - public int getSchemaVersionByIndexId(long indexId) { - if (changedIndexIdToSchemaVersion.containsKey(indexId)) { - return changedIndexIdToSchemaVersion.get(indexId); - } - return -1; - } - - // schema hash - public int getSchemaHashByIndexId(long indexId) { - if (changedIndexIdToSchemaHash.containsKey(indexId)) { - return changedIndexIdToSchemaHash.get(indexId); - } - return -1; - } - - // short key column count - public short getShortKeyColumnCountByIndexId(long indexId) { - if (changedIndexIdToShortKeyColumnCount.containsKey(indexId)) { - return changedIndexIdToShortKeyColumnCount.get(indexId); - } - return (short) -1; - } - - // bloom filter info - public void setTableBloomFilterInfo(boolean hasBfChange, Set bfColumns, double bfFpp) { - this.hasBfChange = hasBfChange; - this.bfColumns = bfColumns; - this.bfFpp = bfFpp; - } - - public void setNewStorageType(TStorageType newStorageType) { - Preconditions.checkState(newStorageType == TStorageType.COLUMN); - this.newStorageType = newStorageType; - } - - public boolean isSchemaHashRelated(int schemaHash) { - return changedIndexIdToSchemaHash.values().contains(schemaHash); - } - - public synchronized int getFinishedReplicaNumByIndexId(long indexId) { - return this.indexIdToFinishedReplicaNum.count(indexId); - } - - public void deleteAllTableHistorySchema() { - Database db = Catalog.getCurrentCatalog().getDbNullable(dbId); - if (db == null) { - LOG.warn("db[{}] does not exist", dbId); - return; - } - - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn(e.getMessage()); - return; - } - - olapTable.readLock(); - try { - // drop all replicas with old schemaHash - for (Partition partition : olapTable.getPartitions()) { - long partitionId = partition.getId(); - for (Long indexId : this.changedIndexIdToSchema.keySet()) { - MaterializedIndex materializedIndex = partition.getIndex(indexId); - if (materializedIndex == null) { - LOG.warn("index[{}] does not exist in partition[{}-{}-{}]", - indexId, dbId, tableId, partitionId); - continue; - } - - // delete schema hash - // the real drop task is handled by report process - // we call 'deleteNewSchemaHash' but we delete old one actually. - // cause schema hash is switched when job is finished. - Catalog.getCurrentInvertedIndex().deleteNewSchemaHash(partitionId, indexId); - LOG.info("delete old schema. db[{}], table[{}], partition[{}], index[{}]", - dbId, tableId, partitionId, indexId); - } - } // end for partitions - } finally { - olapTable.readUnlock(); - } - } - - @Override - public void addReplicaId(long parentId, long replicaId, long backendId) { - // here parent id is index id - this.unfinishedReplicaIds.add(replicaId); - this.indexIdToTotalReplicaNum.put(parentId, replicaId); - // this.backendIdToReplicaIds.put(backendId, replicaId); - } - - @Override - public synchronized void setReplicaFinished(long parentId, long replicaId) { - // here parent id is index id - if (!this.unfinishedReplicaIds.remove(replicaId)) { - // this replica is already removed - return; - } - - if (parentId == -1L) { - // find out which table is replica belongs to - for (long oneIndexId : indexIdToTotalReplicaNum.keySet()) { - Collection replicaIds = indexIdToTotalReplicaNum.get(oneIndexId); - if (replicaIds.contains(replicaId)) { - this.indexIdToFinishedReplicaNum.add(oneIndexId); - break; - } - } - } else { - this.indexIdToFinishedReplicaNum.add(parentId); - } - } - - /* - * return - * 0: sending clear tasks - * 1: all clear tasks are finished, the job is done normally. - * -1: job meet some fatal error, like db or table is missing. - */ - public int checkOrResendClearTasks() { - Preconditions.checkState(this.state == JobState.FINISHING); - // 1. check if all task finished - boolean clearFailed = false; - if (batchClearAlterTask != null) { - List allTasks = batchClearAlterTask.getAllTasks(); - for (AgentTask oneClearAlterTask : allTasks) { - ClearAlterTask clearAlterTask = (ClearAlterTask) oneClearAlterTask; - if (!clearAlterTask.isFinished()) { - clearFailed = true; - } - AgentTaskQueue.removeTask(clearAlterTask.getBackendId(), - TTaskType.CLEAR_ALTER_TASK, clearAlterTask.getSignature()); - // not remove the task from batch task, remove it by gc - } - } - if (!clearFailed && batchClearAlterTask != null) { - return 1; - } - - Database db = Catalog.getCurrentCatalog().getDbNullable(dbId); - if (db == null) { - cancelMsg = "db[" + dbId + "] does not exist"; - LOG.warn(cancelMsg); - return -1; - } - - batchClearAlterTask = new AgentBatchTask(); - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn(e.getMessage()); - return -1; - } - - olapTable.readLock(); - try { - boolean allAddSuccess = true; - LOG.info("sending clear schema change job tasks for table [{}]", tableId); - OUTER_LOOP: - for (Partition partition : olapTable.getPartitions()) { - long partitionId = partition.getId(); - for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { - for (Tablet tablet : index.getTablets()) { - List replicas = tablet.getReplicas(); - for (Replica replica : replicas) { - long backendId = replica.getBackendId(); - ClearAlterTask clearAlterTask = new ClearAlterTask(backendId, dbId, tableId, - partitionId, index.getId(), tablet.getId(), - olapTable.getSchemaHashByIndexId(index.getId())); - if (AgentTaskQueue.addTask(clearAlterTask)) { - batchClearAlterTask.addTask(clearAlterTask); - } else { - allAddSuccess = false; - break OUTER_LOOP; - } - } // end for rollupReplicas - } // end for rollupTablets - } // end for index - } // end for partition - if (!allAddSuccess) { - for (AgentTask task : batchClearAlterTask.getAllTasks()) { - AgentTaskQueue.removeTask(task.getBackendId(), task.getTaskType(), task.getSignature()); - } - batchClearAlterTask = null; - } - } finally { - olapTable.readUnlock(); - } - - LOG.info("successfully sending clear schema change job [{}]", tableId); - return 0; - } - - @Override - public boolean sendTasks() { - Preconditions.checkState(this.state == JobState.PENDING); - // here we just sending tasks to AgentTaskQueue. - // task report process will later resend this task - - LOG.info("sending schema change job {}, start txn id: {}", tableId, transactionId); - - Database db = Catalog.getCurrentCatalog().getDbNullable(dbId); - if (db == null) { - String msg = "db[" + dbId + "] does not exist"; - setMsg(msg); - LOG.warn(msg); - return false; - } - - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn(e.getMessage()); - return false; - } - - olapTable.readLock(); - try { - synchronized (this) { - Preconditions.checkNotNull(this.unfinishedReplicaIds); - - List tasks = new LinkedList(); - for (Partition partition : olapTable.getPartitions()) { - long partitionId = partition.getId(); - short replicationNum = olapTable.getPartitionInfo().getReplicaAllocation(partitionId).getTotalReplicaNum(); - for (Long indexId : this.changedIndexIdToSchema.keySet()) { - MaterializedIndex alterIndex = partition.getIndex(indexId); - if (alterIndex == null) { - cancelMsg = "index[" + indexId + "] does not exist in partition[" + partitionId + "]"; - LOG.warn(cancelMsg); - return false; - } - - List alterSchema = this.changedIndexIdToSchema.get(indexId); - int newSchemaHash = this.changedIndexIdToSchemaHash.get(indexId); - Preconditions.checkState(newSchemaHash != -1); - int baseSchemaHash = olapTable.getSchemaHashByIndexId(indexId); - short newShortKeyColumnCount = this.changedIndexIdToShortKeyColumnCount.get(indexId); - Preconditions.checkState(newShortKeyColumnCount != (short) -1); - KeysType keysType = olapTable.getKeysType(); - TKeysType schemaChangeKeysType; - if (keysType == KeysType.DUP_KEYS) { - schemaChangeKeysType = TKeysType.DUP_KEYS; - } else if (keysType == KeysType.UNIQUE_KEYS) { - schemaChangeKeysType = TKeysType.UNIQUE_KEYS; - } else { - schemaChangeKeysType = TKeysType.AGG_KEYS; - } - - TStorageType storageType = newStorageType == null ? olapTable.getStorageTypeByIndexId(indexId) - : newStorageType; - for (Tablet tablet : alterIndex.getTablets()) { - long tabletId = tablet.getId(); - short replicaSendNum = 0; - for (Replica replica : tablet.getReplicas()) { - if (replica.getState() != ReplicaState.SCHEMA_CHANGE) { - // for now, all replica should be in SCHEMA_CHANGE, - // because we don't allow tablet repair and balance during schema change. - // but in case some edge cases are not took into consideration, we cancel - // the schema change job here. - cancelMsg = String.format( - "replica %d of tablet %d in backend %d state is invalid: %s [send]", - replica.getId(), tablet.getId(), replica.getBackendId(), - replica.getState().name()); - LOG.warn(cancelMsg); - return false; - } - - long backendId = replica.getBackendId(); - long replicaId = replica.getId(); - SchemaChangeTask schemaChangeTask = - new SchemaChangeTask(resourceInfo, backendId, dbId, tableId, - partitionId, indexId, tabletId, replicaId, - alterSchema, newSchemaHash, - baseSchemaHash, newShortKeyColumnCount, - storageType, - bfColumns, bfFpp, schemaChangeKeysType); - addReplicaId(indexId, replicaId, backendId); - tasks.add(schemaChangeTask); - replicaSendNum++; - } - - if (replicaSendNum < replicationNum / 2 + 1) { - // In the case that quorum num of non-NORMAL replica(probably CLONE) - // in this tablet, schema change job can not finish. - // So cancel it. - cancelMsg = String.format("num of normal replica in tablet %d is less than quorum num", - tabletId); - LOG.warn(cancelMsg); - return false; - } - } // end for tablets - } // end for alter indices - } // end for partitions - - AgentBatchTask batchTask = new AgentBatchTask(); - // add all schemaChangeTask to AgentTaskQueue - for (AgentTask task : tasks) { - if (!AgentTaskQueue.addTask(task)) { - cancelMsg = "failed add schema change task[" + task.getTabletId() - + ":" + task.getBackendId() + "]"; - LOG.warn(cancelMsg); - return false; - } else { - batchTask.addTask(task); - } - } - - if (batchTask.getTaskNum() > 0) { - AgentTaskExecutor.submit(batchTask); - } - // change schemaChangeJob's status - this.state = JobState.RUNNING; - } // end synchronized block - } finally { - olapTable.readUnlock(); - } - - Preconditions.checkState(this.state == JobState.RUNNING); - LOG.info("successfully sending schema change job[{}]", tableId); - return true; - } - - @Override - public synchronized void cancel(OlapTable olapTable, String msg) { - // make sure to get table write lock before calling this - if (olapTable != null) { - // 1. remove all task and set state - for (Partition partition : olapTable.getPartitions()) { - if (partition.getState() == PartitionState.NORMAL) { - continue; - } - long partitionId = partition.getId(); - for (Long indexId : this.changedIndexIdToSchema.keySet()) { - MaterializedIndex index = partition.getIndex(indexId); - if (index == null || index.getState() == IndexState.NORMAL) { - continue; - } - for (Tablet tablet : index.getTablets()) { - long tabletId = tablet.getId(); - for (Replica replica : tablet.getReplicas()) { - if (replica.getState() == ReplicaState.CLONE - || replica.getState() == ReplicaState.DECOMMISSION - || replica.getState() == ReplicaState.NORMAL) { - continue; - } - Preconditions.checkState(replica.getState() == ReplicaState.SCHEMA_CHANGE); - replica.setState(ReplicaState.NORMAL); - AgentTaskQueue.removeTask(replica.getBackendId(), TTaskType.SCHEMA_CHANGE, tabletId); - } // end for replicas - } // end for tablets - - // delete schema hash in inverted index - Catalog.getCurrentInvertedIndex().deleteNewSchemaHash(partitionId, indexId); - Preconditions.checkArgument(index.getState() == IndexState.SCHEMA_CHANGE); - index.setState(IndexState.NORMAL); - } // end for indices - partition.setState(PartitionState.NORMAL); - } // end for partitions - olapTable.setState(OlapTableState.NORMAL); - } - - this.state = JobState.CANCELLED; - if (Strings.isNullOrEmpty(cancelMsg) && !Strings.isNullOrEmpty(msg)) { - this.cancelMsg = msg; - } - - this.finishedTime = System.currentTimeMillis(); - - // 2. log - Catalog.getCurrentCatalog().getEditLog().logCancelSchemaChange(this); - LOG.info("cancel schema change job[{}] finished, because: {}", - olapTable == null ? -1 : olapTable.getId(), cancelMsg); - } - - @Override - public synchronized void removeReplicaRelatedTask(long parentId, long tabletId, long replicaId, long backendId) { - // parentId is unused here - setReplicaFinished(-1, replicaId); - - // this.backendIdToReplicaIds.get(backendId).remove(replicaId); - // remove task - AgentTaskQueue.removeTask(backendId, TTaskType.SCHEMA_CHANGE, tabletId); - } - - @Override - public void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo, long reportVersion) - throws MetaNotFoundException { - Preconditions.checkArgument(task instanceof SchemaChangeTask); - SchemaChangeTask schemaChangeTask = (SchemaChangeTask) task; - - // check schema hash to avoid former schema change task try finishing current task - int finishTabletInfoSchemaHash = finishTabletInfo.getSchemaHash(); - int taskSchemaHash = schemaChangeTask.getSchemaHash(); - if (finishTabletInfoSchemaHash != taskSchemaHash) { - throw new MetaNotFoundException("Schema hash is not equal[" + finishTabletInfoSchemaHash + "-" - + taskSchemaHash + "], tablet: " + schemaChangeTask.getTabletId()); - } - - long dbId = schemaChangeTask.getDbId(); - long partitionId = schemaChangeTask.getPartitionId(); - long indexId = schemaChangeTask.getIndexId(); - long tabletId = schemaChangeTask.getTabletId(); - long replicaId = schemaChangeTask.getReplicaId(); - - // update replica's info - Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId); - OlapTable olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - olapTable.writeLockOrMetaException(); - try { - Preconditions.checkState(olapTable.getState() == OlapTableState.SCHEMA_CHANGE); - - Partition partition = olapTable.getPartition(partitionId); - if (partition == null) { - throw new MetaNotFoundException("Cannot find partition[" + partitionId + "]"); - } - Preconditions.checkState(partition.getState() == PartitionState.SCHEMA_CHANGE); - - MaterializedIndex materializedIndex = partition.getIndex(indexId); - if (materializedIndex == null) { - throw new MetaNotFoundException("Cannot find index[" + indexId + "]"); - } - Preconditions.checkState(materializedIndex.getState() == IndexState.SCHEMA_CHANGE); - - Preconditions.checkArgument(finishTabletInfo.getTabletId() == tabletId); - Tablet tablet = materializedIndex.getTablet(tabletId); - if (tablet == null) { - throw new MetaNotFoundException("Cannot find tablet[" + tabletId + "]"); - } - - Replica replica = tablet.getReplicaById(replicaId); - if (replica == null) { - throw new MetaNotFoundException("Cannot find replica[" + replicaId + "]"); - } - // replica's state may be NORMAL(due to clone), so no need to check - - long version = finishTabletInfo.getVersion(); - long versionHash = finishTabletInfo.getVersionHash(); - long dataSize = finishTabletInfo.getDataSize(); - long rowCount = finishTabletInfo.getRowCount(); - // do not need check version > replica.getVersion, because the new replica's version is first set by sc - replica.updateVersionInfo(version, versionHash, dataSize, rowCount); - if (finishTabletInfo.isSetPathHash()) { - replica.setPathHash(finishTabletInfo.getPathHash()); - } - } finally { - olapTable.writeUnlock(); - } - - Catalog.getCurrentSystemInfo().updateBackendReportVersion(schemaChangeTask.getBackendId(), - reportVersion, dbId, tableId); - setReplicaFinished(indexId, replicaId); - - LOG.info("finish schema change replica[{}]. index[{}]. tablet[{}], backend[{}]", - replicaId, indexId, tabletId, task.getBackendId()); - } - - /** - * should consider following cases: - * 1. replica is removed from this tablet. - * 2. backend is dead or is dropped from system - * - * we make new schema visible, but keep table's state as SCHEMA_CHANGE. - * 1. Make the new schema visible, because we want that the following load jobs will only load - * data to the new tablet. - * 2. keep the table's state in SCHEMA_CHANGE, because we don't want another alter job being processed. - */ - @Override - public int tryFinishJob() { - if (this.state != JobState.RUNNING) { - LOG.info("schema change job[{}] is not running.", tableId); - return 0; - } - - Database db = Catalog.getCurrentCatalog().getDbNullable(dbId); - if (db == null) { - cancelMsg = String.format("database %d does not exist", dbId); - LOG.warn(cancelMsg); - return -1; - } - - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - olapTable.writeLockOrMetaException(); - } catch (MetaNotFoundException e) { - LOG.warn(e.getMessage()); - return -1; - } - try { - synchronized (this) { - boolean hasUnfinishedPartition = false; - for (Partition partition : olapTable.getPartitions()) { - long partitionId = partition.getId(); - short expectReplicationNum = olapTable.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum(); - boolean hasUnfinishedIndex = false; - for (long indexId : this.changedIndexIdToSchema.keySet()) { - MaterializedIndex materializedIndex = partition.getIndex(indexId); - if (materializedIndex == null) { - // this should not happen, we do not allow dropping rollup during schema change. - // cancel the job here. - cancelMsg = String.format("index %d is missing", indexId); - LOG.warn(cancelMsg); - return -1; - } - - for (Tablet tablet : materializedIndex.getTablets()) { - List replicas = tablet.getReplicas(); - List errorReplicas = Lists.newArrayList(); - int healthNum = replicas.size(); - for (Replica replica : replicas) { - if (replica.getState() != ReplicaState.SCHEMA_CHANGE) { - // all replicas should be in state SCHEMA_CHANGE - cancelMsg = String.format( - "replica %d of tablet %d in backend %d state is invalid: %s [try finish]", - replica.getId(), tablet.getId(), replica.getBackendId(), - replica.getState().name()); - LOG.warn(cancelMsg); - return -1; - } - - if (!checkBackendState(replica)) { - LOG.warn("backend {} state is abnormal, set replica {} of tablet {} as bad", - replica.getBackendId(), tablet.getId(), replica.getId()); - errorReplicas.add(replica); - --healthNum; - continue; - } - - if (replica.getLastFailedVersion() > 0 && System.currentTimeMillis() - - replica.getLastFailedTimestamp() > Config.max_backend_down_time_second - * 1000) { - LOG.warn("replica {} of tablet {} last failed version > 0, " - + "and last for an hour, set it as bad", replica, tablet.getId()); - --healthNum; - continue; - } - } - - if (healthNum < (expectReplicationNum / 2 + 1)) { - cancelMsg = String.format("schema change job[%d] cancelled. " - + "tablet[%d] has few health replica." - + " num: %d", tableId, tablet.getId(), healthNum); - LOG.warn(cancelMsg); - return -1; - } - - for (Replica errReplica : errorReplicas) { - // For now, err replicas are those replicas which the backends they belong to is dead. - // We need to set these replicas as finished to let the schema change job - // finished. - setReplicaFinished(indexId, errReplica.getId()); - // remove the replica from backend to replica map - // backendIdToReplicaIds.get(errReplica.getBackendId()).remove(errReplica.getId()); - // remove error replica related task - AgentTaskQueue.removeTask(errReplica.getBackendId(), TTaskType.SCHEMA_CHANGE, - tablet.getId()); - } - } // end for tablets - - // check if index is finished - if (!this.partitionIdToFinishedIndexIds.containsKey(partitionId) - || !this.partitionIdToFinishedIndexIds.get(partitionId).contains(indexId)) { - int finishedReplicaNum = indexIdToFinishedReplicaNum.count(indexId); - int totalReplicaNum = indexIdToTotalReplicaNum.get(indexId).size(); - if (finishedReplicaNum < totalReplicaNum) { - LOG.debug("index[{}] has unfinished replica. {}/{}", indexId, - finishedReplicaNum, totalReplicaNum); - hasUnfinishedIndex = true; - continue; - } - } - - // index is finished - // remove task for safety - // task may be left if some backends are down during schema change - if (!this.partitionIdToFinishedIndexIds.containsKey(partitionId) - || !this.partitionIdToFinishedIndexIds.get(partitionId).contains(indexId)) { - for (Tablet tablet : materializedIndex.getTablets()) { - for (Replica replica : tablet.getReplicas()) { - AgentTaskQueue.removeTask(replica.getBackendId(), TTaskType.SCHEMA_CHANGE, - tablet.getId()); - } - } - - Preconditions.checkState(materializedIndex.getState() == IndexState.SCHEMA_CHANGE); - this.partitionIdToFinishedIndexIds.put(partitionId, indexId); - LOG.debug("schema change tasks finished in table[{}]", materializedIndex.getId()); - } - } // end for indices - - if (hasUnfinishedIndex) { - hasUnfinishedPartition = true; - } - - // all table finished in this partition - LOG.info("schema change finished in partition {}, table: {}", partition.getId(), olapTable.getId()); - } // end for partitions - - if (hasUnfinishedPartition) { - return 0; - } - - Preconditions.checkState(unfinishedReplicaIds.isEmpty()); - - // all partitions are finished - // update state and save replica info - Preconditions.checkState(olapTable.getState() == OlapTableState.SCHEMA_CHANGE); - for (Partition partition : olapTable.getPartitions()) { - Preconditions.checkState(partition.getState() == PartitionState.SCHEMA_CHANGE); - long partitionId = partition.getId(); - for (long indexId : this.changedIndexIdToSchema.keySet()) { - MaterializedIndex materializedIndex = partition.getIndex(indexId); - int schemaHash = changedIndexIdToSchemaHash.get(indexId); - Preconditions.checkState(materializedIndex.getState() == IndexState.SCHEMA_CHANGE); - for (Tablet tablet : materializedIndex.getTablets()) { - long tabletId = tablet.getId(); - for (Replica replica : tablet.getReplicas()) { - if (replica.getState() != ReplicaState.SCHEMA_CHANGE) { - // all replicas should be in state SCHEMA_CHANGE - cancelMsg = String.format( - "replica %d of tablet %d in backend %d state is invalid: %s [finish]", - replica.getId(), tablet.getId(), replica.getBackendId(), - replica.getState().name()); - LOG.warn(cancelMsg); - } - - ReplicaPersistInfo replicaInfo = ReplicaPersistInfo.createForSchemaChange(partitionId, - indexId, tabletId, - replica.getBackendId(), - replica.getVersion(), - replica.getVersionHash(), - schemaHash, - replica.getDataSize(), - replica.getRowCount(), - replica.getLastFailedVersion(), - replica.getLastFailedVersionHash(), - replica.getLastSuccessVersion(), - replica.getLastSuccessVersionHash()); - this.replicaInfos.put(partitionId, replicaInfo); - - replica.setState(ReplicaState.NORMAL); - replica.setSchemaHash(schemaHash); - - // remove tasks for safety - AgentTaskQueue.removeTask(replica.getBackendId(), TTaskType.SCHEMA_CHANGE, - tabletId); - } // end for replicas - } // end for tablets - - // update schema hash - Catalog.getCurrentInvertedIndex().updateToNewSchemaHash(partitionId, indexId); - materializedIndex.setState(IndexState.NORMAL); - } // end for indices - partition.setState(PartitionState.NORMAL); - } // end for partitions - - // 2. update to new schema for each index - for (Map.Entry> entry : changedIndexIdToSchema.entrySet()) { - Long indexId = entry.getKey(); - int schemaVersion = changedIndexIdToSchemaVersion.get(indexId); - int schemaHash = changedIndexIdToSchemaHash.get(indexId); - short shortKeyColumnCount = changedIndexIdToShortKeyColumnCount.get(indexId); - olapTable.setIndexMeta(indexId, null, entry.getValue(), schemaVersion, schemaHash, - shortKeyColumnCount, newStorageType, null); - } - - // 3. update base schema if changed - if (this.changedIndexIdToSchema.containsKey(olapTable.getBaseIndexId())) { - olapTable.setNewFullSchema(this.changedIndexIdToSchema.get(olapTable.getBaseIndexId())); - } - - // 4. update table bloom filter columns - if (hasBfChange) { - olapTable.setBloomFilterInfo(bfColumns, bfFpp); - } - - this.state = JobState.FINISHING; - this.transactionId = Catalog.getCurrentGlobalTransactionMgr().getTransactionIDGenerator().getNextTransactionId(); - } - } finally { - olapTable.writeUnlock(); - } - - Catalog.getCurrentCatalog().getEditLog().logFinishingSchemaChange(this); - LOG.info("schema change job is finishing. finishing txn id: {} table {}", transactionId, tableId); - return 1; - } - - @Override - public void finishJob() { - Database db = Catalog.getCurrentCatalog().getDbNullable(dbId); - if (db == null) { - cancelMsg = String.format("database %d does not exist", dbId); - LOG.warn(cancelMsg); - return; - } - - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn(e.getMessage()); - return; - } - - olapTable.writeLock(); - try { - olapTable.setState(OlapTableState.NORMAL); - } finally { - olapTable.writeUnlock(); - } - - this.finishedTime = System.currentTimeMillis(); - LOG.info("finished schema change job: {}", tableId); - } - - @Override - public synchronized void clear() { - changedIndexIdToSchema = null; - resourceInfo = null; - replicaInfos = null; - unfinishedReplicaIds = null; - indexIdToTotalReplicaNum = null; - indexIdToFinishedReplicaNum = null; - partitionIdToFinishedIndexIds = null; - } - - @Override - public void replayInitJob(Database db) { - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn("[INCONSISTENT META] replay init schema change job failed", e); - return; - } - olapTable.writeLock(); - try { - // change the state of table/partition and replica, then add object to related List and Set - for (Partition partition : olapTable.getPartitions()) { - for (Map.Entry entry : changedIndexIdToSchemaHash.entrySet()) { - MaterializedIndex index = partition.getIndex(entry.getKey()); - // set state to SCHEMA_CHANGE - for (Tablet tablet : index.getTablets()) { - for (Replica replica : tablet.getReplicas()) { - if (replica.getState() == ReplicaState.CLONE - || replica.getState() == ReplicaState.DECOMMISSION) { - // add log here, because there should no more CLONE replica when processing alter jobs. - LOG.warn(String.format( - "replica %d of tablet %d in backend %d state is invalid: %s", - replica.getId(), tablet.getId(), replica.getBackendId(), - replica.getState().name())); - continue; - } - replica.setState(ReplicaState.SCHEMA_CHANGE); - } - } - index.setState(IndexState.SCHEMA_CHANGE); - - Catalog.getCurrentInvertedIndex().setNewSchemaHash(partition.getId(), entry.getKey(), - entry.getValue()); - } - - partition.setState(PartitionState.SCHEMA_CHANGE); - } // end for partitions - - olapTable.setState(OlapTableState.SCHEMA_CHANGE); - - // reset status to PENDING for resending the tasks in polling thread - this.state = JobState.PENDING; - } finally { - olapTable.writeUnlock(); - } - } - - @Override - public void replayFinishing(Database db) { - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn("[INCONSISTENT META] replay finishing schema change job failed", e); - return; - } - olapTable.writeLock(); - try { - // set the status to normal - for (Partition partition : olapTable.getPartitions()) { - long partitionId = partition.getId(); - for (Map.Entry entry : changedIndexIdToSchemaHash.entrySet()) { - MaterializedIndex index = partition.getIndex(entry.getKey()); - for (Tablet tablet : index.getTablets()) { - for (Replica replica : tablet.getReplicas()) { - replica.setState(ReplicaState.NORMAL); - } - } - - index.setState(IndexState.NORMAL); - - // update to new schema hash in inverted index - Catalog.getCurrentInvertedIndex().updateToNewSchemaHash(partitionId, index.getId()); - Catalog.getCurrentInvertedIndex().deleteNewSchemaHash(partitionId, index.getId()); - } - partition.setState(PartitionState.NORMAL); - - // update replica info - Collection replicaInfo = replicaInfos.get(partition.getId()); - if (replicaInfo != null) { - for (ReplicaPersistInfo info : replicaInfo) { - MaterializedIndex mIndex = (MaterializedIndex) partition.getIndex(info.getIndexId()); - int schemaHash = info.getSchemaHash(); - // for compatibility - if (schemaHash == -1) { - schemaHash = getSchemaHashByIndexId(info.getIndexId()); - } - Tablet tablet = mIndex.getTablet(info.getTabletId()); - if (info.getOpType() == ReplicaOperationType.SCHEMA_CHANGE) { - Replica replica = tablet.getReplicaByBackendId(info.getBackendId()); - replica.updateVersionInfo(info.getVersion(), info.getVersionHash(), - info.getLastFailedVersion(), - info.getLastFailedVersionHash(), - info.getLastSuccessVersion(), - info.getLastSuccessVersionHash()); - replica.setSchemaHash(schemaHash); - } else if (info.getOpType() == ReplicaOperationType.DELETE) { - // remove the replica from replica group - tablet.deleteReplicaByBackendId(info.getBackendId()); - } - } - } - } // end for partitions - - // update schema - for (Map.Entry> entry : changedIndexIdToSchema.entrySet()) { - long indexId = entry.getKey(); - int schemaVersion = getSchemaVersionByIndexId(indexId); - int schemaHash = getSchemaHashByIndexId(indexId); - short shortKeyColumnCount = getShortKeyColumnCountByIndexId(indexId); - olapTable.setIndexMeta(indexId, null, entry.getValue(), schemaVersion, schemaHash, - shortKeyColumnCount, newStorageType, null); - - if (indexId == olapTable.getBaseIndexId()) { - olapTable.setNewFullSchema(entry.getValue()); - } - } - - // bloom filter columns - if (hasBfChange) { - olapTable.setBloomFilterInfo(bfColumns, bfFpp); - } // end for partitions - } finally { - olapTable.writeUnlock(); - } - - LOG.info("replay finishing schema change job: {}", tableId); - } - - @Override - public void replayFinish(Database db) { - // if this is an old job, then should also update table or replica state - if (transactionId < 0) { - replayFinishing(db); - } - - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn("[INCONSISTENT META] replay finish schema change job failed", e); - return; - } - olapTable.writeLock(); - try { - olapTable.setState(OlapTableState.NORMAL); - } finally { - olapTable.writeUnlock(); - } - LOG.info("replay finish schema change job: {}", tableId); - } - - @Override - public void replayCancel(Database db) { - // restore partition's state - OlapTable olapTable; - try { - olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP); - } catch (MetaNotFoundException e) { - LOG.warn("[INCONSISTENT META] replay cancel schema change job failed", e); - return; - } - olapTable.writeLock(); - try { - for (Partition partition : olapTable.getPartitions()) { - long partitionId = partition.getId(); - for (Long indexId : this.changedIndexIdToSchema.keySet()) { - MaterializedIndex index = partition.getIndex(indexId); - if (index == null) { - continue; - } - for (Tablet tablet : index.getTablets()) { - for (Replica replica : tablet.getReplicas()) { - if (replica.getState() == ReplicaState.CLONE - || replica.getState() == ReplicaState.DECOMMISSION - || replica.getState() == ReplicaState.NORMAL) { - continue; - } - replica.setState(ReplicaState.NORMAL); - } // end for replicas - } // end for tablets - - Preconditions.checkState(index.getState() == IndexState.SCHEMA_CHANGE, index.getState()); - index.setState(IndexState.NORMAL); - - // delete new schema hash in invered index - Catalog.getCurrentInvertedIndex().deleteNewSchemaHash(partitionId, indexId); - } // end for indices - - Preconditions.checkState(partition.getState() == PartitionState.SCHEMA_CHANGE, - partition.getState()); - partition.setState(PartitionState.NORMAL); - } // end for partitions - - olapTable.setState(OlapTableState.NORMAL); - } finally { - olapTable.writeUnlock(); - } - } - - @Override - public void getJobInfo(List> jobInfos, OlapTable tbl) { - if (changedIndexIdToSchemaVersion == null) { - // for compatibility - if (state == JobState.FINISHED || state == JobState.CANCELLED) { - List jobInfo = new ArrayList(); - jobInfo.add(tableId); // job id - jobInfo.add(tbl.getName()); // table name - jobInfo.add(TimeUtils.longToTimeString(createTime)); - jobInfo.add(TimeUtils.longToTimeString(finishedTime)); - jobInfo.add(FeConstants.null_string); // index name - jobInfo.add(FeConstants.null_string); // index id - jobInfo.add(FeConstants.null_string); // origin id - jobInfo.add(FeConstants.null_string); // schema version - jobInfo.add(-1); // transaction id - jobInfo.add(state.name()); // job state - jobInfo.add(cancelMsg); - jobInfo.add(FeConstants.null_string); // progress - jobInfo.add(Config.alter_table_timeout_second); // timeout - jobInfos.add(jobInfo); - return; - } - - // in previous version, changedIndexIdToSchema is set to null - // when job is finished or cancelled. - // so if changedIndexIdToSchema == null, the job'state must be FINISHED or CANCELLED - return; - } - - Map indexProgress = new HashMap(); - Map indexState = new HashMap(); - - // calc progress and state for each table - for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { - if (tbl.getIndexNameById(indexId) == null) { - // this index may be dropped, and this should be a FINISHED job, just use a dummy info to show - indexState.put(indexId, IndexState.NORMAL.name()); - indexProgress.put(indexId, "100%"); - } else { - int totalReplicaNum = 0; - int finishedReplicaNum = 0; - String idxState = IndexState.NORMAL.name(); - for (Partition partition : tbl.getPartitions()) { - MaterializedIndex index = partition.getIndex(indexId); - if (state == JobState.RUNNING) { - int tableReplicaNum = getTotalReplicaNumByIndexId(indexId); - int tableFinishedReplicaNum = getFinishedReplicaNumByIndexId(indexId); - Preconditions.checkState(!(tableReplicaNum == 0 && tableFinishedReplicaNum == -1)); - Preconditions.checkState(tableFinishedReplicaNum <= tableReplicaNum, - tableFinishedReplicaNum + "/" + tableReplicaNum); - totalReplicaNum += tableReplicaNum; - finishedReplicaNum += tableFinishedReplicaNum; - } - - if (index.getState() != IndexState.NORMAL) { - idxState = index.getState().name(); - } - } - - indexState.put(indexId, idxState); - - if (Catalog.getCurrentCatalog().isMaster() && state == JobState.RUNNING && totalReplicaNum != 0) { - indexProgress.put(indexId, (finishedReplicaNum * 100 / totalReplicaNum) + "%"); - } else { - indexProgress.put(indexId, "0%"); - } - } - } - - for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { - List jobInfo = new ArrayList(); - - jobInfo.add(tableId); - jobInfo.add(tbl.getName()); - jobInfo.add(TimeUtils.longToTimeString(createTime)); - jobInfo.add(TimeUtils.longToTimeString(finishedTime)); - jobInfo.add(tbl.getIndexNameById(indexId) == null ? FeConstants.null_string : tbl.getIndexNameById(indexId)); // index name - jobInfo.add(indexId); - jobInfo.add(indexId); // origin index id - // index schema version and schema hash - jobInfo.add(changedIndexIdToSchemaVersion.get(indexId) + ":" + changedIndexIdToSchemaHash.get(indexId)); - jobInfo.add(transactionId); - jobInfo.add(state.name()); // job state - jobInfo.add(cancelMsg); - if (state == JobState.RUNNING) { - jobInfo.add(indexProgress.get(indexId) == null ? FeConstants.null_string : indexProgress.get(indexId)); // progress - } else { - jobInfo.add(FeConstants.null_string); - } - jobInfo.add(Config.alter_table_timeout_second); - jobInfos.add(jobInfo); - } // end for indexIds - } - - @Override - public synchronized void write(DataOutput out) throws IOException { - super.write(out); - - Text.writeString(out, tableName); - - // 'unfinishedReplicaIds', 'indexIdToTotalReplicaNum' and 'indexIdToFinishedReplicaNum' - // don't need persist. build it when send tasks - - // columns - if (changedIndexIdToSchema != null) { - out.writeBoolean(true); - out.writeInt(changedIndexIdToSchema.size()); - for (Entry> entry : changedIndexIdToSchema.entrySet()) { - out.writeLong(entry.getKey()); - out.writeInt(entry.getValue().size()); - for (Column column : entry.getValue()) { - column.write(out); - } - } - } else { - out.writeBoolean(false); - } - - // schema version and hash, and short key - if (changedIndexIdToSchemaVersion != null) { - out.writeBoolean(true); - out.writeInt(changedIndexIdToSchemaVersion.size()); - for (Entry entry : changedIndexIdToSchemaVersion.entrySet()) { - out.writeLong(entry.getKey()); - // schema version - out.writeInt(entry.getValue()); - // schema hash - out.writeInt(changedIndexIdToSchemaHash.get(entry.getKey())); - // short key column count - out.writeShort(changedIndexIdToShortKeyColumnCount.get(entry.getKey())); - } - } - - // replicaInfos is saving for restoring schemaChangeJobFinished - if (replicaInfos != null) { - out.writeBoolean(true); - out.writeInt(replicaInfos.keySet().size()); - for (long partitionId : replicaInfos.keySet()) { - Collection infos = replicaInfos.get(partitionId); - out.writeLong(partitionId); - out.writeInt(infos.size()); - for (ReplicaPersistInfo replicaPersistInfo : infos) { - replicaPersistInfo.write(out); - } - } - } else { - out.writeBoolean(false); - } - - // bloom filter columns - out.writeBoolean(hasBfChange); - if (bfColumns != null) { - out.writeBoolean(true); - out.writeInt(bfColumns.size()); - for (String bfColumn : bfColumns) { - Text.writeString(out, bfColumn); - } - out.writeDouble(bfFpp); - } else { - out.writeBoolean(false); - } - - // storage type - if (newStorageType == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - } - } - - public void readFields(DataInput in) throws IOException { - super.readFields(in); - - tableName = Text.readString(in); - - if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_48) { - if (in.readBoolean()) { - int count = in.readInt(); - for (int i = 0; i < count; i++) { - long indexId = in.readLong(); - int columnNum = in.readInt(); - List columns = new LinkedList(); - for (int j = 0; j < columnNum; j++) { - Column column = Column.read(in); - columns.add(column); - } - changedIndexIdToSchema.put(indexId, columns); - // schema version - changedIndexIdToSchemaVersion.put(indexId, in.readInt()); - // schema hash - changedIndexIdToSchemaHash.put(indexId, in.readInt()); - // short key column count - changedIndexIdToShortKeyColumnCount.put(indexId, in.readShort()); - } - } - } else { - // columns - if (in.readBoolean()) { - int count = in.readInt(); - for (int i = 0; i < count; i++) { - long indexId = in.readLong(); - int columnNum = in.readInt(); - List columns = new LinkedList(); - for (int j = 0; j < columnNum; j++) { - Column column = Column.read(in); - columns.add(column); - } - changedIndexIdToSchema.put(indexId, columns); - } - } - - // schema version and hash, and short key - if (in.readBoolean()) { - int count = in.readInt(); - for (int i = 0; i < count; i++) { - long indexId = in.readLong(); - // schema version - changedIndexIdToSchemaVersion.put(indexId, in.readInt()); - // schema hash - changedIndexIdToSchemaHash.put(indexId, in.readInt()); - // short key column count - changedIndexIdToShortKeyColumnCount.put(indexId, in.readShort()); - } - } - } - - if (in.readBoolean()) { - int count = in.readInt(); - for (int i = 0; i < count; ++i) { - long partitionId = in.readLong(); - int infoSize = in.readInt(); - for (int j = 0; j < infoSize; j++) { - ReplicaPersistInfo info = ReplicaPersistInfo.read(in); - replicaInfos.put(partitionId, info); - } - } - } - - // bloom filter columns - if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_9) { - hasBfChange = in.readBoolean(); - if (in.readBoolean()) { - int bfColumnCount = in.readInt(); - bfColumns = Sets.newHashSet(); - for (int i = 0; i < bfColumnCount; i++) { - bfColumns.add(Text.readString(in)); - } - - bfFpp = in.readDouble(); - } - } - - if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_39) { - if (in.readBoolean()) { - newStorageType = TStorageType.COLUMN; - } - } - } - - public static SchemaChangeJob read(DataInput in) throws IOException { - SchemaChangeJob schemaChangeJob = new SchemaChangeJob(); - schemaChangeJob.readFields(in); - return schemaChangeJob; - } - - public boolean equals(Object obj) { - return true; - } - - @Override - public String toString() { - return "SchemaChangeJob [tableName=" + tableName + ", type=" + type + ", state=" + state + ", dbId=" + dbId - + ", tableId=" + tableId + ", transactionId=" + transactionId + ", isPreviousLoadFinished=" - + isPreviousLoadFinished + ", createTime=" + createTime + ", finishedTime=" + finishedTime + "]"; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java index f8a59bcfb723c8..3db19f621a0987 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java @@ -17,7 +17,6 @@ package org.apache.doris.alter; -import org.apache.doris.alter.AlterJob.JobState; import org.apache.doris.analysis.AddBackendClause; import org.apache.doris.analysis.AddFollowerClause; import org.apache.doris.analysis.AddObserverClause; @@ -71,26 +70,12 @@ public SystemHandler() { super("cluster"); } - @Override - public void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo, long reportVersion) - throws MetaNotFoundException { - } - @Override protected void runAfterCatalogReady() { super.runAfterCatalogReady(); - runOldAlterJob(); runAlterJobV2(); } - @Deprecated - private void runOldAlterJob() { - // just remove all old decommission jobs. the decommission state is already marked in Backend, - // and we no long need decommission job. - alterJobs.clear(); - finishedOrCancelledAlterJobs.clear(); - } - // check all decommissioned backends, if there is no tablet on that backend, drop it. private void runAlterJobV2() { SystemInfoService systemInfoService = Catalog.getCurrentSystemInfo(); @@ -263,25 +248,4 @@ public synchronized void cancel(CancelStmt stmt) throws DdlException { } } } - - @Override - public void replayInitJob(AlterJob alterJob, Catalog catalog) { - DecommissionBackendJob decommissionBackendJob = (DecommissionBackendJob) alterJob; - LOG.debug("replay init decommission backend job: {}", decommissionBackendJob.getBackendIdsString()); - addAlterJob(alterJob); - } - - @Override - public void replayFinish(AlterJob alterJob, Catalog catalog) { - LOG.debug("replay finish decommission backend job: {}", - ((DecommissionBackendJob) alterJob).getBackendIdsString()); - removeAlterJob(alterJob.getTableId()); - alterJob.setState(JobState.FINISHED); - addFinishedOrCancelledAlterJob(alterJob); - } - - @Override - public void replayCancel(AlterJob alterJob, Catalog catalog) { - throw new NotImplementedException(); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DecommissionBackendClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DecommissionBackendClause.java index e83a73902262cf..ed45de5b2e262b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DecommissionBackendClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DecommissionBackendClause.java @@ -17,7 +17,7 @@ package org.apache.doris.analysis; -import org.apache.doris.alter.DecommissionBackendJob.DecommissionType; +import org.apache.doris.alter.DecommissionType; import java.util.List; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index daf2639c402a9b..02a543a7f7965f 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -18,10 +18,9 @@ package org.apache.doris.catalog; import org.apache.doris.alter.Alter; -import org.apache.doris.alter.AlterJob; -import org.apache.doris.alter.AlterJob.JobType; import org.apache.doris.alter.AlterJobV2; -import org.apache.doris.alter.DecommissionBackendJob.DecommissionType; +import org.apache.doris.alter.AlterJobV2.JobType; +import org.apache.doris.alter.DecommissionType; import org.apache.doris.alter.MaterializedViewHandler; import org.apache.doris.alter.SchemaChangeHandler; import org.apache.doris.alter.SystemHandler; @@ -293,6 +292,9 @@ import com.sleepycat.je.rep.InsufficientLogException; import com.sleepycat.je.rep.NetworkRestore; import com.sleepycat.je.rep.NetworkRestoreConfig; + +import scala.reflect.internal.Trees.New; + import org.codehaus.jackson.map.ObjectMapper; public class Catalog { @@ -1803,9 +1805,7 @@ public long loadAlterJob(DataInputStream dis, long checksum) throws IOException long newChecksum = checksum; for (JobType type : JobType.values()) { if (type == JobType.DECOMMISSION_BACKEND) { - if (Catalog.getCurrentCatalogJournalVersion() >= 5) { - newChecksum = loadAlterJob(dis, newChecksum, type); - } + throw new IOException("There should be no DECOMMISSION_BACKEND jobs, not use this FE version"); } else { newChecksum = loadAlterJob(dis, newChecksum, type); } @@ -1815,52 +1815,22 @@ public long loadAlterJob(DataInputStream dis, long checksum) throws IOException } public long loadAlterJob(DataInputStream dis, long checksum, JobType type) throws IOException { - Map alterJobs = null; - ConcurrentLinkedQueue finishedOrCancelledAlterJobs = null; Map alterJobsV2 = Maps.newHashMap(); - if (type == JobType.ROLLUP) { - alterJobs = this.getRollupHandler().unprotectedGetAlterJobs(); - finishedOrCancelledAlterJobs = this.getRollupHandler().unprotectedGetFinishedOrCancelledAlterJobs(); - } else if (type == JobType.SCHEMA_CHANGE) { - alterJobs = this.getSchemaChangeHandler().unprotectedGetAlterJobs(); - finishedOrCancelledAlterJobs = this.getSchemaChangeHandler().unprotectedGetFinishedOrCancelledAlterJobs(); - alterJobsV2 = this.getSchemaChangeHandler().getAlterJobsV2(); - } else if (type == JobType.DECOMMISSION_BACKEND) { - alterJobs = this.getClusterHandler().unprotectedGetAlterJobs(); - finishedOrCancelledAlterJobs = this.getClusterHandler().unprotectedGetFinishedOrCancelledAlterJobs(); - } // alter jobs int size = dis.readInt(); long newChecksum = checksum ^ size; - for (int i = 0; i < size; i++) { - long tableId = dis.readLong(); - newChecksum ^= tableId; - AlterJob job = AlterJob.read(dis); - alterJobs.put(tableId, job); - - // init job - Database db = getDbNullable(job.getDbId()); - // should check job state here because the job is finished but not removed from alter jobs list - if (db != null && (job.getState() == org.apache.doris.alter.AlterJob.JobState.PENDING - || job.getState() == org.apache.doris.alter.AlterJob.JobState.RUNNING)) { - job.replayInitJob(db); - } + if (size > 0) { + // There should be no old alter jobs, if exist throw exception, should not use this FE version + throw new IOException("There are [" + size + "] old alter jobs, should not happen"); } if (Catalog.getCurrentCatalogJournalVersion() >= 2) { // finished or cancelled jobs - long currentTimeMs = System.currentTimeMillis(); size = dis.readInt(); newChecksum ^= size; - for (int i = 0; i < size; i++) { - long tableId = dis.readLong(); - newChecksum ^= tableId; - AlterJob job = AlterJob.read(dis); - if ((currentTimeMs - job.getCreateTimeMs()) / 1000 <= Config.history_job_keep_max_second) { - // delete history jobs - finishedOrCancelledAlterJobs.add(job); - } + if (size > 0) { + throw new IOException("There are [" + size + "] old finished or cancelled alter jobs, should not happen"); } } @@ -2160,43 +2130,19 @@ public long saveAlterJob(CountingDataOutputStream dos, long checksum) throws IOE } public long saveAlterJob(CountingDataOutputStream dos, long checksum, JobType type) throws IOException { - Map alterJobs = null; - ConcurrentLinkedQueue finishedOrCancelledAlterJobs = null; Map alterJobsV2 = Maps.newHashMap(); - if (type == JobType.ROLLUP) { - alterJobs = this.getRollupHandler().unprotectedGetAlterJobs(); - finishedOrCancelledAlterJobs = this.getRollupHandler().unprotectedGetFinishedOrCancelledAlterJobs(); - alterJobsV2 = this.getRollupHandler().getAlterJobsV2(); - } else if (type == JobType.SCHEMA_CHANGE) { - alterJobs = this.getSchemaChangeHandler().unprotectedGetAlterJobs(); - finishedOrCancelledAlterJobs = this.getSchemaChangeHandler().unprotectedGetFinishedOrCancelledAlterJobs(); - alterJobsV2 = this.getSchemaChangeHandler().getAlterJobsV2(); - } else if (type == JobType.DECOMMISSION_BACKEND) { - alterJobs = this.getClusterHandler().unprotectedGetAlterJobs(); - finishedOrCancelledAlterJobs = this.getClusterHandler().unprotectedGetFinishedOrCancelledAlterJobs(); - } - // alter jobs - int size = alterJobs.size(); + // alter jobs == 0 + // If the FE version upgrade from old version, if it have alter jobs, the FE will failed during start process + // the number of old version alter jobs has to be 0 + int size = 0; checksum ^= size; dos.writeInt(size); - for (Entry entry : alterJobs.entrySet()) { - long tableId = entry.getKey(); - checksum ^= tableId; - dos.writeLong(tableId); - entry.getValue().write(dos); - } // finished or cancelled jobs - size = finishedOrCancelledAlterJobs.size(); + size = 0; checksum ^= size; dos.writeInt(size); - for (AlterJob alterJob : finishedOrCancelledAlterJobs) { - long tableId = alterJob.getTableId(); - checksum ^= tableId; - dos.writeLong(tableId); - alterJob.write(dos); - } // alter job v2 size = alterJobsV2.size(); @@ -5982,9 +5928,6 @@ public void clear() { load.getIdToLoadJob().clear(); // load = null; } - - SingletonHolder.INSTANCE.getRollupHandler().unprotectedGetAlterJobs().clear(); - SingletonHolder.INSTANCE.getSchemaChangeHandler().unprotectedGetAlterJobs().clear(); System.gc(); } @@ -7264,8 +7207,6 @@ private void setReplicaStatusInternal(long tabletId, long backendId, ReplicaStat public void eraseDatabase(long dbId, boolean needEditLog) { // remove jobs Catalog.getCurrentCatalog().getLoadInstance().removeDbLoadJob(dbId); - Catalog.getCurrentCatalog().getSchemaChangeHandler().removeDbAlterJob(dbId); - Catalog.getCurrentCatalog().getRollupHandler().removeDbAlterJob(dbId); // remove database transaction manager Catalog.getCurrentCatalog().getGlobalTransactionMgr().removeDatabaseTransactionMgr(dbId); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java index 17fdea629fcf29..e808ec3c05c398 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java @@ -17,7 +17,6 @@ package org.apache.doris.load; -import org.apache.doris.alter.RollupJob; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; @@ -377,14 +376,7 @@ private Set submitPushTasks(LoadJob job, Database db) { } TableLoadInfo tableLoadInfo = tableEntry.getValue(); // check if the job is submit during rollup - RollupJob rollupJob = (RollupJob) Catalog.getCurrentCatalog().getRollupHandler().getAlterJob(tableId); boolean autoLoadToTwoTablet = true; - if (rollupJob != null && job.getTransactionId() > 0) { - long rollupIndexId = rollupJob.getRollupIndexId(); - if (tableLoadInfo.containsIndex(rollupIndexId)) { - autoLoadToTwoTablet = false; - } - } for (Entry partitionEntry : tableLoadInfo.getIdToPartitionLoadInfo().entrySet()) { long partitionId = partitionEntry.getKey(); @@ -418,20 +410,8 @@ private Set submitPushTasks(LoadJob job, Database db) { * 2. just send push tasks to indexes which it contains, ignore others */ if (!tableLoadInfo.containsIndex(indexId)) { - if (rollupJob == null) { - // new process, just continue - continue; - } - - if (rollupJob.getRollupIndexId() == indexId) { - continue; - } else { - // if the index is not during rollup and not contained in table load info, it a fatal error - // return null, will cancel the load job - LOG.warn("could not find index {} in table load info, and could not find " - + "it in rollup job, it is a fatal error", indexId); - return null; - } + // new process, just continue + continue; } // add to jobTotalTablets first. diff --git a/fe/fe-core/src/main/java/org/apache/doris/metric/MetricRepo.java b/fe/fe-core/src/main/java/org/apache/doris/metric/MetricRepo.java index 90221e968bffe0..0d4de20ca08ad7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/metric/MetricRepo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/metric/MetricRepo.java @@ -19,7 +19,7 @@ import org.apache.doris.alter.Alter; -import org.apache.doris.alter.AlterJob.JobType; +import org.apache.doris.alter.AlterJobV2.JobType; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.TabletInvertedIndex; import org.apache.doris.common.Config; diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index 169e89e55137ed..e836007da6eddd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -20,8 +20,6 @@ import org.apache.doris.alter.AlterJobV2; import org.apache.doris.alter.BatchAlterJobPersistInfo; import org.apache.doris.alter.DecommissionBackendJob; -import org.apache.doris.alter.RollupJob; -import org.apache.doris.alter.SchemaChangeJob; import org.apache.doris.analysis.UserIdentity; import org.apache.doris.backup.BackupJob; import org.apache.doris.backup.Repository; @@ -276,26 +274,6 @@ public static void loadJournal(Catalog catalog, JournalEntity journal) { catalog.getBackupHandler().replayAddJob(job); break; } - case OperationType.OP_START_ROLLUP: { - RollupJob job = (RollupJob) journal.getData(); - catalog.getRollupHandler().replayInitJob(job, catalog); - break; - } - case OperationType.OP_FINISHING_ROLLUP: { - RollupJob job = (RollupJob) journal.getData(); - catalog.getRollupHandler().replayFinishing(job, catalog); - break; - } - case OperationType.OP_FINISH_ROLLUP: { - RollupJob job = (RollupJob) journal.getData(); - catalog.getRollupHandler().replayFinish(job, catalog); - break; - } - case OperationType.OP_CANCEL_ROLLUP: { - RollupJob job = (RollupJob) journal.getData(); - catalog.getRollupHandler().replayCancel(job, catalog); - break; - } case OperationType.OP_DROP_ROLLUP: { DropInfo info = (DropInfo) journal.getData(); catalog.getRollupHandler().replayDropRollup(info, catalog); @@ -309,32 +287,6 @@ public static void loadJournal(Catalog catalog, JournalEntity journal) { } break; } - case OperationType.OP_START_SCHEMA_CHANGE: { - SchemaChangeJob job = (SchemaChangeJob) journal.getData(); - LOG.info("Begin to unprotect create schema change job. db = " + job.getDbId() - + " table = " + job.getTableId()); - catalog.getSchemaChangeHandler().replayInitJob(job, catalog); - break; - } - case OperationType.OP_FINISHING_SCHEMA_CHANGE: { - SchemaChangeJob job = (SchemaChangeJob) journal.getData(); - LOG.info("Begin to unprotect replay finishing schema change job. db = " + job.getDbId() - + " table = " + job.getTableId()); - catalog.getSchemaChangeHandler().replayFinishing(job, catalog); - break; - } - case OperationType.OP_FINISH_SCHEMA_CHANGE: { - SchemaChangeJob job = (SchemaChangeJob) journal.getData(); - catalog.getSchemaChangeHandler().replayFinish(job, catalog); - break; - } - case OperationType.OP_CANCEL_SCHEMA_CHANGE: { - SchemaChangeJob job = (SchemaChangeJob) journal.getData(); - LOG.debug("Begin to unprotect cancel schema change. db = " + job.getDbId() - + " table = " + job.getTableId()); - catalog.getSchemaChangeHandler().replayCancel(job, catalog); - break; - } case OperationType.OP_FINISH_CONSISTENCY_CHECK: { ConsistencyCheckInfo info = (ConsistencyCheckInfo) journal.getData(); catalog.getConsistencyChecker().replayFinishConsistencyCheck(info, catalog); @@ -435,18 +387,6 @@ public static void loadJournal(Catalog catalog, JournalEntity journal) { Catalog.getCurrentSystemInfo().updateBackendState(be); break; } - case OperationType.OP_START_DECOMMISSION_BACKEND: { - DecommissionBackendJob job = (DecommissionBackendJob) journal.getData(); - LOG.debug("{}: {}", opCode, job.getTableId()); - catalog.getClusterHandler().replayInitJob(job, catalog); - break; - } - case OperationType.OP_FINISH_DECOMMISSION_BACKEND: { - DecommissionBackendJob job = (DecommissionBackendJob) journal.getData(); - LOG.debug("{}: {}", opCode, job.getTableId()); - catalog.getClusterHandler().replayFinish(job, catalog); - break; - } case OperationType.OP_ADD_FIRST_FRONTEND: case OperationType.OP_ADD_FRONTEND: { Frontend fe = (Frontend) journal.getData(); @@ -1064,22 +1004,6 @@ public void logLoadDone(LoadJob job) { logEdit(OperationType.OP_LOAD_DONE, job); } - public void logStartRollup(RollupJob rollupJob) { - logEdit(OperationType.OP_START_ROLLUP, rollupJob); - } - - public void logFinishingRollup(RollupJob rollupJob) { - logEdit(OperationType.OP_FINISHING_ROLLUP, rollupJob); - } - - public void logFinishRollup(RollupJob rollupJob) { - logEdit(OperationType.OP_FINISH_ROLLUP, rollupJob); - } - - public void logCancelRollup(RollupJob rollupJob) { - logEdit(OperationType.OP_CANCEL_ROLLUP, rollupJob); - } - public void logDropRollup(DropInfo info) { logEdit(OperationType.OP_DROP_ROLLUP, info); } @@ -1088,22 +1012,6 @@ public void logBatchDropRollup(BatchDropInfo batchDropInfo) { logEdit(OperationType.OP_BATCH_DROP_ROLLUP, batchDropInfo); } - public void logStartSchemaChange(SchemaChangeJob schemaChangeJob) { - logEdit(OperationType.OP_START_SCHEMA_CHANGE, schemaChangeJob); - } - - public void logFinishingSchemaChange(SchemaChangeJob schemaChangeJob) { - logEdit(OperationType.OP_FINISHING_SCHEMA_CHANGE, schemaChangeJob); - } - - public void logFinishSchemaChange(SchemaChangeJob schemaChangeJob) { - logEdit(OperationType.OP_FINISH_SCHEMA_CHANGE, schemaChangeJob); - } - - public void logCancelSchemaChange(SchemaChangeJob schemaChangeJob) { - logEdit(OperationType.OP_CANCEL_SCHEMA_CHANGE, schemaChangeJob); - } - public void logFinishConsistencyCheck(ConsistencyCheckInfo info) { logEdit(OperationType.OP_FINISH_CONSISTENCY_CHECK, info); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTest.java index a6724a06e2082e..116ae1fc5755b6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTest.java @@ -17,9 +17,7 @@ package org.apache.doris.catalog; -import org.apache.doris.alter.AlterJob; -import org.apache.doris.alter.AlterJob.JobType; -import org.apache.doris.alter.SchemaChangeJob; +import org.apache.doris.alter.AlterJobV2.JobType; import org.apache.doris.catalog.MaterializedIndex.IndexState; import org.apache.doris.cluster.Cluster; import org.apache.doris.common.FeConstants; @@ -189,52 +187,6 @@ public void testSaveLoadJob() throws Exception { Assert.assertTrue(job1.equals(job2)); dis.close(); - deleteDir(dir); - } - @Test - public void testSaveLoadSchemaChangeJob() throws Exception { - String dir = "testLoadSchemaChangeJob"; - mkdir(dir); - File file = new File(dir, "image"); - file.createNewFile(); - CountingDataOutputStream dos = new CountingDataOutputStream(new FileOutputStream(file)); - Catalog catalog = Catalog.getCurrentCatalog(); - MetaContext.get().setMetaVersion(FeConstants.meta_version); - Field field = catalog.getClass().getDeclaredField("load"); - field.setAccessible(true); - field.set(catalog, new Load()); - - Database db1 = new Database(10000L, "testCluster.db1"); - db1.setClusterName("testCluster"); - final Cluster cluster = new Cluster("testCluster", 10001L); - MaterializedIndex baseIndex = new MaterializedIndex(20000L, IndexState.NORMAL); - Partition partition = new Partition(2000L, "single", baseIndex, new RandomDistributionInfo(10)); - List baseSchema = new LinkedList(); - OlapTable table = new OlapTable(2L, "base", baseSchema, KeysType.AGG_KEYS, - new SinglePartitionInfo(), new RandomDistributionInfo(10)); - table.addPartition(partition); - db1.createTable(table); - - catalog.addCluster(cluster); - catalog.unprotectCreateDb(db1); - SchemaChangeJob job1 = new SchemaChangeJob(db1.getId(), table.getId(), null, table.getName(), -1); - - catalog.getSchemaChangeHandler().replayInitJob(job1, catalog); - long checksum1 = catalog.saveAlterJob(dos, 0, JobType.SCHEMA_CHANGE); - catalog.clear(); - catalog = null; - dos.close(); - - DataInputStream dis = new DataInputStream(new BufferedInputStream(new FileInputStream(file))); - catalog = Catalog.getCurrentCatalog(); - long checksum2 = catalog.loadAlterJob(dis, 0, JobType.SCHEMA_CHANGE); - Assert.assertEquals(checksum1, checksum2); - Map map = catalog.getSchemaChangeHandler().unprotectedGetAlterJobs(); - Assert.assertEquals(1, map.size()); - SchemaChangeJob job2 = (SchemaChangeJob) map.get(table.getId()); - Assert.assertTrue(job1.equals(job2)); - dis.close(); - deleteDir(dir); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/FakeEditLog.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/FakeEditLog.java index 06b7c0fbd2c818..514dfca5e7215e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/FakeEditLog.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/FakeEditLog.java @@ -19,8 +19,6 @@ import org.apache.doris.alter.AlterJobV2; import org.apache.doris.alter.BatchAlterJobPersistInfo; -import org.apache.doris.alter.RollupJob; -import org.apache.doris.alter.SchemaChangeJob; import org.apache.doris.cluster.Cluster; import org.apache.doris.persist.BatchRemoveTransactionsOperation; import org.apache.doris.persist.EditLog; @@ -69,28 +67,6 @@ public void logCreateCluster(Cluster cluster) { // do nothing } - @Mock - public void logStartRollup(RollupJob rollupJob) { - - } - - @Mock - public void logFinishingRollup(RollupJob rollupJob) { - - } - - @Mock - public void logCancelRollup(RollupJob rollupJob) { - } - - @Mock - public void logStartSchemaChange(SchemaChangeJob schemaChangeJob) { - } - - @Mock - public void logFinishingSchemaChange(SchemaChangeJob schemaChangeJob) { - } - @Mock public void logOpRoutineLoadJob(RoutineLoadOperation operation) { } From 974429b82632974ae4cbb3fcbe0ca4bee41641ad Mon Sep 17 00:00:00 2001 From: guoleiyi Date: Sat, 12 Feb 2022 22:35:05 +0800 Subject: [PATCH 2/7] fix code format error --- .../apache/doris/alter/DecommissionType.java | 17 +++++++++++++++++ .../org/apache/doris/alter/SystemHandler.java | 3 --- .../java/org/apache/doris/catalog/Catalog.java | 2 -- 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/DecommissionType.java b/fe/fe-core/src/main/java/org/apache/doris/alter/DecommissionType.java index 64437575c05c2e..41dbbf68ea7331 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/DecommissionType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/DecommissionType.java @@ -1,3 +1,20 @@ +// 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.alter; public enum DecommissionType { diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java index 3db19f621a0987..c8814a78a99266 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java @@ -38,14 +38,11 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; -import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.ha.FrontendNodeType; import org.apache.doris.system.Backend; import org.apache.doris.system.SystemInfoService; -import org.apache.doris.task.AgentTask; -import org.apache.doris.thrift.TTabletInfo; import com.google.common.base.Preconditions; import com.google.common.base.Strings; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index 02a543a7f7965f..77d0263172ec76 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -293,8 +293,6 @@ import com.sleepycat.je.rep.NetworkRestore; import com.sleepycat.je.rep.NetworkRestoreConfig; -import scala.reflect.internal.Trees.New; - import org.codehaus.jackson.map.ObjectMapper; public class Catalog { From 85eacdde3a7a5f22d3bdc3e1d60941b79434d7de Mon Sep 17 00:00:00 2001 From: guoleiyi Date: Sat, 12 Feb 2022 23:00:27 +0800 Subject: [PATCH 3/7] remove useless code --- .../doris/common/proc/BackendsProcDir.java | 2 +- .../apache/doris/journal/JournalEntity.java | 15 ---- .../journal/local/LocalJournalCursor.java | 13 --- .../org/apache/doris/master/MasterImpl.java | 84 ++----------------- .../org/apache/doris/persist/EditLog.java | 9 -- .../java/org/apache/doris/system/Backend.java | 2 +- 6 files changed, 10 insertions(+), 115 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java index 7f220c8db6484e..580c6ae8039ac5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java @@ -17,7 +17,7 @@ package org.apache.doris.common.proc; -import org.apache.doris.alter.DecommissionBackendJob.DecommissionType; +import org.apache.doris.alter.DecommissionType; import org.apache.doris.catalog.Catalog; import org.apache.doris.cluster.Cluster; import org.apache.doris.common.AnalysisException; diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java index 4fca787215f6f7..801e75d6c798a4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java +++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java @@ -17,7 +17,6 @@ package org.apache.doris.journal; -import org.apache.doris.alter.AlterJob; import org.apache.doris.alter.AlterJobV2; import org.apache.doris.alter.BatchAlterJobPersistInfo; import org.apache.doris.analysis.UserIdentity; @@ -226,20 +225,6 @@ public void readFields(DataInput in) throws IOException { isRead = true; break; } - case OperationType.OP_START_ROLLUP: - case OperationType.OP_FINISHING_ROLLUP: - case OperationType.OP_FINISHING_SCHEMA_CHANGE: - case OperationType.OP_FINISH_ROLLUP: - case OperationType.OP_CANCEL_ROLLUP: - case OperationType.OP_START_SCHEMA_CHANGE: - case OperationType.OP_FINISH_SCHEMA_CHANGE: - case OperationType.OP_CANCEL_SCHEMA_CHANGE: - case OperationType.OP_START_DECOMMISSION_BACKEND: - case OperationType.OP_FINISH_DECOMMISSION_BACKEND: { - data = AlterJob.read(in); - isRead = true; - break; - } case OperationType.OP_DROP_ROLLUP: { data = new DropInfo(); ((DropInfo) data).readFields(in); diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/local/LocalJournalCursor.java b/fe/fe-core/src/main/java/org/apache/doris/journal/local/LocalJournalCursor.java index 45e4af9c8d33f4..bbda622e5f0a1d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/journal/local/LocalJournalCursor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/journal/local/LocalJournalCursor.java @@ -17,7 +17,6 @@ package org.apache.doris.journal.local; -import org.apache.doris.alter.AlterJob; import org.apache.doris.catalog.Database; import org.apache.doris.common.io.Text; import org.apache.doris.ha.MasterInfo; @@ -277,18 +276,6 @@ private JournalEntity getJournalEntity(DataInputStream in, short opCode) throws ret.setData(recoverInfo); break; } - case OperationType.OP_START_ROLLUP: - case OperationType.OP_FINISH_ROLLUP: - case OperationType.OP_CANCEL_ROLLUP: - case OperationType.OP_START_SCHEMA_CHANGE: - case OperationType.OP_FINISH_SCHEMA_CHANGE: - case OperationType.OP_CANCEL_SCHEMA_CHANGE: - case OperationType.OP_START_DECOMMISSION_BACKEND: - case OperationType.OP_FINISH_DECOMMISSION_BACKEND: { - AlterJob alterJob = AlterJob.read(in); - ret.setData(alterJob); - break; - } case OperationType.OP_CLEAR_ROLLUP_INFO: { ReplicaPersistInfo info = ReplicaPersistInfo.read(in); ret.setData(info); 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 fdf971e2b70596..0122fd5ad0bcf4 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 @@ -17,12 +17,7 @@ package org.apache.doris.master; -import org.apache.doris.alter.AlterJob; import org.apache.doris.alter.AlterJobV2.JobType; -import org.apache.doris.alter.MaterializedViewHandler; -import org.apache.doris.alter.RollupJob; -import org.apache.doris.alter.SchemaChangeHandler; -import org.apache.doris.alter.SchemaChangeJob; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; @@ -47,12 +42,10 @@ import org.apache.doris.task.ClearAlterTask; import org.apache.doris.task.CloneTask; import org.apache.doris.task.CreateReplicaTask; -import org.apache.doris.task.CreateRollupTask; import org.apache.doris.task.DirMoveTask; import org.apache.doris.task.DownloadTask; import org.apache.doris.task.PublishVersionTask; import org.apache.doris.task.PushTask; -import org.apache.doris.task.SchemaChangeTask; import org.apache.doris.task.SnapshotTask; import org.apache.doris.task.UpdateTabletMetaInfoTask; import org.apache.doris.task.UploadTask; @@ -177,16 +170,8 @@ public TMasterResult finishTask(TFinishTaskRequest request) { finishDropReplica(task); break; case SCHEMA_CHANGE: - Preconditions.checkState(request.isSetReportVersion()); - checkHasTabletInfo(request); - finishTabletInfos = request.getFinishTabletInfos(); - finishSchemaChange(task, finishTabletInfos, request.getReportVersion()); - break; case ROLLUP: - checkHasTabletInfo(request); - finishTabletInfos = request.getFinishTabletInfos(); - finishRollup(task, finishTabletInfos); - break; + throw new RuntimeException("Schema change and rollup job is not used any more, use alter task instead"); case CLONE: finishClone(task, request); break; @@ -458,10 +443,11 @@ private Replica findRelatedReplica(OlapTable olapTable, Partition partition, } MaterializedIndex index = partition.getIndex(indexId); if (index == null) { - // this means the index is under rollup - MaterializedViewHandler materializedViewHandler = Catalog.getCurrentCatalog().getRollupHandler(); - AlterJob alterJob = materializedViewHandler.getAlterJob(olapTable.getId()); - if (alterJob == null && olapTable.getState() == OlapTableState.ROLLUP) { + // In alter job v2 case + // alter job is always == null, so that we could remove the condition + // if alter job is always null, then could not covert it to a rollup + // job, will throw exception, so just throw exception in this case + if (olapTable.getState() == OlapTableState.ROLLUP) { // this happens when: // a rollup job is finish and a delete job is the next first job (no load job before) // and delete task is first send to base tablet, so it will return 2 tablets info. @@ -472,14 +458,7 @@ private Replica findRelatedReplica(OlapTable olapTable, Partition partition, LOG.warn("Cannot find table[{}].", olapTable.getId()); return null; } - RollupJob rollupJob = (RollupJob) alterJob; - MaterializedIndex rollupIndex = rollupJob.getRollupIndex(partition.getId()); - - if (rollupIndex == null) { - LOG.warn("could not find index for tablet {}", tabletId); - return null; - } - index = rollupIndex; + throw new MetaNotFoundException("Could not find related replica"); } Tablet tablet = index.getTablet(tabletId); if (tablet == null) { @@ -671,38 +650,13 @@ private ReplicaPersistInfo updateReplicaInfo(OlapTable olapTable, Partition part LOG.warn("tablet[{}] may be dropped. push index[{}]", tabletId, pushIndexId); return null; } - - MaterializedViewHandler materializedViewHandler = Catalog.getCurrentCatalog().getRollupHandler(); - AlterJob alterJob = materializedViewHandler.getAlterJob(olapTable.getId()); - if (alterJob == null) { - // this happens when: - // a rollup job is finish and a delete job is the next first job (no load job before) - // and delete task is first send to base tablet, so it will return 2 tablets info. - // the second tablet is rollup tablet and it is no longer exist in alterJobs queue. - // just ignore the rollup tablet info. it will be handled in rollup tablet delete task report. - - // add log to observe - LOG.warn("Cannot find table[{}].", olapTable.getId()); - return null; - } - - ((RollupJob) alterJob).updateRollupReplicaInfo(partition.getId(), indexId, tabletId, backendId, - schemaHash, version, versionHash, rowCount, dataSize); - // replica info is saved in rollup job, not in load job return null; } int currentSchemaHash = olapTable.getSchemaHashByIndexId(pushIndexId); if (schemaHash != currentSchemaHash) { if (pushState == PartitionState.SCHEMA_CHANGE) { - SchemaChangeHandler schemaChangeHandler = Catalog.getCurrentCatalog().getSchemaChangeHandler(); - AlterJob alterJob = schemaChangeHandler.getAlterJob(olapTable.getId()); - if (alterJob != null && schemaHash != ((SchemaChangeJob) alterJob).getSchemaHashByIndexId(pushIndexId)) { - // this is a invalid tablet. - throw new MetaNotFoundException("tablet[" + tabletId - + "] schemaHash is not equal to index's switchSchemaHash. " - + ((SchemaChangeJob) alterJob).getSchemaHashByIndexId(pushIndexId) + " vs. " + schemaHash); - } + // Alter job is always null, so that not deal with it } else { // this should not happen. observe(cmy) throw new MetaNotFoundException("Diff tablet[" + tabletId + "] schemaHash. index[" + pushIndexId + "]: " @@ -736,28 +690,6 @@ private void finishDropReplica(AgentTask task) { AgentTaskQueue.removeTask(task.getBackendId(), TTaskType.DROP, task.getSignature()); } - private void finishSchemaChange(AgentTask task, List finishTabletInfos, long reportVersion) - throws MetaNotFoundException { - Preconditions.checkArgument(finishTabletInfos != null && !finishTabletInfos.isEmpty()); - Preconditions.checkArgument(finishTabletInfos.size() == 1); - - SchemaChangeTask schemaChangeTask = (SchemaChangeTask) task; - SchemaChangeHandler schemaChangeHandler = Catalog.getCurrentCatalog().getSchemaChangeHandler(); - schemaChangeHandler.handleFinishedReplica(schemaChangeTask, finishTabletInfos.get(0), reportVersion); - AgentTaskQueue.removeTask(task.getBackendId(), TTaskType.SCHEMA_CHANGE, task.getSignature()); - } - - private void finishRollup(AgentTask task, List finishTabletInfos) - throws MetaNotFoundException { - Preconditions.checkArgument(finishTabletInfos != null && !finishTabletInfos.isEmpty()); - Preconditions.checkArgument(finishTabletInfos.size() == 1); - - CreateRollupTask createRollupTask = (CreateRollupTask) task; - MaterializedViewHandler materializedViewHandler = Catalog.getCurrentCatalog().getRollupHandler(); - materializedViewHandler.handleFinishedReplica(createRollupTask, finishTabletInfos.get(0), -1L); - AgentTaskQueue.removeTask(task.getBackendId(), TTaskType.ROLLUP, task.getSignature()); - } - private void finishClone(AgentTask task, TFinishTaskRequest request) { CloneTask cloneTask = (CloneTask) task; if (cloneTask.getTaskVersion() == CloneTask.VERSION_2) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index e836007da6eddd..c0a591a99a6d7c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -19,7 +19,6 @@ import org.apache.doris.alter.AlterJobV2; import org.apache.doris.alter.BatchAlterJobPersistInfo; -import org.apache.doris.alter.DecommissionBackendJob; import org.apache.doris.analysis.UserIdentity; import org.apache.doris.backup.BackupJob; import org.apache.doris.backup.Repository; @@ -1104,14 +1103,6 @@ public void logDropRole(PrivInfo info) { logEdit(OperationType.OP_DROP_ROLE, info); } - public void logStartDecommissionBackend(DecommissionBackendJob job) { - logEdit(OperationType.OP_START_DECOMMISSION_BACKEND, job); - } - - public void logFinishDecommissionBackend(DecommissionBackendJob job) { - logEdit(OperationType.OP_FINISH_DECOMMISSION_BACKEND, job); - } - public void logDatabaseRename(DatabaseInfo databaseInfo) { logEdit(OperationType.OP_RENAME_DB, databaseInfo); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java b/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java index 3091a77e0aeb01..8b87f9256d9e11 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java @@ -17,7 +17,7 @@ package org.apache.doris.system; -import org.apache.doris.alter.DecommissionBackendJob.DecommissionType; +import org.apache.doris.alter.DecommissionType; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.DiskInfo; import org.apache.doris.catalog.DiskInfo.DiskState; From eb4e37779582ef1482c9cae02c176dbb19dbadad Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Sun, 13 Feb 2022 11:22:09 +0800 Subject: [PATCH 4/7] Apply suggestions from code review Co-authored-by: Mingyu Chen --- fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index 77d0263172ec76..e67ab69a2aa7ab 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -1803,7 +1803,7 @@ public long loadAlterJob(DataInputStream dis, long checksum) throws IOException long newChecksum = checksum; for (JobType type : JobType.values()) { if (type == JobType.DECOMMISSION_BACKEND) { - throw new IOException("There should be no DECOMMISSION_BACKEND jobs, not use this FE version"); + throw new IOException("There should be no DECOMMISSION_BACKEND jobs. Please downgrade FE to an older version and handle residual jobs"); } else { newChecksum = loadAlterJob(dis, newChecksum, type); } From 9a664d08e460436c96defa3ae4ae91c3e90a9884 Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Sun, 13 Feb 2022 11:22:34 +0800 Subject: [PATCH 5/7] Apply suggestions from code review Co-authored-by: Mingyu Chen --- fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index e67ab69a2aa7ab..df8efb7b73e28b 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -1820,7 +1820,7 @@ public long loadAlterJob(DataInputStream dis, long checksum, JobType type) throw long newChecksum = checksum ^ size; if (size > 0) { // There should be no old alter jobs, if exist throw exception, should not use this FE version - throw new IOException("There are [" + size + "] old alter jobs, should not happen"); + throw new IOException("There are [" + size + "] old alter jobs. Please downgrade FE to an older version and handle residual jobs"); } if (Catalog.getCurrentCatalogJournalVersion() >= 2) { From c86652f62d58d375504f351118c2c4da2399ee86 Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Sun, 13 Feb 2022 11:23:00 +0800 Subject: [PATCH 6/7] Apply suggestions from code review Co-authored-by: Mingyu Chen --- fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index df8efb7b73e28b..29e5dd734a4759 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -1828,7 +1828,7 @@ public long loadAlterJob(DataInputStream dis, long checksum, JobType type) throw size = dis.readInt(); newChecksum ^= size; if (size > 0) { - throw new IOException("There are [" + size + "] old finished or cancelled alter jobs, should not happen"); + throw new IOException("There are [" + size + "] old finished or cancelled alter jobs. Please downgrade FE to an older version and handle residual jobs"); } } From d0179c88bf7f95184527334500ece7d1d46a069f Mon Sep 17 00:00:00 2001 From: guoleiyi Date: Sun, 13 Feb 2022 11:26:01 +0800 Subject: [PATCH 7/7] fix bugs --- .../main/java/org/apache/doris/alter/SchemaChangeHandler.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index ad748864a4e139..61bbf6796ee732 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -1776,7 +1776,9 @@ public void cancel(CancelStmt stmt) throws DdlException { List schemaChangeJobV2List = getUnfinishedAlterJobV2ByTableId(olapTable.getId()); // current schemaChangeJob job doesn't support batch operation,so just need to get one job schemaChangeJobV2 = schemaChangeJobV2List.size() == 0 ? null : Iterables.getOnlyElement(schemaChangeJobV2List); - Preconditions.checkNotNull(schemaChangeJobV2, "Table[" + tableName + "] is not under SCHEMA_CHANGE."); + if (schemaChangeJobV2 == null) { + throw new DdlException("Table[" + tableName + "] is under schema change state but could not find related job"); + } } finally { olapTable.writeUnlock(); }