diff --git a/be/src/agent/agent_server.cpp b/be/src/agent/agent_server.cpp index 81ffcf57202808..9e23b982151f22 100644 --- a/be/src/agent/agent_server.cpp +++ b/be/src/agent/agent_server.cpp @@ -329,8 +329,8 @@ void AgentServer::submit_tasks( break; case TTaskType::ROLLUP: case TTaskType::SCHEMA_CHANGE: - case TTaskType::ALTER_TASK: - if (task.__isset.alter_tablet_req) { + case TTaskType::ALTER: + if (task.__isset.alter_tablet_req || task.__isset.alter_tablet_req_v2) { _alter_tablet_workers->submit_task(task); } else { status_code = TStatusCode::ANALYSIS_ERROR; diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index 5bdb75fd0caa31..a098b135e392ab 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -550,7 +550,7 @@ void* TaskWorkerPool::_alter_tablet_worker_thread_callback(void* arg_this) { switch (task_type) { case TTaskType::SCHEMA_CHANGE: case TTaskType::ROLLUP: - case TTaskType::ALTER_TASK: + case TTaskType::ALTER: worker_pool_this->_alter_tablet(worker_pool_this, agent_task_req, signatrue, @@ -588,8 +588,8 @@ void TaskWorkerPool::_alter_tablet( case TTaskType::SCHEMA_CHANGE: process_name = "schema change"; break; - case TTaskType::ALTER_TASK: - process_name = "alter table"; + case TTaskType::ALTER: + process_name = "alter"; break; default: std::string task_name; @@ -607,7 +607,7 @@ void TaskWorkerPool::_alter_tablet( TSchemaHash new_schema_hash = 0; if (status == DORIS_SUCCESS) { OLAPStatus sc_status = OLAP_SUCCESS; - if (task_type == TTaskType::ALTER_TASK) { + if (task_type == TTaskType::ALTER) { new_tablet_id = agent_task_req.alter_tablet_req_v2.new_tablet_id; new_schema_hash = agent_task_req.alter_tablet_req_v2.new_schema_hash; EngineAlterTabletTask engine_task(agent_task_req.alter_tablet_req_v2, signature, task_type, &error_msgs, process_name); diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp index d3f63478c4e967..4407fc359b82fa 100644 --- a/be/src/olap/delta_writer.cpp +++ b/be/src/olap/delta_writer.cpp @@ -142,19 +142,9 @@ OLAPStatus DeltaWriter::init() { return OLAP_ERR_ROWSET_WRITER_INIT; } - const std::vector& slots = _req.tuple_desc->slots(); - const TabletSchema& schema = _tablet->tablet_schema(); - for (size_t col_id = 0; col_id < schema.num_columns(); ++col_id) { - const TabletColumn& column = schema.column(col_id); - for (size_t i = 0; i < slots.size(); ++i) { - if (slots[i]->col_name() == column.name()) { - _col_ids.push_back(i); - } - } - } _tablet_schema = &(_tablet->tablet_schema()); _schema = new Schema(*_tablet_schema); - _mem_table = new MemTable(_schema, _tablet_schema, &_col_ids, + _mem_table = new MemTable(_schema, _tablet_schema, _req.slots, _req.tuple_desc, _tablet->keys_type()); _is_init = true; return OLAP_SUCCESS; @@ -173,7 +163,7 @@ OLAPStatus DeltaWriter::write(Tuple* tuple) { RETURN_NOT_OK(_mem_table->flush(_rowset_writer)); SAFE_DELETE(_mem_table); - _mem_table = new MemTable(_schema, _tablet_schema, &_col_ids, + _mem_table = new MemTable(_schema, _tablet_schema, _req.slots, _req.tuple_desc, _tablet->keys_type()); } return OLAP_SUCCESS; diff --git a/be/src/olap/delta_writer.h b/be/src/olap/delta_writer.h index c909147b075100..730759094b0523 100644 --- a/be/src/olap/delta_writer.h +++ b/be/src/olap/delta_writer.h @@ -47,6 +47,8 @@ struct WriteRequest { PUniqueId load_id; bool need_gen_rollup; TupleDescriptor* tuple_desc; + // slots are in order of tablet's schema + std::vector* slots; }; class DeltaWriter { @@ -76,7 +78,6 @@ class DeltaWriter { MemTable* _mem_table; Schema* _schema; const TabletSchema* _tablet_schema; - std::vector _col_ids; bool _delta_written_success; }; diff --git a/be/src/olap/memtable.cpp b/be/src/olap/memtable.cpp index 33085fe53c3817..18e1141da202a2 100644 --- a/be/src/olap/memtable.cpp +++ b/be/src/olap/memtable.cpp @@ -27,12 +27,12 @@ namespace doris { MemTable::MemTable(Schema* schema, const TabletSchema* tablet_schema, - std::vector* col_ids, TupleDescriptor* tuple_desc, + std::vector* slot_descs, TupleDescriptor* tuple_desc, KeysType keys_type) : _schema(schema), _tablet_schema(tablet_schema), _tuple_desc(tuple_desc), - _col_ids(col_ids), + _slot_descs(slot_descs), _keys_type(keys_type), _row_comparator(_schema) { _schema_size = _schema->schema_size(); @@ -58,11 +58,10 @@ size_t MemTable::memory_usage() { } void MemTable::insert(Tuple* tuple) { - const std::vector& slots = _tuple_desc->slots(); ContiguousRow row(_schema, _tuple_buf); - for (size_t i = 0; i < _col_ids->size(); ++i) { + for (size_t i = 0; i < _slot_descs->size(); ++i) { auto cell = row.cell(i); - const SlotDescriptor* slot = slots[(*_col_ids)[i]]; + const SlotDescriptor* slot = (*_slot_descs)[i]; if (tuple->is_null(slot->null_indicator_offset())) { cell.set_null(); diff --git a/be/src/olap/memtable.h b/be/src/olap/memtable.h index 40861a47da9996..95090475bcdc22 100644 --- a/be/src/olap/memtable.h +++ b/be/src/olap/memtable.h @@ -32,7 +32,7 @@ class RowCursor; class MemTable { public: MemTable(Schema* schema, const TabletSchema* tablet_schema, - std::vector* col_ids, TupleDescriptor* tuple_desc, + std::vector* slot_descs, TupleDescriptor* tuple_desc, KeysType keys_type); ~MemTable(); size_t memory_usage(); @@ -43,7 +43,8 @@ class MemTable { Schema* _schema; const TabletSchema* _tablet_schema; TupleDescriptor* _tuple_desc; - std::vector* _col_ids; + // the slot in _slot_descs are in order of tablet's schema + std::vector* _slot_descs; KeysType _keys_type; struct RowCursorComparator { diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index 340e74940f934e..018cacfb2d782e 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -1329,9 +1329,12 @@ OLAPStatus SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletRe if (res != OLAP_SUCCESS) { break; } - res = _validate_alter_result(new_tablet, request); } while(0); + // _validate_alter_result should be outside the above while loop. + // to avoid requiring the header lock twice. + res = _validate_alter_result(new_tablet, request); + // if failed convert history data, then just remove the new tablet if (res != OLAP_SUCCESS) { LOG(WARNING) << "failed to alter tablet. base_tablet=" << base_tablet->full_name() @@ -2143,7 +2146,7 @@ OLAPStatus SchemaChangeHandler::_validate_alter_result(TabletSharedPtr new_table Version max_continuous_version = {-1, 0}; VersionHash max_continuous_version_hash = 0; new_tablet->max_continuous_version_from_begining(&max_continuous_version, &max_continuous_version_hash); - LOG(INFO) << "find max continuous version " + LOG(INFO) << "find max continuous version of tablet=" << new_tablet->full_name() << ", start_version=" << max_continuous_version.first << ", end_version=" << max_continuous_version.second << ", version_hash=" << max_continuous_version_hash; diff --git a/be/src/olap/task/engine_alter_tablet_task.cpp b/be/src/olap/task/engine_alter_tablet_task.cpp index 54fb1eadd911a9..5a127a14a4062d 100644 --- a/be/src/olap/task/engine_alter_tablet_task.cpp +++ b/be/src/olap/task/engine_alter_tablet_task.cpp @@ -33,11 +33,6 @@ EngineAlterTabletTask::EngineAlterTabletTask(const TAlterTabletReqV2& request, _process_name(process_name) { } OLAPStatus EngineAlterTabletTask::execute() { - LOG(INFO) << "begin to create new alter tablet. base_tablet_id=" << _alter_tablet_req.base_tablet_id - << ", base_schema_hash=" << _alter_tablet_req.base_schema_hash - << ", new_tablet_id=" << _alter_tablet_req.new_tablet_id - << ", new_schema_hash=" << _alter_tablet_req.new_schema_hash; - DorisMetrics::create_rollup_requests_total.increment(1); SchemaChangeHandler handler; diff --git a/be/src/runtime/tablet_writer_mgr.cpp b/be/src/runtime/tablet_writer_mgr.cpp index 1d1c282a57ffeb..ca4a083272fcc7 100644 --- a/be/src/runtime/tablet_writer_mgr.cpp +++ b/be/src/runtime/tablet_writer_mgr.cpp @@ -205,16 +205,16 @@ Status TabletsChannel::close(int sender_id, bool* finished, } Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& params) { - std::vector* columns = nullptr; + std::vector* index_slots = nullptr; int32_t schema_hash = 0; for (auto& index : _schema->indexes()) { if (index->index_id == _index_id) { - columns = &index->slots; + index_slots = &index->slots; schema_hash = index->schema_hash; break; } } - if (columns == nullptr) { + if (index_slots == nullptr) { std::stringstream ss; ss << "unknown index id, key=" << _key; return Status::InternalError(ss.str()); @@ -229,6 +229,7 @@ Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& params) request.load_id = params.id(); request.need_gen_rollup = params.need_gen_rollup(); request.tuple_desc = _tuple_desc; + request.slots = index_slots; DeltaWriter* writer = nullptr; auto st = DeltaWriter::open(&request, &writer); diff --git a/docs/documentation/cn/administrator-guide/operation/tablet-meta-tool.md b/docs/documentation/cn/administrator-guide/operation/tablet-meta-tool.md index 87a6230b5f7603..5846d3b40ba02b 100644 --- a/docs/documentation/cn/administrator-guide/operation/tablet-meta-tool.md +++ b/docs/documentation/cn/administrator-guide/operation/tablet-meta-tool.md @@ -72,7 +72,7 @@ api: 命令: ``` -./lib/meta_tool --operation=delete_header --root_path=/path/to/root_path --tablet_id=xxx --schema_hash=xxx` +./lib/meta_tool --operation=delete_header --root_path=/path/to/root_path --tablet_id=xxx --schema_hash=xxx ``` ### 展示 pb 格式的 TabletMeta diff --git a/fe/src/main/cup/sql_parser.cup b/fe/src/main/cup/sql_parser.cup index 5dbcfe99d66977..b6a39dde46adbd 100644 --- a/fe/src/main/cup/sql_parser.cup +++ b/fe/src/main/cup/sql_parser.cup @@ -1853,11 +1853,15 @@ show_param ::= :} | KW_TABLET INTEGER_LITERAL:tabletId {: - RESULT = new ShowTabletStmt(null, tabletId); + RESULT = new ShowTabletStmt(null, tabletId, null); :} | KW_TABLET KW_FROM table_name:dbTblName {: - RESULT = new ShowTabletStmt(dbTblName, -1L); + RESULT = new ShowTabletStmt(dbTblName, -1L, null); + :} + | KW_TABLET KW_FROM table_name:dbTblName KW_ROLLUP ident:rollup + {: + RESULT = new ShowTabletStmt(dbTblName, -1L, rollup); :} | KW_PROPERTY opt_user:user opt_wild_where {: diff --git a/fe/src/main/java/org/apache/doris/alter/Alter.java b/fe/src/main/java/org/apache/doris/alter/Alter.java index 4d0828a4e4eb8c..90a8ce72255c23 100644 --- a/fe/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/src/main/java/org/apache/doris/alter/Alter.java @@ -45,6 +45,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.UserException; import com.google.common.base.Preconditions; @@ -73,7 +74,7 @@ public void start() { clusterHandler.start(); } - public void processAlterTable(AlterTableStmt stmt) throws DdlException { + public void processAlterTable(AlterTableStmt stmt) throws UserException { TableName dbTableName = stmt.getTbl(); String dbName = dbTableName.getDb(); final String clusterName = stmt.getClusterName(); @@ -171,13 +172,8 @@ public void processAlterTable(AlterTableStmt stmt) throws DdlException { throw new DdlException("table with empty parition cannot do schema change. [" + tableName + "]"); } - if (olapTable.getState() == OlapTableState.SCHEMA_CHANGE - || olapTable.getState() == OlapTableState.RESTORE) { - throw new DdlException("Table[" + table.getName() + "]'s state[" + olapTable.getState() - + "] does not allow doing ALTER ops"); - // here we pass NORMAL and ROLLUP - // NORMAL: ok to do any alter ops - // ROLLUP: we allow user DROP a rollup index when it's under ROLLUP + if (olapTable.getState() != OlapTableState.NORMAL) { + throw new DdlException("Table[" + table.getName() + "]'s state is not NORMAL. Do not allow doing ALTER ops"); } if (hasSchemaChange || hasModifyProp || hasRollup) { diff --git a/fe/src/main/java/org/apache/doris/alter/AlterHandler.java b/fe/src/main/java/org/apache/doris/alter/AlterHandler.java index 06849f41e6ef4b..748a5022d81bda 100644 --- a/fe/src/main/java/org/apache/doris/alter/AlterHandler.java +++ b/fe/src/main/java/org/apache/doris/alter/AlterHandler.java @@ -22,15 +22,25 @@ import org.apache.doris.analysis.CancelStmt; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.MaterializedIndex; import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.Tablet; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.UserException; import org.apache.doris.common.util.Daemon; import org.apache.doris.common.util.TimeUtils; +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; + import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -39,16 +49,21 @@ 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 Daemon { 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(); + /* * lock to perform atomic operations. * eg. @@ -70,19 +85,41 @@ public AlterHandler(String name) { super(name, 10000); } + protected void addAlterJobV2(AlterJobV2 alterJob) { + this.alterJobsV2.put(alterJob.getJobId(), alterJob); + LOG.info("add {} job {}", alterJob.getType(), alterJob.getJobId()); + } + + public AlterJobV2 getAlterJobV2(long tblId) { + for (AlterJobV2 alterJob : alterJobsV2.values()) { + if (alterJob.getTableId() == tblId && alterJob.getJobState() == AlterJobV2.JobState.RUNNING) { + return alterJob; + } + } + return null; + } + + public Map getAlterJobsV2() { + return this.alterJobsV2; + } + + @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) { @@ -121,24 +158,29 @@ public int getAlterJobNum(JobState state, long dbId) { return jobNum; } + @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()) { @@ -154,6 +196,7 @@ public void removeDbAlterJob(long dbId) { * 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(); @@ -282,13 +325,14 @@ public void start() { * entry function. handle alter ops */ public abstract void process(List alterClauses, String clusterName, Database db, OlapTable olapTable) - throws DdlException; + throws UserException; /* * cancel alter ops */ public abstract void cancel(CancelStmt stmt) throws DdlException; + @Deprecated public Integer getAlterJobNumByState(JobState state) { int jobNum = 0; for (AlterJob alterJob : alterJobs.values()) { @@ -298,4 +342,95 @@ public Integer getAlterJobNumByState(JobState state) { } 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. + * So here we should modify the replica's version. + * We assume that the specified version is X. + * Case 1: + * After alter table process starts, there is no new load job being submitted. So the new replica + * should be with version (1-0). So we just modify the replica's version to partition's visible version, which is X. + * Case 2: + * After alter table process starts, there are some load job being processed. + * Case 2.1: + * Only one new load job, and it failed on this replica. so the replica's last failed version should be X + 1 + * and version is still 1. We should modify the replica's version to (last failed version - 1) + * Case 2.2 + * There are new load jobs after alter task, and at least one of them is succeed on this replica. + * So the replica's version should be larger than X. So we don't need to modify the replica version + * because its already looks like normal. + */ + public void handleFinishAlterTask(AlterReplicaTask task) throws MetaNotFoundException { + Database db = Catalog.getCurrentCatalog().getDb(task.getDbId()); + if (db == null) { + throw new MetaNotFoundException("database " + task.getDbId() + " does not exist"); + } + + db.writeLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(task.getTableId()); + if (tbl == null) { + throw new MetaNotFoundException("tbl " + task.getTableId() + " does not exist"); + } + Partition partition = tbl.getPartition(task.getPartitionId()); + if (partition == null) { + throw new MetaNotFoundException("partition " + task.getPartitionId() + " does not exist"); + } + MaterializedIndex index = partition.getIndex(task.getIndexId()); + if (index == null) { + throw new MetaNotFoundException("index " + task.getIndexId() + " does not exist"); + } + Tablet tablet = index.getTablet(task.getTabletId()); + Preconditions.checkNotNull(tablet, task.getTabletId()); + Replica replica = tablet.getReplicaById(task.getNewReplicaId()); + if (replica == null) { + throw new MetaNotFoundException("replica " + task.getNewReplicaId() + " does not exist"); + } + + LOG.info("before handle alter task replica: {}, task version: {}-{}", + replica, task.getVersion(), task.getVersionHash()); + boolean versionChanged = false; + if (replica.getVersion() > task.getVersion()) { + // Case 2.2, do nothing + } else { + if (replica.getLastFailedVersion() > task.getVersion()) { + // Case 2.1 + replica.updateVersionInfo(task.getVersion(), task.getVersionHash(), replica.getDataSize(), replica.getRowCount()); + versionChanged = true; + } else { + // Case 1 + Preconditions.checkState(replica.getLastFailedVersion() == -1, replica.getLastFailedVersion()); + replica.updateVersionInfo(task.getVersion(), task.getVersionHash(), replica.getDataSize(), replica.getRowCount()); + versionChanged = true; + } + } + + if (versionChanged) { + ReplicaPersistInfo info = ReplicaPersistInfo.createForClone(task.getDbId(), task.getTableId(), + task.getPartitionId(), task.getIndexId(), task.getTabletId(), task.getBackendId(), + replica.getId(), replica.getVersion(), replica.getVersionHash(), -1, + replica.getDataSize(), replica.getRowCount(), + replica.getLastFailedVersion(), replica.getLastFailedVersionHash(), + replica.getLastSuccessVersion(), replica.getLastSuccessVersionHash()); + Catalog.getInstance().getEditLog().logUpdateReplica(info); + } + + LOG.info("after handle alter task replica: {}", replica); + } finally { + db.writeUnlock(); + } + } + + // replay the alter job v2 + public void replayAlterJobV2(AlterJobV2 alterJob) { + AlterJobV2 existingJob = alterJobsV2.get(alterJob.getJobId()); + if (existingJob == null) { + // This is the first time to replay the alter job, so just using the replayed alterJob to call replay(); + alterJob.replay(alterJob); + alterJobsV2.put(alterJob.getJobId(), alterJob); + } else { + existingJob.replay(alterJob); + } + } } diff --git a/fe/src/main/java/org/apache/doris/alter/AlterJobV2.java b/fe/src/main/java/org/apache/doris/alter/AlterJobV2.java new file mode 100644 index 00000000000000..8cc6369b7ff543 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/alter/AlterJobV2.java @@ -0,0 +1,224 @@ +// 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.common.io.Text; +import org.apache.doris.common.io.Writable; + +import com.google.common.base.Preconditions; + +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.List; + +/* + * Author: Chenmingyu + * Date: Jul 8, 2019 + */ + +/* + * Version 2 of AlterJob, for replacing the old version of AlterJob. + * This base class of RollupJob and SchemaChangeJob + */ +public class AlterJobV2 implements Writable { + private static final Logger LOG = LogManager.getLogger(AlterJobV2.class); + + public enum JobState { + PENDING, // Job is created + WAITING_TXN, // New replicas are created and Shadow catalog object is visible for incoming txns, + // waiting for previous txns to be finished + RUNNING, // alter tasks are sent to BE, and waiting for them finished. + FINISHED, // job is done + CANCELLED; // job is cancelled(failed or be cancelled by user) + + public boolean isFinalState() { + return this == JobState.FINISHED || this == JobState.CANCELLED; + } + } + + public enum JobType { + ROLLUP, SCHEMA_CHANGE + } + + protected JobType type; + protected long jobId; + protected JobState jobState; + + protected long dbId; + protected long tableId; + protected String tableName; + + protected String errMsg = ""; + protected long createTimeMs = -1; + protected long finishedTimeMs = -1; + protected long timeoutMs = -1; + + public AlterJobV2(long jobId, JobType jobType, long dbId, long tableId, String tableName, long timeoutMs) { + this.jobId = jobId; + this.type = jobType; + this.dbId = dbId; + this.tableId = tableId; + this.tableName = tableName; + this.timeoutMs = timeoutMs; + + this.createTimeMs = System.currentTimeMillis(); + this.jobState = JobState.PENDING; + } + + protected AlterJobV2(JobType type) { + this.type = type; + } + + public long getJobId() { + return jobId; + } + + public JobState getJobState() { + return jobState; + } + + public JobType getType() { + return type; + } + + public long getDbId() { + return dbId; + } + + public long getTableId() { + return tableId; + } + + public String getTableName() { + return tableName; + } + + private boolean isTimeout() { + return System.currentTimeMillis() - createTimeMs > timeoutMs; + } + + public boolean isDone() { + return jobState.isFinalState(); + } + + /* + * The keyword 'synchronized' only protects 2 methods: + * run() and cancel() + * Only these 2 methods can be visited by different thread(internal working thread and user connection thread) + * So using 'synchronized' to make sure only one thread can run the job at one time. + * + * lock order: + * synchronized + * db lock + */ + public synchronized void run() { + if (isTimeout()) { + cancel("Timeout"); + return; + } + + switch (jobState) { + case PENDING: + runPendingJob(); + break; + case WAITING_TXN: + runWaitingTxnJob(); + break; + case RUNNING: + runRunningJob(); + break; + default: + break; + } + } + + protected void runPendingJob() { + throw new NotImplementedException(); + } + + protected void runWaitingTxnJob() { + throw new NotImplementedException(); + } + + protected void runRunningJob() { + throw new NotImplementedException(); + } + + public synchronized boolean cancel(String errMsg) { + throw new NotImplementedException(); + } + + protected void getInfo(List> infos) { + throw new NotImplementedException(); + } + + public void replay(AlterJobV2 replayedJob) { + throw new NotImplementedException(); + } + + public static AlterJobV2 read(DataInput in) throws IOException { + JobType type = JobType.valueOf(Text.readString(in)); + switch (type) { + case ROLLUP: + return RollupJobV2.read(in); + case SCHEMA_CHANGE: + return SchemaChangeJobV2.read(in); + default: + Preconditions.checkState(false); + return null; + } + } + + @Override + public void write(DataOutput out) throws IOException { + Text.writeString(out, type.name()); + Text.writeString(out, jobState.name()); + + out.writeLong(jobId); + out.writeLong(dbId); + out.writeLong(tableId); + Text.writeString(out, tableName); + + Text.writeString(out, errMsg); + out.writeLong(createTimeMs); + out.writeLong(finishedTimeMs); + out.writeLong(timeoutMs); + } + + @Override + public void readFields(DataInput in) throws IOException { + // read common members as write in AlterJobV2.write(). + // except 'type' member, which is read in AlterJobV2.read() + jobState = JobState.valueOf(Text.readString(in)); + + jobId = in.readLong(); + dbId = in.readLong(); + tableId = in.readLong(); + tableName = Text.readString(in); + + errMsg = Text.readString(in); + createTimeMs = in.readLong(); + finishedTimeMs = in.readLong(); + timeoutMs = in.readLong(); + } +} diff --git a/fe/src/main/java/org/apache/doris/alter/RollupHandler.java b/fe/src/main/java/org/apache/doris/alter/RollupHandler.java index ca4900da06daec..136e95db1ae188 100644 --- a/fe/src/main/java/org/apache/doris/alter/RollupHandler.java +++ b/fe/src/main/java/org/apache/doris/alter/RollupHandler.java @@ -33,30 +33,22 @@ 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.AnalysisException; import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.util.ListComparator; -import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.common.util.Util; +import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.persist.DropInfo; import org.apache.doris.persist.EditLog; import org.apache.doris.qe.ConnectContext; -import org.apache.doris.task.AgentBatchTask; -import org.apache.doris.task.AgentTaskExecutor; -import org.apache.doris.task.DropReplicaTask; -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 com.google.common.base.Preconditions; import com.google.common.base.Strings; @@ -68,11 +60,15 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; +/* + * RollupHandler is responsible for ADD/DROP rollup. + */ public class RollupHandler extends AlterHandler { private static final Logger LOG = LogManager.getLogger(RollupHandler.class); @@ -80,17 +76,23 @@ public RollupHandler() { super("rollup"); } - private void processAddRollup(AddRollupClause alterClause, Database db, OlapTable olapTable, boolean isRestore) + /* + * Handle the Add Rollup request. + * 3 main steps: + * 1. Validate the request. + * 2. Create RollupJob with rollup index + * All replicas of the rollup index will be created in meta and added to TabletInvertedIndex + * 3. Set table's state to ROLLUP. + */ + private void processAddRollup(AddRollupClause alterClause, Database db, OlapTable olapTable) throws DdlException { - if (!isRestore) { - // table is under rollup or has a finishing alter job - if (olapTable.getState() == OlapTableState.ROLLUP || this.hasUnfinishedAlterJob(olapTable.getId())) { - throw new DdlException("Table[" + olapTable.getName() + "]'s is under ROLLUP"); - } - // up to here, table's state can only be NORMAL - Preconditions.checkState(olapTable.getState() == OlapTableState.NORMAL, olapTable.getState().name()); + // table is under rollup or has a finishing alter job + if (olapTable.getState() == OlapTableState.ROLLUP || this.hasUnfinishedAlterJob(olapTable.getId())) { + throw new DdlException("Table[" + olapTable.getName() + "]'s is under ROLLUP"); } + // up to here, table's state can only be NORMAL + Preconditions.checkState(olapTable.getState() == OlapTableState.NORMAL, olapTable.getState().name()); String rollupIndexName = alterClause.getRollupName(); String baseIndexName = alterClause.getBaseRollupName(); @@ -120,7 +122,7 @@ private void processAddRollup(AddRollupClause alterClause, Database db, OlapTabl Preconditions.checkState(baseIndex.getState() == IndexState.NORMAL, baseIndex.getState().name()); } - // 3 check if rollup columns are valid + // 3. check if rollup columns are valid // a. all columns should exist in base rollup schema // b. value after key // c. if rollup contains REPLACE column, all keys on base index should be included. @@ -184,7 +186,7 @@ private void processAddRollup(AddRollupClause alterClause, Database db, OlapTabl if (alterClause.getDupKeys() == null || alterClause.getDupKeys().isEmpty()) { // user does not specify duplicate key for rollup, // use base table's duplicate key. - // so we should check if rollup column contains all base table's duplicate key. + // so we should check if rollup columns contains all base table's duplicate key. List baseIdxCols = olapTable.getSchemaByIndexId(baseIndexId); Set baseIdxKeyColNames = Sets.newHashSet(); for (Column baseCol : baseIdxCols) { @@ -210,6 +212,7 @@ private void processAddRollup(AddRollupClause alterClause, Database db, OlapTabl } } + // check (a)(b) for (String columnName : rollupColumnNames) { Column oneColumn = olapTable.getColumn(columnName); if (oneColumn == null) { @@ -230,10 +233,10 @@ private void processAddRollup(AddRollupClause alterClause, Database db, OlapTabl throw new DdlException("No key column is found"); } } else { - // rollup have different dup keys with base table + // user specify the duplicate keys for rollup index List dupKeys = alterClause.getDupKeys(); if (dupKeys.size() > rollupColumnNames.size()) { - throw new DdlException("Duplicate key should be the prefix of rollup columns. Exceeded"); + throw new DdlException("Num of duplicate keys should less than or equal to num of rollup columns."); } for (int i = 0; i < rollupColumnNames.size(); i++) { @@ -242,7 +245,7 @@ private void processAddRollup(AddRollupClause alterClause, Database db, OlapTabl if (i < dupKeys.size()) { String dupKeyName = dupKeys.get(i); if (!rollupColName.equalsIgnoreCase(dupKeyName)) { - throw new DdlException("Duplicate key should be the prefix of rollup columns"); + throw new DdlException("Duplicate keys should be the prefix of rollup columns"); } isKey = true; } @@ -270,82 +273,46 @@ private void processAddRollup(AddRollupClause alterClause, Database db, OlapTabl } } - // 4. do create things - // 4.1 get storage type. default is COLUMN - - TKeysType rollupKeysType; - if (keysType == KeysType.DUP_KEYS) { - rollupKeysType = TKeysType.DUP_KEYS; - } else if (keysType == KeysType.UNIQUE_KEYS) { - rollupKeysType = TKeysType.UNIQUE_KEYS; - } else { - rollupKeysType = TKeysType.AGG_KEYS; - } + // assign rollup index's key type, same as base index's + KeysType rollupKeysType = keysType; - Map properties = alterClause.getProperties(); - TStorageType rollupStorageType = null; - try { - rollupStorageType = PropertyAnalyzer.analyzeStorageType(properties); - } catch (AnalysisException e) { - throw new DdlException(e.getMessage()); - } - // check storage type if has null column - boolean hasNullColumn = false; - for (Column column : rollupSchema) { - if (column.isAllowNull()) { - hasNullColumn = true; - break; - } - } - if (hasNullColumn && rollupStorageType != TStorageType.COLUMN) { - throw new DdlException("Only column rollup support null columns"); - } - - // 4.2 get rollup schema hash - int schemaVersion = 0; - try { - schemaVersion = PropertyAnalyzer.analyzeSchemaVersion(properties); - } catch (AnalysisException e) { - throw new DdlException(e.getMessage()); - } - int rollupSchemaHash = Util.schemaHash(schemaVersion, rollupSchema, olapTable.getCopiedBfColumns(), + // get rollup schema hash + int rollupSchemaHash = Util.schemaHash(0 /* init schema version */, rollupSchema, olapTable.getCopiedBfColumns(), olapTable.getBfFpp()); - // 4.3 get short key column count + // get short key column count + Map properties = alterClause.getProperties(); short rollupShortKeyColumnCount = Catalog.calcShortKeyColumnCount(rollupSchema, properties); + + // get timeout + long timeoutMs = alterClause.getTimeoutSecond() * 1000; - // 4.4 get user resource info - TResourceInfo resourceInfo = null; - if (ConnectContext.get() != null) { - resourceInfo = ConnectContext.get().toResourceCtx(); - } - - // 4.5 create rollup job + // 4. create rollup job long dbId = db.getId(); long tableId = olapTable.getId(); int baseSchemaHash = olapTable.getSchemaHashByIndexId(baseIndexId); - Catalog catalog = Catalog.getInstance(); + Catalog catalog = Catalog.getCurrentCatalog(); + long jobId = catalog.getNextId(); long rollupIndexId = catalog.getNextId(); - long transactionId = Catalog.getCurrentGlobalTransactionMgr().getTransactionIDGenerator().getNextTransactionId(); - RollupJob rollupJob = new RollupJob(dbId, tableId, baseIndexId, rollupIndexId, - baseIndexName, rollupIndexName, rollupSchema, - baseSchemaHash, rollupSchemaHash, rollupStorageType, - rollupShortKeyColumnCount, resourceInfo, rollupKeysType, transactionId); - + RollupJobV2 rollupJob = new RollupJobV2(jobId, dbId, tableId, olapTable.getName(), timeoutMs, + baseIndexId, rollupIndexId, baseIndexName, rollupIndexName, + rollupSchema, baseSchemaHash, rollupSchemaHash, + rollupKeysType, rollupShortKeyColumnCount); + + /* + * create all rollup indexes. and set state. + * After setting, Tables' state will be RO + */ for (Partition partition : olapTable.getPartitions()) { long partitionId = partition.getId(); TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty(partitionId).getStorageMedium(); - MaterializedIndex rollupIndex = new MaterializedIndex(rollupIndexId, IndexState.ROLLUP); - if (isRestore) { - rollupIndex.setState(IndexState.NORMAL); - } + // index state is SHADOW + MaterializedIndex rollupIndex = new MaterializedIndex(rollupIndexId, IndexState.SHADOW); MaterializedIndex baseIndex = partition.getIndex(baseIndexId); - TabletMeta rollupTabletMeta = new TabletMeta(dbId, tableId, partitionId, rollupIndexId, - rollupSchemaHash, medium); - short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId()); + TabletMeta rollupTabletMeta = new TabletMeta(dbId, tableId, partitionId, rollupIndexId, rollupSchemaHash, medium); for (Tablet baseTablet : baseIndex.getTablets()) { long baseTabletId = baseTablet.getId(); long rollupTabletId = catalog.getNextId(); @@ -353,10 +320,9 @@ private void processAddRollup(AddRollupClause alterClause, Database db, OlapTabl Tablet newTablet = new Tablet(rollupTabletId); rollupIndex.addTablet(newTablet, rollupTabletMeta); - rollupJob.setTabletIdMap(partitionId, rollupTabletId, baseTabletId); + rollupJob.addTabletIdMap(partitionId, rollupTabletId, baseTabletId); List baseReplicas = baseTablet.getReplicas(); - int replicaNum = 0; for (Replica baseReplica : baseReplicas) { long rollupReplicaId = catalog.getNextId(); long backendId = baseReplica.getBackendId(); @@ -367,68 +333,35 @@ private void processAddRollup(AddRollupClause alterClause, Database db, OlapTabl continue; } Preconditions.checkState(baseReplica.getState() == ReplicaState.NORMAL); - ++replicaNum; - // the new replica's init version is -1 until finished history rollup - Replica rollupReplica = new Replica(rollupReplicaId, backendId, rollupSchemaHash, - ReplicaState.ROLLUP); - // new replica's last failed version should be set to the partition's next version - 1, - // if all go well, the last failed version will be overwritten when rollup task finished and update - // replica version info. - // If not set, there is no other way to know that this replica has failed version. - rollupReplica.updateVersionInfo(rollupReplica.getVersion(), rollupReplica.getVersionHash(), - partition.getCommittedVersion(), partition.getCommittedVersionHash(), - rollupReplica.getLastSuccessVersion(), rollupReplica.getLastSuccessVersionHash()); - if (isRestore) { - rollupReplica.setState(ReplicaState.NORMAL); - } - // yiguolei: the rollup tablet's replica num maybe less than base tablet's replica num + // replica's init state is ALTER, so that tablet report process will ignore its report + Replica rollupReplica = new Replica(rollupReplicaId, backendId, ReplicaState.ALTER, + Partition.PARTITION_INIT_VERSION, Partition.PARTITION_INIT_VERSION_HASH, + rollupSchemaHash); newTablet.addReplica(rollupReplica); } // end for baseReplica - - if (replicaNum < replicationNum / 2 + 1) { - String errMsg = "Tablet[" + baseTabletId + "] does not have enough replica. [" - + replicaNum + "/" + replicationNum + "]"; - LOG.warn(errMsg); - throw new DdlException(errMsg); - } } // end for baseTablets - if (isRestore) { - partition.createRollupIndex(rollupIndex); - } else { - rollupJob.addRollupIndex(partitionId, rollupIndex); - } + rollupJob.addRollupIndex(partitionId, rollupIndex); - LOG.debug("create rollup index[{}] based on index[{}] in partition[{}], restore: {}", - rollupIndexId, baseIndexId, partitionId, isRestore); + LOG.debug("create rollup index {} based on index {} in partition {}", + rollupIndexId, baseIndexId, partitionId); } // end for partitions - if (isRestore) { - olapTable.setIndexSchemaInfo(rollupIndexId, rollupIndexName, rollupSchema, 0, - rollupSchemaHash, rollupShortKeyColumnCount); - olapTable.setStorageTypeToIndex(rollupIndexId, rollupStorageType); - } else { - // update partition and table state - for (Partition partition : olapTable.getPartitions()) { - partition.setState(PartitionState.ROLLUP); - } - olapTable.setState(OlapTableState.ROLLUP); + // update table state + olapTable.setState(OlapTableState.ROLLUP); - addAlterJob(rollupJob); + addAlterJobV2(rollupJob); - // log rollup operation - EditLog editLog = catalog.getEditLog(); - editLog.logStartRollup(rollupJob); - LOG.debug("sync start create rollup index[{}] in table[{}]", rollupIndexId, tableId); - } + // log rollup operation + catalog.getEditLog().logAlterJob(rollupJob); + LOG.info("finished to create rollup job: {}", rollupJob.getJobId()); } public void processDropRollup(DropRollupClause alterClause, Database db, OlapTable olapTable) throws DdlException { - // make sure we got db write lock here - // up to here, table's state can be NORMAL or ROLLUP - Preconditions.checkState(olapTable.getState() == OlapTableState.NORMAL - || olapTable.getState() == OlapTableState.ROLLUP, olapTable.getState().name()); + // make sure we got db write lock here. + // up to here, table's state can only be NORMAL. + Preconditions.checkState(olapTable.getState() == OlapTableState.NORMAL, olapTable.getState().name()); String rollupIndexName = alterClause.getRollupName(); if (rollupIndexName.equals(olapTable.getName())) { @@ -438,74 +371,37 @@ public void processDropRollup(DropRollupClause alterClause, Database db, OlapTab long dbId = db.getId(); long tableId = olapTable.getId(); if (!olapTable.hasMaterializedIndex(rollupIndexName)) { - // when rollup job is unfinished, rollup index is not added to the table - AlterJob alterJob = getAlterJob(tableId); - if (alterJob == null || !((RollupJob) alterJob).getRollupIndexName().equals(rollupIndexName)) { - throw new DdlException("Rollup index[" + rollupIndexName + "] does not exist in table[" - + olapTable.getName() + "]"); - } - - // cancel rollup job - cancelInternal(alterJob, olapTable, "rollup index is dropped"); - return; - } - - // 1. check if any rollup job is based on this index - AlterJob alterJob = null; - if ((alterJob = checkIfAnyRollupBasedOn(tableId, rollupIndexName)) != null) { - throw new DdlException("Rollup index[" + ((RollupJob) alterJob).getRollupIndexName() - + "] is doing rollup based on this index[" + rollupIndexName + "] and not finished yet."); + throw new DdlException("Rollup index[" + rollupIndexName + "] does not exist in table[" + + olapTable.getName() + "]"); } - // if the index is a during rollup and in finishing state, then it could not be dropped - // because the finishing state could not be roll back, it is very difficult - alterJob = getAlterJob(tableId); - if (alterJob != null && ((RollupJob) alterJob).getRollupIndexName().equals(rollupIndexName) - && alterJob.getState() == JobState.FINISHING) { - throw new DdlException("Rollup index[" + rollupIndexName + "] in table[" - + olapTable.getName() + "] is in finishing state, waiting it to finish"); - } - - // drop rollup for each partition long rollupIndexId = olapTable.getIndexIdByName(rollupIndexName); int rollupSchemaHash = olapTable.getSchemaHashByIndexId(rollupIndexId); Preconditions.checkState(rollupSchemaHash != -1); - Preconditions.checkState(olapTable.getState() == OlapTableState.NORMAL); + // drop rollup for each partition. + // also remove tablets from inverted index. TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); - AgentBatchTask batchTask = new AgentBatchTask(); for (Partition partition : olapTable.getPartitions()) { MaterializedIndex rollupIndex = partition.getIndex(rollupIndexId); Preconditions.checkNotNull(rollupIndex); - // 2. delete rollup index + // delete rollup index partition.deleteRollupIndex(rollupIndexId); - // 3. send DropReplicaTask + // remove tablets from inverted index for (Tablet tablet : rollupIndex.getTablets()) { long tabletId = tablet.getId(); - List replicas = tablet.getReplicas(); - for (Replica replica : replicas) { - long backendId = replica.getBackendId(); - DropReplicaTask dropTask = new DropReplicaTask(backendId, tabletId, rollupSchemaHash); - batchTask.addTask(dropTask); - } // end for replicas - - // remove from inverted index invertedIndex.deleteTablet(tabletId); - } // end for tablets - } // end for partitions + } + } olapTable.deleteIndexInfo(rollupIndexName); - AgentTaskExecutor.submit(batchTask); - // 5. log drop rollup operation + // log drop rollup operation EditLog editLog = Catalog.getInstance().getEditLog(); DropInfo dropInfo = new DropInfo(dbId, tableId, rollupIndexId); editLog.logDropRollup(dropInfo); - LOG.debug("log drop rollup index[{}] finished in table[{}]", dropInfo.getIndexId(), - dropInfo.getTableId()); - LOG.info("finished drop rollup index[{}] in table[{}]", rollupIndexName, olapTable.getName()); } @@ -534,37 +430,30 @@ public void replayDropRollup(DropInfo dropInfo, Catalog catalog) { } finally { db.writeUnlock(); } + LOG.info("replay drop rollup {}", dropInfo.getIndexId()); } - public void removeReplicaRelatedTask(long tableId, long partitionId, long indexId, long tabletId, long backendId) { - // make sure to get db writeLock - AlterJob alterJob = checkIfAnyRollupBasedOn(tableId, indexId); - if (alterJob != null) { - alterJob.removeReplicaRelatedTask(partitionId, tabletId, -1L, backendId); - } - } - - // this is for handle delete replica op - private AlterJob checkIfAnyRollupBasedOn(long tableId, long baseIndexId) { - AlterJob alterJob = this.alterJobs.get(tableId); - if (alterJob != null && ((RollupJob) alterJob).getBaseIndexId() == baseIndexId) { - return alterJob; - } - return null; + @Override + protected void runOneCycle() { + super.runOneCycle(); + runOldAlterJob(); + runAlterJobV2(); } - // this is for drop rollup op - private AlterJob checkIfAnyRollupBasedOn(long tableId, String baseIndexName) { - AlterJob alterJob = this.alterJobs.get(tableId); - if (alterJob != null && ((RollupJob) alterJob).getBaseIndexName().equals(baseIndexName)) { - return alterJob; + private void runAlterJobV2() { + Iterator> iter = alterJobsV2.entrySet().iterator(); + while (iter.hasNext()) { + Map.Entry entry = iter.next(); + AlterJobV2 alterJob = entry.getValue(); + if (alterJob.isDone()) { + continue; + } + alterJob.run(); } - return null; } - @Override - protected void runOneCycle() { - super.runOneCycle(); + @Deprecated + private void runOldAlterJob() { List cancelledJobs = Lists.newArrayList(); List finishedJobs = Lists.newArrayList(); @@ -674,8 +563,36 @@ protected void runOneCycle() { @Override public List> getAlterJobInfosByDb(Database db) { List> rollupJobInfos = new LinkedList>(); - List jobs = Lists.newArrayList(); + getOldAlterJobInfos(db, rollupJobInfos); + getAlterJobV2Infos(db, rollupJobInfos); + + // sort by + // "JobId", "TableName", "CreateTime", "FinishedTime", "BaseIndexName", "RollupIndexName" + ListComparator> comparator = new ListComparator>(0, 1, 2, 3, 4, 5); + Collections.sort(rollupJobInfos, comparator); + + return rollupJobInfos; + } + + private void getAlterJobV2Infos(Database db, List> rollupJobInfos) { + ConnectContext ctx = ConnectContext.get(); + for (AlterJobV2 alterJob : alterJobsV2.values()) { + if (alterJob.getDbId() != db.getId()) { + continue; + } + if (ctx != null) { + if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ctx, db.getFullName(), alterJob.getTableName(), PrivPredicate.ALTER)) { + continue; + } + } + alterJob.getInfo(rollupJobInfos); + } + } + + @Deprecated + private void getOldAlterJobInfos(Database db, List> rollupJobInfos) { + List jobs = Lists.newArrayList(); // lock to perform atomically lock(); try { @@ -707,26 +624,14 @@ public List> getAlterJobInfosByDb(Database db) { } finally { db.readUnlock(); } - - // sort by - // "JobId", "TableName", "CreateTime", "FinishedTime", "BaseIndexName", "RollupIndexName" - ListComparator> comparator = new ListComparator>(0, 1, 2, 3, 4, 5); - Collections.sort(rollupJobInfos, comparator); - - return rollupJobInfos; } @Override public void process(List alterClauses, String clusterName, Database db, OlapTable olapTable) throws DdlException { - process(alterClauses, db, olapTable, false); - } - - public void process(List alterClauses, Database db, OlapTable olapTable, boolean isRestore) - throws DdlException { for (AlterClause alterClause : alterClauses) { if (alterClause instanceof AddRollupClause) { - processAddRollup((AddRollupClause) alterClause, db, olapTable, isRestore); + processAddRollup((AddRollupClause) alterClause, db, olapTable); } else if (alterClause instanceof DropRollupClause) { processDropRollup((DropRollupClause) alterClause, db, olapTable); } else { @@ -750,6 +655,7 @@ public void cancel(CancelStmt stmt) throws DdlException { } AlterJob rollupJob = null; + AlterJobV2 rollupJobV2 = null; db.writeLock(); try { Table table = db.getTable(tableName); @@ -765,18 +671,33 @@ public void cancel(CancelStmt stmt) throws DdlException { + "Use 'ALTER TABLE DROP ROLLUP' if you want to."); } - rollupJob = getAlterJob(olapTable.getId()); - Preconditions.checkNotNull(rollupJob); - - if (rollupJob.getState() == JobState.FINISHED || rollupJob.getState() == JobState.CANCELLED) { - throw new DdlException("job is already " + rollupJob.getState().name() + ", can not cancel it"); + // find from new alter jobs first + rollupJobV2 = getAlterJobV2(olapTable.getId()); + if (rollupJobV2 == null) { + rollupJob = getAlterJob(olapTable.getId()); + Preconditions.checkNotNull(rollupJob, olapTable.getId()); + 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"); } - - rollupJob.cancel(olapTable, "user cancelled"); } finally { db.writeUnlock(); } - jobDone(rollupJob); + // alter job v2's cancel must be called outside the database lock + if (rollupJobV2 != null) { + if (!rollupJobV2.cancel("user cancelled")) { + throw new DdlException("Job can not be cancelled. State: " + rollupJobV2.getJobState()); + } + return; + } + + // handle old alter job + if (rollupJob != null && rollupJob.getState() == JobState.CANCELLED) { + jobDone(rollupJob); + } } } diff --git a/fe/src/main/java/org/apache/doris/alter/RollupJob.java b/fe/src/main/java/org/apache/doris/alter/RollupJob.java index bd39a219e739ce..af181babc6ec3c 100644 --- a/fe/src/main/java/org/apache/doris/alter/RollupJob.java +++ b/fe/src/main/java/org/apache/doris/alter/RollupJob.java @@ -31,6 +31,7 @@ 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.FeMetaVersion; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.io.Text; @@ -986,7 +987,6 @@ public void getJobInfo(List> jobInfos, OlapTable tbl) { // transaction id jobInfo.add(transactionId); - // job state jobInfo.add(state.name()); @@ -1002,6 +1002,7 @@ public void getJobInfo(List> jobInfos, OlapTable tbl) { } else { jobInfo.add("N/A"); } + jobInfo.add(Config.alter_table_timeout_second); jobInfos.add(jobInfo); } diff --git a/fe/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/src/main/java/org/apache/doris/alter/RollupJobV2.java new file mode 100644 index 00000000000000..f7471a6634f506 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/alter/RollupJobV2.java @@ -0,0 +1,741 @@ +// 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.Replica; +import org.apache.doris.catalog.Replica.ReplicaState; +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.MarkedCountDownLatch; +import org.apache.doris.common.io.Text; +import org.apache.doris.common.util.TimeUtils; +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.AlterReplicaTask; +import org.apache.doris.task.CreateReplicaTask; +import org.apache.doris.thrift.TStorageMedium; +import org.apache.doris.thrift.TStorageType; +import org.apache.doris.thrift.TTaskType; + +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +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; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.TimeUnit; + +/* + * Author: Chenmingyu + * Date: Jul 8, 2019 + */ + +/* + * Version 2 of RollupJob. + * This is for replacing the old RollupJob + * https://github.com/apache/incubator-doris/issues/1429 + */ +public class RollupJobV2 extends AlterJobV2 { + private static final Logger LOG = LogManager.getLogger(RollupJobV2.class); + + // partition id -> (rollup tablet id -> base tablet id) + private Map> partitionIdToBaseRollupTabletIdMap = Maps.newHashMap(); + private Map partitionIdToRollupIndex = Maps.newHashMap(); + + // rollup and base schema info + private long baseIndexId; + private long rollupIndexId; + private String baseIndexName; + private String rollupIndexName; + + private List rollupSchema = Lists.newArrayList(); + private int baseSchemaHash; + private int rollupSchemaHash; + + private KeysType rollupKeysType; + private short rollupShortKeyColumnCount; + + // The rollup job will wait all transactions before this txn id finished, then send the rollup tasks. + protected long watershedTxnId = -1; + + // save all create rollup tasks + private AgentBatchTask rollupBatchTask = new AgentBatchTask(); + + public RollupJobV2(long jobId, long dbId, long tableId, String tableName, long timeoutMs, + long baseIndexId, long rollupIndexId, String baseIndexName, String rollupIndexName, + List rollupSchema, int baseSchemaHash, int rollupSchemaHash, + KeysType rollupKeysType, short rollupShortKeyColumnCount) { + super(jobId, JobType.ROLLUP, dbId, tableId, tableName, timeoutMs); + + this.baseIndexId = baseIndexId; + this.rollupIndexId = rollupIndexId; + this.baseIndexName = baseIndexName; + this.rollupIndexName = rollupIndexName; + + this.rollupSchema = rollupSchema; + this.baseSchemaHash = baseSchemaHash; + this.rollupSchemaHash = rollupSchemaHash; + this.rollupKeysType = rollupKeysType; + this.rollupShortKeyColumnCount = rollupShortKeyColumnCount; + } + + private RollupJobV2() { + super(JobType.ROLLUP); + } + + public void addTabletIdMap(long partitionId, long rollupTabletId, long baseTabletId) { + Map tabletIdMap = partitionIdToBaseRollupTabletIdMap.get(partitionId); + if (tabletIdMap == null) { + tabletIdMap = Maps.newHashMap(); + partitionIdToBaseRollupTabletIdMap.put(partitionId, tabletIdMap); + } + tabletIdMap.put(rollupTabletId, baseTabletId); + } + + public void addRollupIndex(long partitionId, MaterializedIndex rollupIndex) { + this.partitionIdToRollupIndex.put(partitionId, rollupIndex); + } + + /* + * runPendingJob(): + * 1. Create all rollup replicas and wait them finished. + * 2. After creating done, add this shadow rollup index to catalog, user can not see this + * rollup, but internal load process will generate data for this rollup index. + * 3. Get a new transaction id, then set job's state to WAITING_TXN + */ + @Override + protected void runPendingJob() { + Preconditions.checkState(jobState == JobState.PENDING, jobState); + + LOG.info("begin to send create rollup replica tasks. job: {}", jobId); + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + cancel("Databasee " + dbId + " does not exist"); + return; + } + + // 1. create rollup replicas + AgentBatchTask batchTask = new AgentBatchTask(); + // count total replica num + int totalReplicaNum = 0; + for (MaterializedIndex rollupIdx : partitionIdToRollupIndex.values()) { + for (Tablet tablet : rollupIdx.getTablets()) { + totalReplicaNum += tablet.getReplicas().size(); + } + } + MarkedCountDownLatch countDownLatch = new MarkedCountDownLatch(totalReplicaNum); + db.readLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + cancel("Table " + tableId + " does not exist"); + return; + } + Preconditions.checkState(tbl.getState() == OlapTableState.ROLLUP); + + for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { + long partitionId = entry.getKey(); + Partition partition = tbl.getPartition(partitionId); + if (partition == null) { + continue; + } + TStorageMedium storageMedium = tbl.getPartitionInfo().getDataProperty(partitionId).getStorageMedium(); + 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(); + Preconditions.checkNotNull(tabletIdMap.get(rollupTabletId)); // baseTabletId + countDownLatch.addMark(backendId, rollupTabletId); + // create replica with version 1. + // version will be updated by following load process, or when rollup task finished. + CreateReplicaTask createReplicaTask = new CreateReplicaTask( + backendId, dbId, tableId, partitionId, rollupIndexId, rollupTabletId, + rollupShortKeyColumnCount, rollupSchemaHash, + Partition.PARTITION_INIT_VERSION, Partition.PARTITION_INIT_VERSION_HASH, + rollupKeysType, TStorageType.COLUMN, storageMedium, + rollupSchema, tbl.getCopiedBfColumns(), tbl.getBfFpp(), countDownLatch); + createReplicaTask.setBaseTablet(tabletIdMap.get(rollupTabletId), baseSchemaHash); + + batchTask.addTask(createReplicaTask); + } // end for rollupReplicas + } // end for rollupTablets + } + } finally { + db.readUnlock(); + } + + // send all tasks and wait them finished + AgentTaskQueue.addBatchTask(batchTask); + AgentTaskExecutor.submit(batchTask); + long timeout = Math.min(Config.tablet_create_timeout_second * 1000L * totalReplicaNum, + Config.max_create_table_timeout_second * 1000L); + boolean ok = false; + try { + ok = countDownLatch.await(timeout, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + LOG.warn("InterruptedException: ", e); + ok = false; + } + + if (!ok || !countDownLatch.getStatus().ok()) { + // create rollup replicas failed. just cancel the job + // clear tasks and show the failed replicas to user + AgentTaskQueue.removeBatchTask(batchTask, TTaskType.CREATE); + String errMsg = null; + if (!countDownLatch.getStatus().ok()) { + errMsg = countDownLatch.getStatus().getErrorMsg(); + } else { + List> unfinishedMarks = countDownLatch.getLeftMarks(); + // only show at most 3 results + List> subList = unfinishedMarks.subList(0, Math.min(unfinishedMarks.size(), 3)); + errMsg = "Error replicas:" + Joiner.on(", ").join(subList); + } + LOG.warn("failed to create rollup replicas for job: {}, {}", jobId, errMsg); + cancel("Create rollup replicas failed. Error: " + errMsg); + return; + } + + // create all rollup replicas success. + // add rollup index to catalog + db.writeLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + cancel("Table " + tableId + " does not exist"); + return; + } + Preconditions.checkState(tbl.getState() == OlapTableState.ROLLUP); + addRollupIndexToCatalog(tbl); + } finally { + db.writeUnlock(); + } + + this.watershedTxnId = Catalog.getCurrentGlobalTransactionMgr().getTransactionIDGenerator().getNextTransactionId(); + this.jobState = JobState.WAITING_TXN; + + // write edit log + Catalog.getCurrentCatalog().getEditLog().logAlterJob(this); + LOG.info("transfer rollup job {} state to {}, watershed txn id: {}", jobId, this.jobState, watershedTxnId); + } + + private void addRollupIndexToCatalog(OlapTable tbl) { + for (Partition partition : tbl.getPartitions()) { + long partitionId = partition.getId(); + MaterializedIndex rollupIndex = this.partitionIdToRollupIndex.get(partitionId); + Preconditions.checkNotNull(rollupIndex); + Preconditions.checkState(rollupIndex.getState() == IndexState.SHADOW, rollupIndex.getState()); + partition.createRollupIndex(rollupIndex); + } + + tbl.setIndexSchemaInfo(rollupIndexId, rollupIndexName, rollupSchema, 0 /* init schema version */, + rollupSchemaHash, rollupShortKeyColumnCount); + tbl.setStorageTypeToIndex(rollupIndexId, TStorageType.COLUMN); + } + + /* + * runWaitingTxnJob(): + * 1. Wait the transactions before the watershedTxnId to be finished. + * 2. If all previous transactions finished, send create rollup tasks to BE. + * 3. Change job state to RUNNING. + */ + @Override + protected void runWaitingTxnJob() { + Preconditions.checkState(jobState == JobState.WAITING_TXN, jobState); + + if (!isPreviousLoadFinished()) { + LOG.info("wait transactions before {} to be finished, rollup job: {}", watershedTxnId, jobId); + return; + } + + LOG.info("previous transactions are all finished, begin to send rollup tasks. job: {}", jobId); + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + cancel("Databasee " + dbId + " does not exist"); + return; + } + + db.readLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + cancel("Table " + tableId + " does not exist"); + return; + } + Preconditions.checkState(tbl.getState() == OlapTableState.ROLLUP); + for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { + long partitionId = entry.getKey(); + Partition partition = tbl.getPartition(partitionId); + Preconditions.checkNotNull(partition, partitionId); + + // the rollup task will transform the data before visible version(included). + long visibleVersion = partition.getVisibleVersion(); + long visibleVersionHash = partition.getVisibleVersionHash(); + + MaterializedIndex rollupIndex = entry.getValue(); + Map tabletIdMap = this.partitionIdToBaseRollupTabletIdMap.get(partitionId); + for (Tablet rollupTablet : rollupIndex.getTablets()) { + long rollupTabletId = rollupTablet.getId(); + long baseTabletId = tabletIdMap.get(rollupTabletId); + + List rollupReplicas = rollupTablet.getReplicas(); + for (Replica rollupReplica : rollupReplicas) { + AlterReplicaTask rollupTask = new AlterReplicaTask( + rollupReplica.getBackendId(), dbId, tableId, partitionId, + rollupIndexId, baseIndexId, + rollupTabletId, baseTabletId, rollupReplica.getId(), + rollupSchemaHash, baseSchemaHash, + visibleVersion, visibleVersionHash, jobId, JobType.ROLLUP); + rollupBatchTask.addTask(rollupTask); + } + } + } + } finally { + db.readUnlock(); + } + + AgentTaskQueue.addBatchTask(rollupBatchTask); + AgentTaskExecutor.submit(rollupBatchTask); + this.jobState = JobState.RUNNING; + + // DO NOT write edit log here, tasks will be send again if FE restart or master changed. + LOG.info("transfer rollup job {} state to {}", jobId, this.jobState); + } + + /* + * runRunningJob() + * 1. Wait all create rollup tasks to be finished. + * 2. Check the integrity of the newly created rollup index. + * 3. Set rollup index's state to NORMAL to let it visible to query. + * 4. Set job'state as FINISHED. + */ + @Override + protected void runRunningJob() { + Preconditions.checkState(jobState == JobState.RUNNING, jobState); + if (!rollupBatchTask.isFinished()) { + LOG.info("rollup tasks not finished. job: {}", jobId); + return; + } + + /* + * all tasks are finished. check the integrity. + * we just check whether all rollup replicas are healthy. + */ + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + cancel("Databasee " + dbId + " does not exist"); + return; + } + + db.writeLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + cancel("Table " + tableId + " does not exist"); + return; + } + Preconditions.checkState(tbl.getState() == OlapTableState.ROLLUP); + for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { + long partitionId = entry.getKey(); + Partition partition = tbl.getPartition(partitionId); + if (partition == null) { + continue; + } + + long visiableVersion = partition.getVisibleVersion(); + long visiableVersionHash = partition.getVisibleVersionHash(); + short expectReplicationNum = tbl.getPartitionInfo().getReplicationNum(partition.getId()); + + MaterializedIndex rollupIndex = entry.getValue(); + for (Tablet rollupTablet : rollupIndex.getTablets()) { + List replicas = rollupTablet.getReplicas(); + int healthyReplicaNum = 0; + for (Replica replica : replicas) { + if (replica.getLastFailedVersion() < 0 + && replica.checkVersionCatchUp(visiableVersion, visiableVersionHash, false)) { + healthyReplicaNum++; + } + } + + if (healthyReplicaNum < expectReplicationNum / 2 + 1) { + LOG.warn("rollup tablet {} has few healthy replicas: {}, rollup job: {}", + rollupTablet.getId(), replicas, jobId); + cancel("rollup tablet " + rollupTablet.getId() + " has few healthy replicas"); + return; + } + } // end for tablets + } // end for partitions + + onFinished(tbl); + } finally { + db.writeUnlock(); + } + + this.jobState = JobState.FINISHED; + this.finishedTimeMs = System.currentTimeMillis(); + + Catalog.getCurrentCatalog().getEditLog().logAlterJob(this); + LOG.info("rollup job finished: {}", jobId); + } + + private void onFinished(OlapTable tbl) { + for (Partition partition : tbl.getPartitions()) { + MaterializedIndex rollupIndex = partition.getIndex(rollupIndexId); + Preconditions.checkNotNull(rollupIndex, rollupIndexId); + for (Tablet tablet : rollupIndex.getTablets()) { + for (Replica replica : tablet.getReplicas()) { + replica.setState(ReplicaState.NORMAL); + } + } + partition.visualiseShadowIndex(rollupIndexId, false); + } + tbl.setState(OlapTableState.NORMAL); + } + + /* + * cancel() can be called any time any place. + * We need to clean any possible residual of this job. + */ + @Override + public synchronized boolean cancel(String errMsg) { + if (jobState.isFinalState()) { + return false; + } + + cancelInternal(); + + jobState = JobState.CANCELLED; + this.errMsg = errMsg; + this.finishedTimeMs = System.currentTimeMillis(); + LOG.info("cancel {} job {}, err: {}", this.type, jobId, errMsg); + Catalog.getCurrentCatalog().getEditLog().logAlterJob(this); + return true; + } + + private void cancelInternal() { + // clear tasks if has + AgentTaskQueue.removeBatchTask(rollupBatchTask, TTaskType.ALTER); + // remove all rollup indexes, and set state to NORMAL + TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db != null) { + db.writeLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl != null) { + for (Long partitionId : partitionIdToRollupIndex.keySet()) { + MaterializedIndex rollupIndex = partitionIdToRollupIndex.get(partitionId); + for (Tablet rollupTablet : rollupIndex.getTablets()) { + invertedIndex.deleteTablet(rollupTablet.getId()); + } + Partition partition = tbl.getPartition(partitionId); + partition.deleteRollupIndex(rollupIndexId); + } + tbl.deleteIndexInfo(rollupIndexName); + tbl.setState(OlapTableState.NORMAL); + } + } finally { + db.writeUnlock(); + } + } + } + + // Check whether transactions of the given database which txnId is less than 'watershedTxnId' are finished. + protected boolean isPreviousLoadFinished() { + return Catalog.getCurrentGlobalTransactionMgr().isPreviousTransactionsFinished(watershedTxnId, dbId); + } + + public static RollupJobV2 read(DataInput in) throws IOException { + RollupJobV2 rollupJob = new RollupJobV2(); + rollupJob.readFields(in); + return rollupJob; + } + + @Override + public void write(DataOutput out) throws IOException { + super.write(out); + + 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.writeLong(baseIndexId); + out.writeLong(rollupIndexId); + Text.writeString(out, baseIndexName); + Text.writeString(out, rollupIndexName); + + // rollup schema + out.writeInt(rollupSchema.size()); + for (Column column : rollupSchema) { + column.write(out); + } + out.writeInt(baseSchemaHash); + out.writeInt(rollupSchemaHash); + + Text.writeString(out, rollupKeysType.name()); + out.writeShort(rollupShortKeyColumnCount); + + out.writeLong(watershedTxnId); + } + + @Override + public void readFields(DataInput in) throws IOException { + super.readFields(in); + + int size = in.readInt(); + for (int i = 0; i < size; i++) { + long partitionId = in.readLong(); + int size2 = in.readInt(); + Map tabletIdMap = partitionIdToBaseRollupTabletIdMap.get(partitionId); + if (tabletIdMap == null) { + tabletIdMap = Maps.newHashMap(); + partitionIdToBaseRollupTabletIdMap.put(partitionId, tabletIdMap); + } + for (int j = 0; j < size2; j++) { + long rollupTabletId = in.readLong(); + long baseTabletId = in.readLong(); + tabletIdMap.put(rollupTabletId, baseTabletId); + } + + partitionIdToRollupIndex.put(partitionId, MaterializedIndex.read(in)); + } + + baseIndexId = in.readLong(); + rollupIndexId = in.readLong(); + baseIndexName = Text.readString(in); + rollupIndexName = Text.readString(in); + + size = in.readInt(); + for (int i = 0; i < size; i++) { + Column column = Column.read(in); + rollupSchema.add(column); + } + baseSchemaHash = in.readInt(); + rollupSchemaHash = in.readInt(); + + rollupKeysType = KeysType.valueOf(Text.readString(in)); + rollupShortKeyColumnCount = in.readShort(); + + watershedTxnId = in.readLong(); + } + + /* + * Replay job in PENDING state. + * Should replay all changes before this job's state transfer to PENDING. + * These changes should be same as changes in RollupHander.processAddRollup() + */ + private void replayPending(RollupJobV2 replayedJob) { + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + // database may be dropped before replaying this log. just return + return; + } + + db.writeLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + // table may be dropped before replaying this log. just return + return; + } + + // add all rollup replicas to tablet inverted index + TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); + for (Long partitionId : partitionIdToRollupIndex.keySet()) { + MaterializedIndex rollupIndex = partitionIdToRollupIndex.get(partitionId); + TStorageMedium medium = tbl.getPartitionInfo().getDataProperty(partitionId).getStorageMedium(); + TabletMeta rollupTabletMeta = new TabletMeta(dbId, tableId, partitionId, rollupIndexId, + rollupSchemaHash, medium); + + for (Tablet rollupTablet : rollupIndex.getTablets()) { + invertedIndex.addTablet(rollupTablet.getId(), rollupTabletMeta); + for (Replica rollupReplica : rollupTablet.getReplicas()) { + invertedIndex.addReplica(rollupTablet.getId(), rollupReplica); + } + } + } + tbl.setState(OlapTableState.ROLLUP); + } finally { + db.writeUnlock(); + } + + this.jobState = JobState.WAITING_TXN; + this.watershedTxnId = replayedJob.watershedTxnId; + + LOG.info("replay pending rollup job: {}", jobId); + } + + /* + * Replay job in WAITING_TXN state. + * Should replay all changes in runPendingJob() + */ + private void replayWaitingTxn(RollupJobV2 replayedJob) { + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + // database may be dropped before replaying this log. just return + return; + } + + db.writeLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + // table may be dropped before replaying this log. just return + return; + } + addRollupIndexToCatalog(tbl); + } finally { + db.writeUnlock(); + } + + this.jobState = JobState.RUNNING; + this.watershedTxnId = replayedJob.watershedTxnId; + + LOG.info("replay waiting txn rollup job: {}", jobId); + } + + /* + * Replay job in FINISHED state. + * Should replay all changes in runRuningJob() + */ + private void replayFinished(RollupJobV2 replayedJob) { + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db != null) { + db.writeLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl != null) { + Preconditions.checkState(tbl.getState() == OlapTableState.ROLLUP); + onFinished(tbl); + } + } finally { + db.writeUnlock(); + } + } + + this.jobState = JobState.FINISHED; + this.finishedTimeMs = replayedJob.finishedTimeMs; + + LOG.info("replay finished rollup job: {}", jobId); + } + + /* + * Replay job in CANCELLED state. + */ + private void replayCancelled(RollupJobV2 replayedJob) { + cancelInternal(); + this.jobState = JobState.CANCELLED; + this.finishedTimeMs = replayedJob.finishedTimeMs; + this.errMsg = replayedJob.errMsg; + LOG.info("replay cancelled rollup job: {}", jobId); + } + + @Override + public void replay(AlterJobV2 replayedJob) { + RollupJobV2 replayedRollupJob = (RollupJobV2) replayedJob; + switch (replayedJob.jobState) { + case PENDING: + replayPending(replayedRollupJob); + break; + case WAITING_TXN: + replayWaitingTxn(replayedRollupJob); + break; + case FINISHED: + replayFinished(replayedRollupJob); + break; + case CANCELLED: + replayCancelled(replayedRollupJob); + break; + default: + break; + } + } + + @Override + protected void getInfo(List> infos) { + List info = Lists.newArrayList(); + info.add(jobId); + info.add(tableName); + info.add(TimeUtils.longToTimeString(createTimeMs)); + info.add(TimeUtils.longToTimeString(finishedTimeMs)); + info.add(baseIndexName); + info.add(rollupIndexName); + info.add(rollupIndexId); + info.add(watershedTxnId); + info.add(jobState.name()); + info.add(errMsg); + // progress + if (jobState == JobState.RUNNING && rollupBatchTask.getTaskNum() > 0) { + info.add(rollupBatchTask.getFinishedTaskNum() + "/" + rollupBatchTask.getTaskNum()); + } else { + info.add("N/A"); + } + info.add(timeoutMs / 1000); + infos.add(info); + } + + public List> getUnfinishedTasks(int limit) { + List> taskInfos = Lists.newArrayList(); + if (jobState == JobState.RUNNING) { + List tasks = rollupBatchTask.getUnfinishedTasks(limit); + for (AgentTask agentTask : tasks) { + AlterReplicaTask rollupTask = (AlterReplicaTask)agentTask; + List info = Lists.newArrayList(); + info.add(String.valueOf(rollupTask.getBackendId())); + info.add(String.valueOf(rollupTask.getBaseTabletId())); + info.add(String.valueOf(rollupTask.getSignature())); + taskInfos.add(info); + } + } + return taskInfos; + } +} diff --git a/fe/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index b2dbd3118d6606..95d4622c7e0858 100644 --- a/fe/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -41,27 +41,32 @@ 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.PartitionInfo; import org.apache.doris.catalog.PartitionType; import org.apache.doris.catalog.RangePartitionInfo; 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.TabletMeta; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; import org.apache.doris.common.FeConstants; +import org.apache.doris.common.UserException; import org.apache.doris.common.util.ListComparator; import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.common.util.Util; +import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; -import org.apache.doris.thrift.TResourceInfo; -import org.apache.doris.thrift.TStorageType; +import org.apache.doris.thrift.TStorageMedium; import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.logging.log4j.LogManager; @@ -79,6 +84,9 @@ public class SchemaChangeHandler extends AlterHandler { private static final Logger LOG = LogManager.getLogger(SchemaChangeHandler.class); + // all shadow indexes should have this prefix in name + public static final String SHADOW_NAME_PRFIX = "__doris_shadow_"; + public SchemaChangeHandler() { super("schema change"); } @@ -90,10 +98,6 @@ private void processAddColumn(AddColumnClause alterClause, OlapTable olapTable, String targetIndexName = alterClause.getRollupName(); checkIndexExists(olapTable, targetIndexName); - if (column.isKey()) { - checkKeyModificationIfInRandomDistributedTable(olapTable); - } - String baseIndexName = olapTable.getName(); checkAssignedTargetIndexName(baseIndexName, targetIndexName); @@ -116,9 +120,6 @@ private void processAddColumns(AddColumnsClause alterClause, OlapTable olapTable Set newColNameSet = Sets.newHashSet(); for (Column column : columns) { - if (column.isKey()) { - checkKeyModificationIfInRandomDistributedTable(olapTable); - } newColNameSet.add(column.getName()); } @@ -143,14 +144,15 @@ private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable String targetIndexName = alterClause.getRollupName(); checkIndexExists(olapTable, targetIndexName); - Column dropColumn = olapTable.getColumn(dropColName); - if (dropColumn != null && dropColumn.isKey()) { - checkKeyModificationIfInRandomDistributedTable(olapTable); - } - String baseIndexName = olapTable.getName(); checkAssignedTargetIndexName(baseIndexName, targetIndexName); + /* + * UNIQUE: + * Can not drop any key column. + * AGGREGATION: + * Can not drp any key column is has value with REPLACE method + */ if (KeysType.UNIQUE_KEYS == olapTable.getKeysType()) { long baseIndexId = olapTable.getBaseIndexId(); List baseSchema = indexSchemaMap.get(baseIndexId); @@ -163,7 +165,7 @@ private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable } if (isKey) { - throw new DdlException("key column of unique key table cannot be droped"); + throw new DdlException("Can not drop key column in Unique data model table"); } } else if (KeysType.AGG_KEYS == olapTable.getKeysType()) { @@ -181,10 +183,10 @@ private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable } } if (isKey && hasReplaceColumn) { - throw new DdlException("key column of table with replace aggregation method cannot be droped"); + throw new DdlException("Can not drop key column when table has value column with REPLACE aggregation method"); } } else { - // drop column in rollup and basetable + // drop column in rollup and base index long targetIndexId = olapTable.getIndexIdByName(targetIndexName); // find column List targetIndexSchema = indexSchemaMap.get(targetIndexId); @@ -198,14 +200,14 @@ private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable } } if (isKey && hasReplaceColumn) { - throw new DdlException("key column of table with replace aggregation method cannot be droped"); + throw new DdlException("Can not drop key column when rollup has value column with REPLACE aggregation metho"); } } } long baseIndexId = olapTable.getBaseIndexId(); if (targetIndexName == null) { - // drop base index and all rollup indices's column + // if not specify rollup index, column should be dropped from both base and rollup indexes. List indexIds = new ArrayList(); indexIds.add(baseIndexId); for (long indexId : olapTable.getIndexIdToSchema().keySet()) { @@ -228,7 +230,7 @@ private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable } } if (!found) { - throw new DdlException("Column[" + dropColName + "] does not exists"); + throw new DdlException("Column does not exists: " + dropColName); } // remove column in rollup index if exists (i = 1 to skip base index) @@ -244,7 +246,7 @@ private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable } } // end for index names } else { - // only drop column from specified rollup index + // if specify rollup index, only drop column from specified rollup index long targetIndexId = olapTable.getIndexIdByName(targetIndexName); // find column List targetIndexSchema = indexSchemaMap.get(targetIndexId); @@ -259,33 +261,34 @@ private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable } } if (!found) { - throw new DdlException("Column[" + dropColName + "] does not exists"); + throw new DdlException("Column does not exists: " + dropColName); } } } + // User can modify column type and column position private void processModifyColumn(ModifyColumnClause alterClause, OlapTable olapTable, Map> indexSchemaMap) throws DdlException { Column modColumn = alterClause.getColumn(); if (KeysType.AGG_KEYS == olapTable.getKeysType()) { if (modColumn.isKey() && null != modColumn.getAggregationType()) { - throw new DdlException("key column of aggregate key table cannot use aggregation method"); + throw new DdlException("Can not assign aggregation method on key column: " + modColumn.getName()); } else if (null == modColumn.getAggregationType()) { - // in aggregate key table, no aggreation method indicate key column + // in aggregate key table, no aggregation method indicate key column modColumn.setIsKey(true); } } else if (KeysType.UNIQUE_KEYS == olapTable.getKeysType()) { if (null != modColumn.getAggregationType()) { - throw new DdlException("column of unique key table cannot use aggregation method"); + throw new DdlException("Can not assign aggregation method on column in Unique data model table: " + modColumn.getName()); } if (false == modColumn.isKey()) { modColumn.setAggregationType(AggregateType.REPLACE, true); } } else { if (null != modColumn.getAggregationType()) { - throw new DdlException("column of duplicate key table cannot use aggregation method"); + throw new DdlException("Can not assign aggregation method on column in Duplicate data model table: " + modColumn.getName()); } - if (false == modColumn.isKey()) { + if (!modColumn.isKey()) { modColumn.setAggregationType(AggregateType.NONE, true); } } @@ -293,10 +296,6 @@ private void processModifyColumn(ModifyColumnClause alterClause, OlapTable olapT String targetIndexName = alterClause.getRollupName(); checkIndexExists(olapTable, targetIndexName); - if (modColumn.isKey()) { - checkKeyModificationIfInRandomDistributedTable(olapTable); - } - String baseIndexName = olapTable.getName(); checkAssignedTargetIndexName(baseIndexName, targetIndexName); @@ -316,6 +315,7 @@ private void processModifyColumn(ModifyColumnClause alterClause, OlapTable olapT String newColName = modColumn.getName(); boolean hasColPos = (columnPos != null && !columnPos.isFirst()); boolean found = false; + boolean typeChanged = false; int modColIndex = -1; int lastColIndex = -1; for (int i = 0; i < schemaForFinding.size(); i++) { @@ -323,6 +323,9 @@ private void processModifyColumn(ModifyColumnClause alterClause, OlapTable olapT if (col.getName().equalsIgnoreCase(newColName)) { modColIndex = i; found = true; + if (!col.equals(modColumn)) { + typeChanged = true; + } } if (hasColPos) { if (col.getName().equalsIgnoreCase(columnPos.getLastCol())) { @@ -370,8 +373,6 @@ private void processModifyColumn(ModifyColumnClause alterClause, OlapTable olapT } else { schemaForFinding.set(modColIndex, modColumn); } - int temp = modColIndex; - Column tempCol = schemaForFinding.get(temp); // check if column being mod if (!modColumn.equals(oriColumn)) { @@ -387,20 +388,19 @@ private void processModifyColumn(ModifyColumnClause alterClause, OlapTable olapT } List schema = entry.getValue(); for (Column column : schema) { - if (column.getName().equals(modColumn.getName())) { + if (column.getName().equalsIgnoreCase(modColumn.getName())) { otherIndexIds.add(entry.getKey()); break; } } } - if (KeysType.AGG_KEYS == olapTable.getKeysType() - || KeysType.UNIQUE_KEYS == olapTable.getKeysType()) { + if (KeysType.AGG_KEYS == olapTable.getKeysType() || KeysType.UNIQUE_KEYS == olapTable.getKeysType()) { for (Long otherIndexId : otherIndexIds) { List otherIndexSchema = indexSchemaMap.get(otherIndexId); modColIndex = -1; for (int i = 0; i < otherIndexSchema.size(); i++) { - if (otherIndexSchema.get(i).getName().equals(modColumn.getName())) { + if (otherIndexSchema.get(i).getName().equalsIgnoreCase(modColumn.getName())) { modColIndex = i; break; } @@ -410,11 +410,12 @@ private void processModifyColumn(ModifyColumnClause alterClause, OlapTable olapT otherIndexSchema.set(modColIndex, modColumn); } // end for other indices } else { + // DUPLICATE data model has a little for (Long otherIndexId : otherIndexIds) { List otherIndexSchema = indexSchemaMap.get(otherIndexId); modColIndex = -1; for (int i = 0; i < otherIndexSchema.size(); i++) { - if (otherIndexSchema.get(i).getName().equals(modColumn.getName())) { + if (otherIndexSchema.get(i).getName().equalsIgnoreCase(modColumn.getName())) { modColIndex = i; break; } @@ -432,9 +433,24 @@ private void processModifyColumn(ModifyColumnClause alterClause, OlapTable olapT } otherIndexSchema.set(modColIndex, otherCol); } - tempCol = schemaForFinding.get(temp); } } // end for handling other indices + + if (typeChanged) { + /* + * In new alter table process (AlterJobV2), any modified columns are treated as new columns. + * But the modified columns' name does not changed. So in order to distinguish this, we will add + * a prefix in the name of these modified columns. + * This prefix only exist during the schema change process. Once the schema change is finished, + * it will be removed. + * + * After adding this prefix, modify a column is just same as 'add' a column. + * + * And if the column type is not changed, the same column name is still to the same column type, + * so no need to add prefix. + */ + modColumn.setName(SHADOW_NAME_PRFIX + modColumn.getName()); + } } private void processReorderColumn(ReorderColumnsClause alterClause, OlapTable olapTable, @@ -443,13 +459,6 @@ private void processReorderColumn(ReorderColumnsClause alterClause, OlapTable ol String targetIndexName = alterClause.getRollupName(); checkIndexExists(olapTable, targetIndexName); - for (String colName : orderedColNames) { - Column reorderdCol = olapTable.getColumn(colName); - if (reorderdCol != null && reorderdCol.isKey()) { - checkKeyModificationIfInRandomDistributedTable(olapTable); - } - } - String baseIndexName = olapTable.getName(); checkAssignedTargetIndexName(baseIndexName, targetIndexName); @@ -489,27 +498,34 @@ private void processReorderColumn(ReorderColumnsClause alterClause, OlapTable ol indexSchemaMap.put(targetIndexId, newSchema); } + /* + * Add 'newColumn' to specified index. + * Modified schema will be saved in 'indexSchemaMap' + */ private void addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosition columnPos, long targetIndexId, long baseIndexId, String baseIndexName, Map> indexSchemaMap, Set newColNameSet) throws DdlException { + String newColName = newColumn.getName(); + // check the validation of aggregation method on column. + // also fill the default aggregation method if not specified. if (KeysType.AGG_KEYS == olapTable.getKeysType()) { if (newColumn.isKey() && newColumn.getAggregationType() != null) { - throw new DdlException("key column of aggregate table cannot use aggregation method"); + throw new DdlException("Can not assign aggregation method on key column: " + newColName); } else if (null == newColumn.getAggregationType()) { newColumn.setIsKey(true); } } else if (KeysType.UNIQUE_KEYS == olapTable.getKeysType()) { if (newColumn.getAggregationType() != null) { - throw new DdlException("column of unique table cannot use aggregation method"); + throw new DdlException("Can not assign aggregation method on column in Unique data model table: " + newColName); } if (!newColumn.isKey()) { newColumn.setAggregationType(AggregateType.REPLACE, true); } } else { if (newColumn.getAggregationType() != null) { - throw new DdlException("column of duplicate table cannot use aggregation method"); + throw new DdlException("Can not assign aggregation method on column in Duplicate data model table: " + newColName); } if (!newColumn.isKey()) { newColumn.setAggregationType(AggregateType.NONE, true); @@ -518,11 +534,12 @@ private void addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosi // hll must be used in agg_keys if (newColumn.getType().isHllType() && KeysType.AGG_KEYS != olapTable.getKeysType()) { - throw new DdlException("HLL must be used in AGG_KEYS"); + throw new DdlException("HLL type column can only be in Aggregation data model table: " + newColName); } + // check if the new column already exist in base schema. + // do not support adding new column which already exist in base schema. List baseSchema = olapTable.getBaseSchema(); - String newColName = newColumn.getName(); boolean found = false; for (Column column : baseSchema) { if (column.getName().equalsIgnoreCase(newColName)) { @@ -531,17 +548,27 @@ private void addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosi } } if (found) { - throw new DdlException("Column[" + newColName + "] already exists in base index[" + baseIndexName + "]"); + throw new DdlException("Can not add column which already exists in base table: " + newColName); } + /* + * add new column to indexes. + * UNIQUE: + * 1. If new column is key, it should be added to all indexes. + * 2. Else, add the new column to base index and specified rollup index. + * DUPLICATE: + * 1. If not specify rollup index, just add it to base index. + * 2. Else, first add it to specify rollup index. Then if the new column is key, add it to base + * index, at the end of all other existing key columns. If new new column is value, add it to + * base index by user specified position. + * AGGREGATION: + * 1. Add it to base index, as well as specified rollup index. + */ if (KeysType.UNIQUE_KEYS == olapTable.getKeysType()) { - // check if has default value. this should be done in Analyze phase - // 1. add to base index first List modIndexSchema; if (newColumn.isKey()) { - // add key column to unique key table, should add to all rollups - // Column column = olapTable.getColumn(columnPos.getLastCol()); - // add to all table including base and rollup + // add key column to unique key table + // add to all indexes including base and rollup for (Map.Entry> entry : indexSchemaMap.entrySet()) { modIndexSchema = entry.getValue(); boolean isBaseIdex = entry.getKey() == baseIndexId; @@ -554,28 +581,26 @@ private void addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosi if (targetIndexId == -1L) { return; } - // 2. add to rollup modIndexSchema = indexSchemaMap.get(targetIndexId); checkAndAddColumn(modIndexSchema, newColumn, columnPos, newColNameSet, false); } } else if (KeysType.DUP_KEYS == olapTable.getKeysType()) { if (targetIndexId == -1L) { - // check if has default value. this should be done in Analyze phase - // 1. add to base index first + // add to base index List modIndexSchema = indexSchemaMap.get(baseIndexId); checkAndAddColumn(modIndexSchema, newColumn, columnPos, newColNameSet, true); // no specified target index. return return; } else { - // 2. add to rollup index + // add to rollup index List modIndexSchema = indexSchemaMap.get(targetIndexId); checkAndAddColumn(modIndexSchema, newColumn, columnPos, newColNameSet, false); if (newColumn.isKey()) { /* * if add column in rollup is key, - * then put the column in base table as end key + * then put the column in base table as the last key column */ modIndexSchema = indexSchemaMap.get(baseIndexId); checkAndAddColumn(modIndexSchema, newColumn, null, newColNameSet, true); @@ -601,6 +626,15 @@ private void addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosi } } + /* + * add new column to specified index schema('modIndexSchema'). + * if 'isBaseIndex' is true, which means 'modIndexSchema' is base index's schema. + * so we will not check repeat adding of column. + * For example, user want to add column k1 to both rollup1 and rollup2 in one alter stmt: + * ADD COLUMN k1 int to rollup1, + * ADD COLUMN k1 int to rollup2 + * So that k1 will be added to base index 'twice', and we just ignore this repeat adding. + */ private void checkAndAddColumn(List modIndexSchema, Column newColumn, ColumnPosition columnPos, Set newColNameSet, boolean isBaseIndex) throws DdlException { int posIndex = -1; @@ -611,13 +645,13 @@ private void checkAndAddColumn(List modIndexSchema, Column newColumn, Co if (col.getName().equalsIgnoreCase(newColName)) { if (!isBaseIndex || !newColNameSet.contains(newColName)) { // if this is not a base index, we should check if user repeatedly add columns - throw new DdlException("Repeatedly add column[" + newColName + "]"); + throw new DdlException("Repeatedly add column: " + newColName); } // this is a base index, and the column we check here is added by previous 'add column clause' // in same ALTER stmt. // so here we will check if the 2 columns is exactly same. if not, throw exception if (!col.equals(newColumn)) { - throw new DdlException("Repeatedly add same column[" + newColName + "] with different definition"); + throw new DdlException("Repeatedly add same column with different definition: " + newColName); } // column already exist, return @@ -663,16 +697,7 @@ private void checkAndAddColumn(List modIndexSchema, Column newColumn, Co checkRowLength(modIndexSchema); } - private void checkKeyModificationIfInRandomDistributedTable(OlapTable olapTable) throws DdlException { - for (Partition partition : olapTable.getPartitions()) { - DistributionInfo distributionInfo = partition.getDistributionInfo(); - if (distributionInfo.getType() == DistributionInfoType.RANDOM) { - throw new DdlException("Cannot add/del/reorder/modify key column " - + "in table which is distributed by random"); - } - } - } - + // row length can not large than limit private void checkRowLength(List modIndexSchema) throws DdlException { int rowLengthBytes = 0; for (Column column : modIndexSchema) { @@ -685,8 +710,24 @@ private void checkRowLength(List modIndexSchema) throws DdlException { } } + private void checkIndexExists(OlapTable olapTable, String targetIndexName) throws DdlException { + if (targetIndexName != null && !olapTable.hasMaterializedIndex(targetIndexName)) { + throw new DdlException("Index[" + targetIndexName + "] does not exist in table[" + olapTable.getName() + + "]"); + } + } + + private void checkAssignedTargetIndexName(String baseIndexName, String targetIndexName) throws DdlException { + // user cannot assign base index to do schema change + if (targetIndexName != null) { + if (targetIndexName.equals(baseIndexName)) { + throw new DdlException("Do not need to assign base index[" + baseIndexName + "] to do schema change"); + } + } + } + private void createJob(long dbId, OlapTable olapTable, Map> indexSchemaMap, - Map propertyMap) throws DdlException { + Map propertyMap) throws UserException { if (olapTable.getState() == OlapTableState.ROLLUP) { throw new DdlException("Table[" + olapTable.getName() + "]'s is doing ROLLUP job"); } @@ -787,26 +828,20 @@ private void createJob(long dbId, OlapTable olapTable, Map indexIdToShortKeyColumnCount = new HashMap(); + Map indexIdToShortKeyColumnCount = Maps.newHashMap(); + Map> changedIndexIdToSchema = Maps.newHashMap(); for (Long alterIndexId : indexSchemaMap.keySet()) { List originSchema = olapTable.getSchemaByIndexId(alterIndexId); List alterSchema = indexSchemaMap.get(alterIndexId); @@ -825,7 +860,7 @@ private void createJob(long dbId, OlapTable olapTable, Map 0) { - // just skip it (replica cloned from old schema will be deleted) - continue; - } - ++replicaNum; - } // end for replicas - - if (replicaNum < replicationNum / 2 + 1) { - String errMsg = "Tablet[" + tablet.getId() + "] does not have enough replicas. [" - + replicaNum + "/" + replicationNum + "]"; - LOG.warn(errMsg); - throw new DdlException(errMsg); - } - } // end for tablets - } // end for partitions - - // 6. calc short key + // 5. calc short key short newShortKeyColumnCount = Catalog.calcShortKeyColumnCount(alterSchema, indexIdToProperties.get(alterIndexId)); LOG.debug("alter index[{}] short key column count: {}", alterIndexId, newShortKeyColumnCount); indexIdToShortKeyColumnCount.put(alterIndexId, newShortKeyColumnCount); - // 7. check storage type if has null column - TStorageType storageType = olapTable.getStorageTypeByIndexId(alterIndexId); - boolean hasNullColumn = false; - for (Column column : alterSchema) { - if (column.isAllowNull()) { - hasNullColumn = true; - break; - } - } - if (hasNullColumn && storageType != TStorageType.COLUMN) { - throw new DdlException("Only column rollup support null columns"); - } - - // 8. store the changed columns for edit log - schemaChangeJob.putToChangedIndexSchemaMap(alterIndexId, alterSchema); + // 6. store the changed columns for edit log + changedIndexIdToSchema.put(alterIndexId, alterSchema); LOG.debug("schema change[{}-{}-{}] check pass.", dbId, tableId, alterIndexId); } // end for indices - if (schemaChangeJob.getChangedIndexToSchema().isEmpty()) { + if (changedIndexIdToSchema.isEmpty()) { throw new DdlException("Nothing is changed. please check your alter stmt."); } - // from now on, storage type can only be column - schemaChangeJob.setNewStorageType(TStorageType.COLUMN); - // the following operations are done outside the 'for indices' loop // to avoid partial check success - // 1. create schema change job - int newSchemaHash = -1; - for (Partition onePartition : olapTable.getPartitions()) { - for (Map.Entry> entry : schemaChangeJob.getChangedIndexToSchema().entrySet()) { - long indexId = entry.getKey(); - MaterializedIndex alterIndex = onePartition.getIndex(indexId); - Preconditions.checkState(alterIndex.getState() == IndexState.NORMAL, alterIndex.getState()); - - // set new schema - int currentSchemaVersion = olapTable.getSchemaVersionByIndexId(indexId); - int newSchemaVersion = currentSchemaVersion + 1; - List alterColumns = entry.getValue(); - // int newSchemaHash = Util.schemaHash(newSchemaVersion, alterColumns, bfColumns, bfFpp); - // new schema hash should only be generate one time, or the schema hash will differenent from each other in different partitions - if (newSchemaHash == -1) { - newSchemaHash = Util.generateSchemaHash(); - int currentSchemaHash = olapTable.getSchemaHashByIndexId(indexId); - // has to generate a new schema hash not equal to current schema hash - while (currentSchemaHash == newSchemaHash) { - newSchemaHash = Util.generateSchemaHash(); + /* + * Create schema change job + * 1. For each index which has been changed, create a SHADOW index, and save the mapping of origin index to SHADOW index. + * 2. Create all tablets and replicas of all SHADOW index, add them to tablet inverted index. + * 3. Change table's state as SCHEMA_CHANGE + */ + for (Map.Entry> entry : changedIndexIdToSchema.entrySet()) { + long originIndexId = entry.getKey(); + // 1. get new schema version/schema version hash, short key column count + int currentSchemaVersion = olapTable.getSchemaVersionByIndexId(originIndexId); + int newSchemaVersion = currentSchemaVersion + 1; + // generate schema hash for new index has to generate a new schema hash not equal to current schema hash + int currentSchemaHash = olapTable.getSchemaHashByIndexId(originIndexId); + int newSchemaHash = Util.generateSchemaHash(); + while (currentSchemaHash == newSchemaHash) { + newSchemaHash = Util.generateSchemaHash(); + } + String newIndexName = SHADOW_NAME_PRFIX + olapTable.getIndexNameById(originIndexId); + short newShortKeyColumnCount = indexIdToShortKeyColumnCount.get(originIndexId); + long shadowIndexId = catalog.getNextId(); + + // create SHADOW index for each partition + for (Partition partition : olapTable.getPartitions()) { + long partitionId = partition.getId(); + TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty(partitionId).getStorageMedium(); + // index state is SHADOW + MaterializedIndex shadowIndex = new MaterializedIndex(shadowIndexId, IndexState.SHADOW); + MaterializedIndex originIndex = partition.getIndex(originIndexId); + TabletMeta shadowTabletMeta = new TabletMeta(dbId, tableId, partitionId, shadowIndexId, newSchemaHash, medium); + for (Tablet originTablet : originIndex.getTablets()) { + long originTabletId = originTablet.getId(); + long shadowTabletId = catalog.getNextId(); + + Tablet shadowTablet = new Tablet(shadowTabletId); + shadowIndex.addTablet(shadowTablet, shadowTabletMeta); + + schemaChangeJob.addTabletIdMap(partitionId, shadowIndexId, shadowTabletId, originTabletId); + List originReplicas = originTablet.getReplicas(); + + for (Replica originReplica : originReplicas) { + long shadowReplicaId = catalog.getNextId(); + long backendId = originReplica.getBackendId(); + Preconditions.checkState(originReplica.getState() == ReplicaState.NORMAL); + Replica shadowReplica = new Replica(shadowReplicaId, backendId, ReplicaState.ALTER, + Partition.PARTITION_INIT_VERSION, Partition.PARTITION_INIT_VERSION_HASH, + newSchemaHash); + shadowTablet.addReplica(shadowReplica); } } - short newShortKeyColumnCount = indexIdToShortKeyColumnCount.get(indexId); - schemaChangeJob.setNewSchemaInfo(indexId, newSchemaVersion, newSchemaHash, newShortKeyColumnCount); - - // set replica state - for (Tablet tablet : alterIndex.getTablets()) { - for (Replica replica : tablet.getReplicas()) { - if (replica.getState() == ReplicaState.CLONE - || replica.getState() == ReplicaState.DECOMMISSION - || replica.getLastFailedVersion() > 0) { - // this should not happen, cause we only allow schema change when table is stable. - LOG.error("replica {} of tablet {} on backend {} is not NORMAL: {}", - replica.getId(), tablet.getId(), replica.getBackendId(), replica); - continue; - } - Preconditions.checkState(replica.getState() == ReplicaState.NORMAL, replica.getState()); - replica.setState(ReplicaState.SCHEMA_CHANGE); - } // end for replicas - } // end for tablets - - Catalog.getCurrentInvertedIndex().setNewSchemaHash(onePartition.getId(), indexId, newSchemaHash); - - alterIndex.setState(IndexState.SCHEMA_CHANGE); - } // end for indices - - onePartition.setState(PartitionState.SCHEMA_CHANGE); - } // end for partitions - + + schemaChangeJob.addPartitionShadowIndex(partitionId, shadowIndexId, shadowIndex); + } // end for partition + schemaChangeJob.addIndexSchema(shadowIndexId, originIndexId, newIndexName, newSchemaVersion, newSchemaHash, newShortKeyColumnCount, entry.getValue()); + } // end for index + + // set table state olapTable.setState(OlapTableState.SCHEMA_CHANGE); // 2. add schemaChangeJob - addAlterJob(schemaChangeJob); + addAlterJobV2(schemaChangeJob); - // 3. log schema change start operation - Catalog.getInstance().getEditLog().logStartSchemaChange(schemaChangeJob); - LOG.info("schema change job created. table[{}]", olapTable.getName()); + // 3. write edit log + Catalog.getInstance().getEditLog().logAlterJob(schemaChangeJob); + LOG.info("finished to create schema change job: {}", schemaChangeJob.getJobId()); } - private void checkIndexExists(OlapTable olapTable, String targetIndexName) throws DdlException { - if (targetIndexName != null && !olapTable.hasMaterializedIndex(targetIndexName)) { - throw new DdlException("Index[" + targetIndexName + "] does not exist in table[" + olapTable.getName() - + "]"); - } + @Override + protected void runOneCycle() { + super.runOneCycle(); + runOldAlterJob(); + runAlterJobV2(); } - private void checkAssignedTargetIndexName(String baseIndexName, String targetIndexName) throws DdlException { - // user cannot assign base index to do schema change - if (targetIndexName != null) { - if (targetIndexName.equals(baseIndexName)) { - throw new DdlException("Do not need to assign base index[" + baseIndexName + "] to do schema change"); + private void runAlterJobV2() { + Iterator> iter = alterJobsV2.entrySet().iterator(); + while (iter.hasNext()) { + Map.Entry entry = iter.next(); + AlterJobV2 alterJob = entry.getValue(); + if (alterJob.isDone()) { + continue; } + alterJob.run(); } } - public void removeReplicaRelatedTask(long tableId, long tabletId, long replicaId, long backendId) { - AlterJob job = getAlterJob(tableId); - if (job != null) { - job.removeReplicaRelatedTask(-1L, tabletId, replicaId, backendId); - } - } - - @Override - protected void runOneCycle() { - super.runOneCycle(); + @Deprecated + private void runOldAlterJob() { List cancelledJobs = Lists.newArrayList(); List finishedJobs = Lists.newArrayList(); @@ -1218,6 +1205,32 @@ protected void runOneCycle() { @Override public List> getAlterJobInfosByDb(Database db) { List> schemaChangeJobInfos = new LinkedList>(); + getOldAlterJobInfos(db, schemaChangeJobInfos); + getAlterJobV2Infos(db, schemaChangeJobInfos); + + // sort by "JobId", "PartitionName", "CreateTime", "FinishTime", "IndexName", "IndexState" + ListComparator> comparator = new ListComparator>(0, 1, 2, 3, 4, 5); + Collections.sort(schemaChangeJobInfos, comparator); + return schemaChangeJobInfos; + } + + private void getAlterJobV2Infos(Database db, List> schemaChangeJobInfos) { + ConnectContext ctx = ConnectContext.get(); + for (AlterJobV2 alterJob : alterJobsV2.values()) { + if (alterJob.getDbId() != db.getId()) { + continue; + } + if (ctx != null) { + if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ctx, db.getFullName(), alterJob.getTableName(), PrivPredicate.ALTER)) { + continue; + } + } + alterJob.getInfo(schemaChangeJobInfos); + } + } + + @Deprecated + private void getOldAlterJobInfos(Database db, List> schemaChangeJobInfos) { List selectedJobs = Lists.newArrayList(); lock(); @@ -1252,15 +1265,11 @@ public List> getAlterJobInfosByDb(Database db) { } finally { db.readUnlock(); } - - // sort by "JobId", "PartitionName", "CreateTime", "FinishTime", "IndexName", "IndexState" - ListComparator> comparator = new ListComparator>(0, 1, 2, 3, 4, 5); - Collections.sort(schemaChangeJobInfos, comparator); - return schemaChangeJobInfos; } @Override - public void process(List alterClauses, String clusterName, Database db, OlapTable olapTable) throws DdlException { + public void process(List alterClauses, String clusterName, Database db, OlapTable olapTable) + throws UserException { // index id -> index schema Map> indexSchemaMap = new HashMap>(); for (Map.Entry> entry : olapTable.getIndexIdToSchema().entrySet()) { @@ -1278,6 +1287,9 @@ public void process(List alterClauses, String clusterName, Database throw new DdlException("reduplicated PROPERTIES"); } + // modification of colocate property is handle alone. + // And because there should be only one colocate property modification clause in stmt, + // so just return after finished handling. if (properties.containsKey(PropertyAnalyzer.PROPERTIES_COLOCATE_WITH)) { String colocateGroup = properties.get(PropertyAnalyzer.PROPERTIES_COLOCATE_WITH); Catalog.getInstance().modifyTableColocate(db, olapTable, colocateGroup, false, null); @@ -1325,33 +1337,49 @@ public void cancel(CancelStmt stmt) throws DdlException { throw new DdlException("Database[" + dbName + "] does not exist"); } - AlterJob alterJob = null; + AlterJob schemaChangeJob = null; + AlterJobV2 schemaChangeJobV2 = null; db.writeLock(); try { - // 1. get table - OlapTable olapTable = (OlapTable) db.getTable(tableName); - if (olapTable == null) { - throw new DdlException("Table[" + tableName + "] does not exist"); + Table table = db.getTable(tableName); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); } - - // 2. find schema change job - alterJob = alterJobs.get(olapTable.getId()); - if (alterJob == null) { - throw new DdlException("Table[" + tableName + "] is not under SCHEMA CHANGE"); + if (!(table instanceof OlapTable)) { + ErrorReport.reportDdlException(ErrorCode.ERR_NOT_OLAP_TABLE, tableName); } - - if (alterJob.getState() == JobState.FINISHING || - alterJob.getState() == JobState.FINISHED || - alterJob.getState() == JobState.CANCELLED) { - throw new DdlException("job is already " + alterJob.getState().name() + ", can not cancel it"); + OlapTable olapTable = (OlapTable) table; + if (olapTable.getState() != OlapTableState.SCHEMA_CHANGE) { + throw new DdlException("Table[" + tableName + "] is not under SCHEMA_CHANGE."); } - // 3. cancel schema change job - alterJob.cancel(olapTable, "user cancelled"); + // find from new alter jobs first + schemaChangeJobV2 = getAlterJobV2(olapTable.getId()); + if (schemaChangeJobV2 == null) { + schemaChangeJob = getAlterJob(olapTable.getId()); + Preconditions.checkNotNull(schemaChangeJob, olapTable.getId()); + 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"); + } } finally { db.writeUnlock(); } - jobDone(alterJob); + // alter job v2's cancel must be called outside the database lock + if (schemaChangeJobV2 != null) { + if (!schemaChangeJobV2.cancel("user cancelled")) { + throw new DdlException("Job can not be cancelled. State: " + schemaChangeJobV2.getJobState()); + } + return; + } + + // handle old alter job + if (schemaChangeJob != null && schemaChangeJob.getState() == JobState.CANCELLED) { + jobDone(schemaChangeJob); + } } } diff --git a/fe/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/src/main/java/org/apache/doris/alter/SchemaChangeJob.java index adcb2f0780312a..77a94d9429c7ae 100644 --- a/fe/src/main/java/org/apache/doris/alter/SchemaChangeJob.java +++ b/fe/src/main/java/org/apache/doris/alter/SchemaChangeJob.java @@ -22,6 +22,7 @@ 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; @@ -333,7 +334,7 @@ public int checkOrResendClearTasks() { OUTER_LOOP: for (Partition partition : olapTable.getPartitions()) { long partitionId = partition.getId(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { for (Tablet tablet : index.getTablets()) { List replicas = tablet.getReplicas(); for (Replica replica : replicas) { @@ -850,7 +851,7 @@ public int tryFinishJob() { // 3. update base schema if changed if (this.changedIndexIdToSchema.containsKey(olapTable.getBaseIndexId())) { - table.setNewBaseSchema(this.changedIndexIdToSchema.get(olapTable.getBaseIndexId())); + table.setNewFullSchema(this.changedIndexIdToSchema.get(olapTable.getBaseIndexId())); } // 4. update table bloom filter columns @@ -1015,7 +1016,7 @@ public void replayFinishing(Database db) { olapTable.setIndexStorageType(indexId, newStorageType); } if (indexId == olapTable.getBaseIndexId()) { - olapTable.setNewBaseSchema(entry.getValue()); + olapTable.setNewFullSchema(entry.getValue()); } } @@ -1105,12 +1106,13 @@ public void getJobInfo(List> jobInfos, OlapTable tbl) { jobInfo.add(TimeUtils.longToTimeString(finishedTime)); jobInfo.add("N/A"); // index name jobInfo.add("N/A"); // index id + jobInfo.add("N/A"); // origin id jobInfo.add("N/A"); // schema version - jobInfo.add("N/A"); // index state jobInfo.add(-1); // transaction id jobInfo.add(state.name()); // job state - jobInfo.add("N/A"); // progress jobInfo.add(cancelMsg); + jobInfo.add("N/A"); // progress + jobInfo.add(Config.alter_table_timeout_second); // timeout jobInfos.add(jobInfo); return; } @@ -1170,19 +1172,18 @@ public void getJobInfo(List> jobInfos, OlapTable tbl) { jobInfo.add(TimeUtils.longToTimeString(finishedTime)); jobInfo.add(tbl.getIndexNameById(indexId) == null ? "N/A" : 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(indexState.get(indexId)); // index state + 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 ? "N/A" : indexProgress.get(indexId)); // progress } else { jobInfo.add("N/A"); } - - jobInfo.add(cancelMsg); + jobInfo.add(Config.alter_table_timeout_second); jobInfos.add(jobInfo); } // end for indexIds diff --git a/fe/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java new file mode 100644 index 00000000000000..0c7596a85eb5e3 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java @@ -0,0 +1,872 @@ +// 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.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.Replica; +import org.apache.doris.catalog.Replica.ReplicaState; +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.MarkedCountDownLatch; +import org.apache.doris.common.Pair; +import org.apache.doris.common.io.Text; +import org.apache.doris.common.util.TimeUtils; +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.AlterReplicaTask; +import org.apache.doris.task.CreateReplicaTask; +import org.apache.doris.thrift.TStorageMedium; +import org.apache.doris.thrift.TStorageType; +import org.apache.doris.thrift.TTaskType; + +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.collect.HashBasedTable; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.google.common.collect.Table; +import com.google.common.collect.Table.Cell; + +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; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +/* + * Author: Chenmingyu + * Date: Jul 8, 2019 + */ + +/* + * Version 2 of SchemaChangeJob. + * This is for replacing the old SchemaChangeJob + * https://github.com/apache/incubator-doris/issues/1429 + */ +public class SchemaChangeJobV2 extends AlterJobV2 { + private static final Logger LOG = LogManager.getLogger(SchemaChangeJobV2.class); + + // partition id -> (shadow index id -> (shadow tablet id -> origin tablet id)) + private Table> partitionIndexTabletMap = HashBasedTable.create(); + // partition id -> (shadow index id -> shadow index)) + private Table partitionIndexMap = HashBasedTable.create(); + // shadow index id -> origin index id + private Map indexIdMap = Maps.newHashMap(); + // shadow index id -> shadow index name(__doris_shadow_xxx) + private Map indexIdToName = Maps.newHashMap(); + // shadow index id -> index schema + private Map> indexSchemaMap = Maps.newHashMap(); + // shadow index id -> (shadow index schema version : schema hash) + private Map> indexSchemaVersionAndHashMap = Maps.newHashMap(); + // shadow index id -> shadow index short key count + private Map indexShortKeyMap = Maps.newHashMap(); + + // bloom filter info + private boolean hasBfChange; + private Set bfColumns = null; + private double bfFpp = 0; + + // The schema change job will wait all transactions before this txn id finished, then send the schema change tasks. + protected long watershedTxnId = -1; + + // save all schema change tasks + private AgentBatchTask schemaChangeBatchTask = new AgentBatchTask(); + + public SchemaChangeJobV2(long jobId, long dbId, long tableId, String tableName, long timeoutMs) { + super(jobId, JobType.SCHEMA_CHANGE, dbId, tableId, tableName, timeoutMs); + + } + + private SchemaChangeJobV2() { + super(JobType.SCHEMA_CHANGE); + } + + public void addTabletIdMap(long partitionId, long shadowIdxId, long shadowTabletId, long originTabletId) { + Map tabletMap = partitionIndexTabletMap.get(partitionId, shadowIdxId); + if (tabletMap == null) { + tabletMap = Maps.newHashMap(); + partitionIndexTabletMap.put(partitionId, shadowIdxId, tabletMap); + } + tabletMap.put(shadowTabletId, originTabletId); + } + + public void addPartitionShadowIndex(long partitionId, long shadowIdxId, MaterializedIndex shadowIdx) { + partitionIndexMap.put(partitionId, shadowIdxId, shadowIdx); + } + + public void addIndexSchema(long shadowIdxId, long originIdxId, + String shadowIndexName, int shadowSchemaVersion, int shadowSchemaHash, + short shadowIdxShortKeyCount, List shadowIdxSchema) { + indexIdMap.put(shadowIdxId, originIdxId); + indexIdToName.put(shadowIdxId, shadowIndexName); + indexSchemaVersionAndHashMap.put(shadowIdxId, Pair.create(shadowSchemaVersion, shadowSchemaHash)); + indexShortKeyMap.put(shadowIdxId, shadowIdxShortKeyCount); + indexSchemaMap.put(shadowIdxId, shadowIdxSchema); + } + + public void setBloomFilterInfo(boolean hasBfChange, Set bfColumns, double bfFpp) { + this.hasBfChange = hasBfChange; + this.bfColumns = bfColumns; + this.bfFpp = bfFpp; + } + + /* + * runPendingJob(): + * 1. Create all replicas of all shadow indexes and wait them finished. + * 2. After creating done, add the shadow indexes to catalog, user can not see this + * shadow index, but internal load process will generate data for these indexes. + * 3. Get a new transaction id, then set job's state to WAITING_TXN + */ + @Override + protected void runPendingJob() { + Preconditions.checkState(jobState == JobState.PENDING, jobState); + + LOG.info("begin to send create replica tasks. job: {}", jobId); + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + cancel("Databasee " + dbId + " does not exist"); + return; + } + + // 1. create replicas + AgentBatchTask batchTask = new AgentBatchTask(); + // count total replica num + int totalReplicaNum = 0; + for (MaterializedIndex shadowIdx : partitionIndexMap.values()) { + for (Tablet tablet : shadowIdx.getTablets()) { + totalReplicaNum += tablet.getReplicas().size(); + } + } + MarkedCountDownLatch countDownLatch = new MarkedCountDownLatch(totalReplicaNum); + db.readLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + cancel("Table " + tableId + " does not exist"); + return; + } + Preconditions.checkState(tbl.getState() == OlapTableState.SCHEMA_CHANGE); + + for (long partitionId : partitionIndexMap.rowKeySet()) { + Partition partition = tbl.getPartition(partitionId); + if (partition == null) { + continue; + } + TStorageMedium storageMedium = tbl.getPartitionInfo().getDataProperty(partitionId).getStorageMedium(); + + Map shadowIndexMap = partitionIndexMap.row(partitionId); + for (Map.Entry entry : shadowIndexMap.entrySet()) { + long shadowIdxId = entry.getKey(); + MaterializedIndex shadowIdx = entry.getValue(); + + short shadowShortKeyColumnCount = indexShortKeyMap.get(shadowIdxId); + List shadowSchema = indexSchemaMap.get(shadowIdxId); + int shadowSchemaHash = indexSchemaVersionAndHashMap.get(shadowIdxId).second; + int originSchemaHash = tbl.getSchemaHashByIndexId(indexIdMap.get(shadowIdxId)); + + for (Tablet shadowTablet : shadowIdx.getTablets()) { + long shadowTabletId = shadowTablet.getId(); + List shadowReplicas = shadowTablet.getReplicas(); + for (Replica shadowReplica : shadowReplicas) { + long backendId = shadowReplica.getBackendId(); + countDownLatch.addMark(backendId, shadowTabletId); + CreateReplicaTask createReplicaTask = new CreateReplicaTask( + backendId, dbId, tableId, partitionId, shadowIdxId, shadowTabletId, + shadowShortKeyColumnCount, shadowSchemaHash, + Partition.PARTITION_INIT_VERSION, Partition.PARTITION_INIT_VERSION_HASH, + tbl.getKeysType(), TStorageType.COLUMN, storageMedium, + shadowSchema, bfColumns, bfFpp, countDownLatch); + createReplicaTask.setBaseTablet(partitionIndexTabletMap.get(partitionId, shadowIdxId).get(shadowTabletId), originSchemaHash); + + batchTask.addTask(createReplicaTask); + } // end for rollupReplicas + } // end for rollupTablets + } + } + } finally { + db.readUnlock(); + } + + // send all tasks and wait them finished + AgentTaskQueue.addBatchTask(batchTask); + AgentTaskExecutor.submit(batchTask); + // max timeout is 1 min + long timeout = Math.min(Config.tablet_create_timeout_second * 1000L * totalReplicaNum, 60000); + boolean ok = false; + try { + ok = countDownLatch.await(timeout, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + LOG.warn("InterruptedException: ", e); + ok = false; + } + + if (!ok) { + // create replicas failed. just cancel the job + // clear tasks and show the failed replicas to user + AgentTaskQueue.removeBatchTask(batchTask, TTaskType.CREATE); + String errMsg = null; + if (!countDownLatch.getStatus().ok()) { + errMsg = countDownLatch.getStatus().getErrorMsg(); + } else { + List> unfinishedMarks = countDownLatch.getLeftMarks(); + // only show at most 3 results + List> subList = unfinishedMarks.subList(0, Math.min(unfinishedMarks.size(), 3)); + errMsg = "Error replicas:" + Joiner.on(", ").join(subList); + } + LOG.warn("failed to create replicas for job: {}, {}", jobId, errMsg); + cancel("Create replicas failed. Error: " + errMsg); + return; + } + + // create all replicas success. + // add all shadow indexes to catalog + db.writeLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + cancel("Table " + tableId + " does not exist"); + return; + } + Preconditions.checkState(tbl.getState() == OlapTableState.SCHEMA_CHANGE); + addShadowIndexToCatalog(tbl); + } finally { + db.writeUnlock(); + } + + this.watershedTxnId = Catalog.getCurrentGlobalTransactionMgr().getTransactionIDGenerator().getNextTransactionId(); + this.jobState = JobState.WAITING_TXN; + + // write edit log + Catalog.getCurrentCatalog().getEditLog().logAlterJob(this); + LOG.info("transfer schema change job {} state to {}, watershed txn id: {}", jobId, this.jobState, watershedTxnId); + } + + private void addShadowIndexToCatalog(OlapTable tbl) { + for (long partitionId : partitionIndexMap.rowKeySet()) { + Partition partition = tbl.getPartition(partitionId); + if (partition == null) { + continue; + } + Map shadowIndexMap = partitionIndexMap.row(partitionId); + for (MaterializedIndex shadowIndex : shadowIndexMap.values()) { + Preconditions.checkState(shadowIndex.getState() == IndexState.SHADOW, shadowIndex.getState()); + partition.createRollupIndex(shadowIndex); + } + } + + for (long shadowIdxId : indexIdMap.keySet()) { + tbl.setIndexSchemaInfo(shadowIdxId, indexIdToName.get(shadowIdxId), indexSchemaMap.get(shadowIdxId), + indexSchemaVersionAndHashMap.get(shadowIdxId).first, + indexSchemaVersionAndHashMap.get(shadowIdxId).second, + indexShortKeyMap.get(shadowIdxId)); + tbl.setStorageTypeToIndex(shadowIdxId, TStorageType.COLUMN); + } + + tbl.rebuildFullSchema(); + } + + /* + * runWaitingTxnJob(): + * 1. Wait the transactions before the watershedTxnId to be finished. + * 2. If all previous transactions finished, send schema change tasks to BE. + * 3. Change job state to RUNNING. + */ + @Override + protected void runWaitingTxnJob() { + Preconditions.checkState(jobState == JobState.WAITING_TXN, jobState); + + if (!isPreviousLoadFinished()) { + LOG.info("wait transactions before {} to be finished, schema change job: {}", watershedTxnId, jobId); + return; + } + + LOG.info("previous transactions are all finished, begin to send schema change tasks. job: {}", jobId); + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + cancel("Databasee " + dbId + " does not exist"); + return; + } + + db.readLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + cancel("Table " + tableId + " does not exist"); + return; + } + Preconditions.checkState(tbl.getState() == OlapTableState.SCHEMA_CHANGE); + + for (long partitionId : partitionIndexMap.rowKeySet()) { + Partition partition = tbl.getPartition(partitionId); + Preconditions.checkNotNull(partition, partitionId); + + // the schema change task will transform the data before visible version(included). + long visibleVersion = partition.getVisibleVersion(); + long visibleVersionHash = partition.getVisibleVersionHash(); + + Map shadowIndexMap = partitionIndexMap.row(partitionId); + for (Map.Entry entry : shadowIndexMap.entrySet()) { + long shadowIdxId = entry.getKey(); + MaterializedIndex shadowIdx = entry.getValue(); + + long originIdxId = indexIdMap.get(shadowIdxId); + int shadowSchemaHash = indexSchemaVersionAndHashMap.get(shadowIdxId).second; + int originSchemaHash = tbl.getSchemaHashByIndexId(indexIdMap.get(shadowIdxId)); + + for (Tablet shadowTablet : shadowIdx.getTablets()) { + long shadowTabletId = shadowTablet.getId(); + long originTabletId = partitionIndexTabletMap.get(partitionId, shadowIdxId).get(shadowTabletId); + List shadowReplicas = shadowTablet.getReplicas(); + for (Replica shadowReplica : shadowReplicas) { + AlterReplicaTask rollupTask = new AlterReplicaTask( + shadowReplica.getBackendId(), dbId, tableId, partitionId, + shadowIdxId, originIdxId, + shadowTabletId, originTabletId, shadowReplica.getId(), + shadowSchemaHash, originSchemaHash, + visibleVersion, visibleVersionHash, jobId, JobType.SCHEMA_CHANGE); + schemaChangeBatchTask.addTask(rollupTask); + } + } + } + } // end for partitions + } finally { + db.readUnlock(); + } + + AgentTaskQueue.addBatchTask(schemaChangeBatchTask); + AgentTaskExecutor.submit(schemaChangeBatchTask); + this.jobState = JobState.RUNNING; + + // DO NOT write edit log here, tasks will be send again if FE restart or master changed. + LOG.info("transfer schema change job {} state to {}", jobId, this.jobState); + } + + /* + * runRunningJob() + * 1. Wait all schema change tasks to be finished. + * 2. Check the integrity of the newly created shadow indexes. + * 3. Replace the origin index with shadow index, and set shadow index's state as NORMAL to be visible to user. + * 4. Set job'state as FINISHED. + */ + @Override + protected void runRunningJob() { + Preconditions.checkState(jobState == JobState.RUNNING, jobState); + if (!schemaChangeBatchTask.isFinished()) { + LOG.info("schema change tasks not finished. job: {}", jobId); + return; + } + + /* + * all tasks are finished. check the integrity. + * we just check whether all new replicas are healthy. + */ + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + cancel("Databasee " + dbId + " does not exist"); + return; + } + + db.writeLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + cancel("Table " + tableId + " does not exist"); + return; + } + Preconditions.checkState(tbl.getState() == OlapTableState.SCHEMA_CHANGE); + + for (long partitionId : partitionIndexMap.rowKeySet()) { + Partition partition = tbl.getPartition(partitionId); + Preconditions.checkNotNull(partition, partitionId); + + long visiableVersion = partition.getVisibleVersion(); + long visiableVersionHash = partition.getVisibleVersionHash(); + short expectReplicationNum = tbl.getPartitionInfo().getReplicationNum(partition.getId()); + + Map shadowIndexMap = partitionIndexMap.row(partitionId); + for (Map.Entry entry : shadowIndexMap.entrySet()) { + MaterializedIndex shadowIdx = entry.getValue(); + + for (Tablet shadowTablet : shadowIdx.getTablets()) { + List replicas = shadowTablet.getReplicas(); + int healthyReplicaNum = 0; + for (Replica replica : replicas) { + if (replica.getLastFailedVersion() < 0 + && replica.checkVersionCatchUp(visiableVersion, visiableVersionHash, false)) { + healthyReplicaNum++; + } + } + + if (healthyReplicaNum < expectReplicationNum / 2 + 1) { + LOG.warn("shadow tablet {} has few healthy replicas: {}, schema change job: {}", + shadowTablet.getId(), replicas, jobId); + cancel("shadow tablet " + shadowTablet.getId() + " has few healthy replicas"); + return; + } + } // end for tablets + } + } // end for partitions + + // all partitions are good + onFinished(tbl); + } finally { + db.writeUnlock(); + } + + this.jobState = JobState.FINISHED; + this.finishedTimeMs = System.currentTimeMillis(); + + Catalog.getCurrentCatalog().getEditLog().logAlterJob(this); + LOG.info("schema change job finished: {}", jobId); + } + + private void onFinished(OlapTable tbl) { + // replace the origin index with shadow index, set index state as NORMAL + for (Partition partition : tbl.getPartitions()) { + // drop the origin index from partitions + for (Map.Entry entry : indexIdMap.entrySet()) { + long shadowIdxId = entry.getKey(); + long originIdxId = entry.getValue(); + // get index from catalog, not from 'partitionIdToRollupIndex'. + // because if this alter job is recovered from edit log, index in 'partitionIndexMap' + // is not the same object in catalog. So modification on that index can not reflect to the index + // in catalog. + MaterializedIndex shadowIdx = partition.getIndex(shadowIdxId); + Preconditions.checkNotNull(shadowIdx, shadowIdxId); + partition.deleteRollupIndex(originIdxId); + // set replica state + for (Tablet tablet : shadowIdx.getTablets()) { + for (Replica replica : tablet.getReplicas()) { + replica.setState(ReplicaState.NORMAL); + } + } + + partition.visualiseShadowIndex(shadowIdxId, originIdxId == partition.getBaseIndex().getId()); + } + } + + // update index schema info of each index + for (Map.Entry entry : indexIdMap.entrySet()) { + long shadowIdxId = entry.getKey(); + long originIdxId = entry.getValue(); + String shadowIdxName = tbl.getIndexNameById(shadowIdxId); + String originIdxName = tbl.getIndexNameById(originIdxId); + tbl.deleteIndexInfo(originIdxName); + // the shadow index name is '__doris_shadow_xxx', rename it to origin name 'xxx' + // this will also remove the prefix of columns + tbl.renameIndexForSchemaChange(shadowIdxName, originIdxName); + tbl.renameColumnNamePrefix(shadowIdxId); + + if (originIdxId == tbl.getBaseIndexId()) { + // set base index + tbl.setBaseIndexId(shadowIdxId); + } + } + // rebuild table's full schema + tbl.rebuildFullSchema(); + + // update bloom filter + if (hasBfChange) { + tbl.setBloomFilterInfo(bfColumns, bfFpp); + } + + tbl.setState(OlapTableState.NORMAL); + } + + /* + * cancel() can be called any time any place. + * We need to clean any possible residual of this job. + */ + @Override + public synchronized boolean cancel(String errMsg) { + if (jobState.isFinalState()) { + return false; + } + + cancelInternal(); + + this.errMsg = errMsg; + this.finishedTimeMs = System.currentTimeMillis(); + LOG.info("cancel {} job {}, err: {}", this.type, jobId, errMsg); + Catalog.getCurrentCatalog().getEditLog().logAlterJob(this); + return true; + } + + private void cancelInternal() { + // clear tasks if has + AgentTaskQueue.removeBatchTask(schemaChangeBatchTask, TTaskType.ALTER); + // remove all shadow indexes, and set state to NORMAL + TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db != null) { + db.writeLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl != null) { + for (long partitionId : partitionIndexMap.rowKeySet()) { + Partition partition = tbl.getPartition(partitionId); + Preconditions.checkNotNull(partition, partitionId); + + Map shadowIndexMap = partitionIndexMap.row(partitionId); + for (Map.Entry entry : shadowIndexMap.entrySet()) { + MaterializedIndex shadowIdx = entry.getValue(); + for (Tablet shadowTablet : shadowIdx.getTablets()) { + invertedIndex.deleteTablet(shadowTablet.getId()); + } + partition.deleteRollupIndex(shadowIdx.getId()); + } + } + for (String shadowIndexName : indexIdToName.values()) { + tbl.deleteIndexInfo(shadowIndexName); + } + tbl.setState(OlapTableState.NORMAL); + } + } finally { + db.writeUnlock(); + } + } + + jobState = JobState.CANCELLED; + } + + // Check whether transactions of the given database which txnId is less than 'watershedTxnId' are finished. + protected boolean isPreviousLoadFinished() { + return Catalog.getCurrentGlobalTransactionMgr().isPreviousTransactionsFinished(watershedTxnId, dbId); + } + + public static SchemaChangeJobV2 read(DataInput in) throws IOException { + SchemaChangeJobV2 schemaChangeJob = new SchemaChangeJobV2(); + schemaChangeJob.readFields(in); + return schemaChangeJob; + } + + /* + * Replay job in PENDING state. + * Should replay all changes before this job's state transfer to PENDING. + * These changes should be same as changes in SchemaChangeHandler.createJob() + */ + private void replayPending(SchemaChangeJobV2 replayedJob) { + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + // database may be dropped before replaying this log. just return + return; + } + + db.writeLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + // table may be dropped before replaying this log. just return + return; + } + + TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); + for (Cell cell : partitionIndexMap.cellSet()) { + long partitionId = cell.getRowKey(); + long shadowIndexId = cell.getColumnKey(); + MaterializedIndex shadowIndex = cell.getValue(); + + TStorageMedium medium = tbl.getPartitionInfo().getDataProperty(partitionId).getStorageMedium(); + TabletMeta shadowTabletMeta = new TabletMeta(dbId, tableId, partitionId, shadowIndexId, + indexSchemaVersionAndHashMap.get(shadowIndexId).second, medium); + + for (Tablet shadownTablet : shadowIndex.getTablets()) { + invertedIndex.addTablet(shadownTablet.getId(), shadowTabletMeta); + for (Replica shadowReplica : shadownTablet.getReplicas()) { + invertedIndex.addReplica(shadownTablet.getId(), shadowReplica); + } + } + } + + // set table state + tbl.setState(OlapTableState.SCHEMA_CHANGE); + } finally { + db.writeUnlock(); + } + + this.watershedTxnId = replayedJob.watershedTxnId; + jobState = JobState.WAITING_TXN; + LOG.info("replay pending schema change job: {}", jobId); + } + + /* + * Replay job in WAITING_TXN state. + * Should replay all changes in runPendingJob() + */ + private void replayWaitingTxn(SchemaChangeJobV2 replayedJob) { + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + // database may be dropped before replaying this log. just return + return; + } + + db.writeLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + // table may be dropped before replaying this log. just return + return; + } + addShadowIndexToCatalog(tbl); + } finally { + db.writeUnlock(); + } + + jobState = JobState.RUNNING; + this.watershedTxnId = replayedJob.watershedTxnId; + LOG.info("replay waiting txn schema change job: {}", jobId); + } + + /* + * Replay job in FINISHED state. + * Should replay all changes in runRuningJob() + */ + private void replayFinished(SchemaChangeJobV2 replayedJob) { + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db != null) { + db.writeLock(); + try { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl != null) { + onFinished(tbl); + } + } finally { + db.writeUnlock(); + } + } + jobState = JobState.FINISHED; + this.finishedTimeMs = replayedJob.finishedTimeMs; + LOG.info("replay finished schema change job: {}", jobId); + } + + /* + * Replay job in CANCELLED state. + */ + private void replayCancelled(SchemaChangeJobV2 replayedJob) { + cancelInternal(); + this.jobState = JobState.CANCELLED; + this.finishedTimeMs = replayedJob.finishedTimeMs; + this.errMsg = replayedJob.errMsg; + LOG.info("replay cancelled schema change job: {}", jobId); + } + + @Override + public void replay(AlterJobV2 replayedJob) { + SchemaChangeJobV2 replayedSchemaChangeJob = (SchemaChangeJobV2) replayedJob; + switch (replayedJob.jobState) { + case PENDING: + replayPending(replayedSchemaChangeJob); + break; + case WAITING_TXN: + replayWaitingTxn(replayedSchemaChangeJob); + break; + case FINISHED: + replayFinished(replayedSchemaChangeJob); + break; + case CANCELLED: + replayCancelled(replayedSchemaChangeJob); + break; + default: + break; + } + } + + @Override + protected void getInfo(List> infos) { + // calc progress first. all index share the same process + String progress = "N/A"; + if (jobState == JobState.RUNNING && schemaChangeBatchTask.getTaskNum() > 0) { + progress = schemaChangeBatchTask.getFinishedTaskNum() + "/" + schemaChangeBatchTask.getTaskNum(); + } + + // one line for one shadow index + for (Map.Entry entry : indexIdMap.entrySet()) { + long shadowIndexId = entry.getKey(); + List info = Lists.newArrayList(); + info.add(jobId); + info.add(tableName); + info.add(TimeUtils.longToTimeString(createTimeMs)); + info.add(TimeUtils.longToTimeString(finishedTimeMs)); + // only show the origin index name + info.add(indexIdToName.get(shadowIndexId).substring(SchemaChangeHandler.SHADOW_NAME_PRFIX.length())); + info.add(shadowIndexId); + info.add(entry.getValue()); + info.add(indexSchemaVersionAndHashMap.get(shadowIndexId).toString()); + info.add(watershedTxnId); + info.add(jobState.name()); + info.add(errMsg); + info.add(progress); + info.add(timeoutMs / 1000); + infos.add(info); + } + } + + public List> getUnfinishedTasks(int limit) { + List> taskInfos = Lists.newArrayList(); + if (jobState == JobState.RUNNING) { + List tasks = schemaChangeBatchTask.getUnfinishedTasks(limit); + for (AgentTask agentTask : tasks) { + AlterReplicaTask alterTask = (AlterReplicaTask) agentTask; + List info = Lists.newArrayList(); + info.add(String.valueOf(alterTask.getBackendId())); + info.add(String.valueOf(alterTask.getBaseTabletId())); + info.add(String.valueOf(alterTask.getSignature())); + taskInfos.add(info); + } + } + return taskInfos; + } + + @Override + public void write(DataOutput out) throws IOException { + super.write(out); + + out.writeInt(partitionIndexTabletMap.rowKeySet().size()); + for (Long partitionId : partitionIndexTabletMap.rowKeySet()) { + out.writeLong(partitionId); + Map> indexTabletMap = partitionIndexTabletMap.row(partitionId); + out.writeInt(indexTabletMap.size()); + for (Long shadowIndexId : indexTabletMap.keySet()) { + out.writeLong(shadowIndexId); + // tablet id map + Map tabletMap = indexTabletMap.get(shadowIndexId); + out.writeInt(tabletMap.size()); + for (Map.Entry entry : tabletMap.entrySet()) { + out.writeLong(entry.getKey()); + out.writeLong(entry.getValue()); + } + // shadow index + MaterializedIndex shadowIndex = partitionIndexMap.get(partitionId, shadowIndexId); + shadowIndex.write(out); + } + } + + // shadow index info + out.writeInt(indexIdMap.size()); + for (Map.Entry entry : indexIdMap.entrySet()) { + long shadowIndexId = entry.getKey(); + out.writeLong(shadowIndexId); + // index id map + out.writeLong(entry.getValue()); + // index name + Text.writeString(out, indexIdToName.get(shadowIndexId)); + // index schema + out.writeInt(indexSchemaMap.get(shadowIndexId).size()); + for (Column column : indexSchemaMap.get(shadowIndexId)) { + column.write(out); + } + // index schema version and hash + out.writeInt(indexSchemaVersionAndHashMap.get(shadowIndexId).first); + out.writeInt(indexSchemaVersionAndHashMap.get(shadowIndexId).second); + // short key count + out.writeShort(indexShortKeyMap.get(shadowIndexId)); + } + + // bloom filter + out.writeBoolean(hasBfChange); + if (hasBfChange) { + out.writeInt(bfColumns.size()); + for (String bfCol : bfColumns) { + Text.writeString(out, bfCol); + } + out.writeDouble(bfFpp); + } + + out.writeLong(watershedTxnId); + } + + @Override + 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(); + int indexNum = in.readInt(); + for (int j = 0; j < indexNum; j++) { + long shadowIndexId = in.readLong(); + int tabletNum = in.readInt(); + Map tabletMap = Maps.newHashMapWithExpectedSize(tabletNum); + for (int k = 0; k < tabletNum; k++) { + long shadowTabletId = in.readLong(); + long originTabletId = in.readLong(); + tabletMap.put(shadowTabletId, originTabletId); + } + partitionIndexTabletMap.put(partitionId, shadowIndexId, tabletMap); + // shadow index + MaterializedIndex shadowIndex = MaterializedIndex.read(in); + partitionIndexMap.put(partitionId, shadowIndexId, shadowIndex); + } + } + + // shadow index info + int indexNum = in.readInt(); + for (int i = 0; i < indexNum; i++) { + long shadowIndexId = in.readLong(); + long originIndexId = in.readLong(); + String indexName = Text.readString(in); + // index schema + int colNum = in.readInt(); + List schema = Lists.newArrayListWithCapacity(colNum); + for (int j = 0; j < colNum; j++) { + schema.add(Column.read(in)); + } + int schemaVersion = in.readInt(); + int schemaVersionHash = in.readInt(); + Pair schemaVersionAndHash = Pair.create(schemaVersion, schemaVersionHash); + short shortKeyCount = in.readShort(); + + indexIdMap.put(shadowIndexId, originIndexId); + indexIdToName.put(shadowIndexId, indexName); + indexSchemaMap.put(shadowIndexId, schema); + indexSchemaVersionAndHashMap.put(shadowIndexId, schemaVersionAndHash); + indexShortKeyMap.put(shadowIndexId, shortKeyCount); + } + + // bloom filter + hasBfChange = in.readBoolean(); + if (hasBfChange) { + int bfNum = in.readInt(); + bfColumns = Sets.newHashSetWithExpectedSize(bfNum); + for (int i = 0; i < bfNum; i++) { + bfColumns.add(Text.readString(in)); + } + bfFpp = in.readDouble(); + } + + watershedTxnId = in.readLong(); + } +} diff --git a/fe/src/main/java/org/apache/doris/alter/SystemHandler.java b/fe/src/main/java/org/apache/doris/alter/SystemHandler.java index e951f3439a4424..992cdf155bce34 100644 --- a/fe/src/main/java/org/apache/doris/alter/SystemHandler.java +++ b/fe/src/main/java/org/apache/doris/alter/SystemHandler.java @@ -38,8 +38,8 @@ import org.apache.doris.catalog.PartitionInfo; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Table.TableType; -import org.apache.doris.common.Config; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.Pair; diff --git a/fe/src/main/java/org/apache/doris/analysis/AddRollupClause.java b/fe/src/main/java/org/apache/doris/analysis/AddRollupClause.java index e674d32df0c4bb..454cf31cb72d1d 100644 --- a/fe/src/main/java/org/apache/doris/analysis/AddRollupClause.java +++ b/fe/src/main/java/org/apache/doris/analysis/AddRollupClause.java @@ -18,10 +18,12 @@ package org.apache.doris.analysis; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.FeNameFormat; import org.apache.doris.common.io.Text; +import org.apache.doris.common.util.PropertyAnalyzer; import com.google.common.base.Strings; import com.google.common.collect.Lists; @@ -43,6 +45,8 @@ public class AddRollupClause extends AlterClause { private List columnNames; private String baseRollupName; private List dupKeys; + private long timeoutSecond; + private Map properties; public AddRollupClause() { @@ -66,6 +70,10 @@ public String getBaseRollupName() { return baseRollupName; } + public long getTimeoutSecond() { + return timeoutSecond; + } + public AddRollupClause(String rollupName, List columnNames, List dupKeys, String baseRollupName, Map properties) { @@ -93,6 +101,8 @@ public void analyze(Analyzer analyzer) throws AnalysisException { } } baseRollupName = Strings.emptyToNull(baseRollupName); + + timeoutSecond = PropertyAnalyzer.analyzeTimeout(properties, Config.alter_table_timeout_second); } @Override diff --git a/fe/src/main/java/org/apache/doris/analysis/DataDescription.java b/fe/src/main/java/org/apache/doris/analysis/DataDescription.java index 0623a38ea30928..66562afb1a37e0 100644 --- a/fe/src/main/java/org/apache/doris/analysis/DataDescription.java +++ b/fe/src/main/java/org/apache/doris/analysis/DataDescription.java @@ -22,6 +22,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.Pair; @@ -34,6 +35,7 @@ import com.google.common.base.Strings; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -63,33 +65,42 @@ public class DataDescription { private static final Logger LOG = LogManager.getLogger(DataDescription.class); public static String FUNCTION_HASH_HLL = "hll_hash"; - private static final List hadoopSupportFunctionName = Arrays.asList("strftime", "time_format", - "alignment_timestamp", - "default_value", "md5sum", - "replace_value", "now", - "hll_hash"); + private static final List HADOOP_SUPPORT_FUNCTION_NAMES = Arrays.asList( + "strftime", + "time_format", + "alignment_timestamp", + "default_value", + "md5sum", + "replace_value", + "now", "hll_hash", + "substitute"); + private final String tableName; private final List partitionNames; private final List filePaths; // the column name list of data desc - private final List columns; + private List columns; private final ColumnSeparator columnSeparator; private final String fileFormat; private final boolean isNegative; + // save column mapping in SET(xxx = xxx) clause private final List columnMappingList; // Used for mini load private TNetworkAddress beAddr; private String lineDelimiter; - // This param only include the hadoop function which need to be checked in the future. - // For hadoop load, this param is also used to persistence. - private Map>> columnToHadoopFunction; - /** + /* * Merged from columns and columnMappingList * ImportColumnDesc: column name to expr or null - **/ - private List parsedColumnExprList; + */ + private List parsedColumnExprList = Lists.newArrayList(); + /* + * This param only include the hadoop function which need to be checked in the future. + * For hadoop load, this param is also used to persistence. + * The function in this param is copied from 'parsedColumnExprList' + */ + private Map>> columnToHadoopFunction = Maps.newHashMap(); private boolean isHadoopLoad = false; @@ -163,13 +174,9 @@ public void setLineDelimiter(String lineDelimiter) { } public void addColumnMapping(String functionName, Pair> pair) { - if (Strings.isNullOrEmpty(functionName) || pair == null) { return; } - if (columnToHadoopFunction == null) { - columnToHadoopFunction = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); - } columnToHadoopFunction.put(functionName, pair); } @@ -196,32 +203,33 @@ public boolean isHadoopLoad() { * "col2": "tmp_col2+1", "col3": "strftime("%Y-%m-%d %H:%M:%S", tmp_col3)"} */ private void analyzeColumns() throws AnalysisException { - if (columns == null || columns.isEmpty()) { - return; - } - // merge columns exprs from columns and columnMappingList - // used to check duplicated column name + // used to check duplicated column name in COLUMNS Set columnNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER); - parsedColumnExprList = Lists.newArrayList(); + + // merge columns exprs from columns and columnMappingList // Step1: analyze columns - for (String columnName : columns) { - if (!columnNames.add(columnName)) { - throw new AnalysisException("Duplicate column : " + columnName); + if (columns != null && !columns.isEmpty()) { + // Step1: analyze columns + for (String columnName : columns) { + if (!columnNames.add(columnName)) { + throw new AnalysisException("Duplicate column: " + columnName); + } + ImportColumnDesc importColumnDesc = new ImportColumnDesc(columnName, null); + parsedColumnExprList.add(importColumnDesc); } - ImportColumnDesc importColumnDesc = new ImportColumnDesc(columnName, null); - parsedColumnExprList.add(importColumnDesc); } - + // Step2: analyze column mapping if (columnMappingList == null || columnMappingList.isEmpty()) { return; } + + // used to check duplicated column name in SET clause + Set columnMappingNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER); // Step2: analyze column mapping // the column expr only support the SlotRef or eq binary predicate which's child(0) must be a SloRef. // the duplicate column name of SloRef is forbidden. - columnToHadoopFunction = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); for (Expr columnExpr : columnMappingList) { - if (!(columnExpr instanceof BinaryPredicate)) { throw new AnalysisException("Mapping function expr only support the column or eq binary predicate. " + "Expr: " + columnExpr.toSql()); @@ -237,7 +245,7 @@ private void analyzeColumns() throws AnalysisException { + "The mapping column error. column: " + child0.toSql()); } String column = ((SlotRef) child0).getColumnName(); - if (!columnNames.add(column)) { + if (!columnMappingNames.add(column)) { throw new AnalysisException("Duplicate column mapping: " + column); } // hadoop load only supports the FunctionCallExpr @@ -249,14 +257,16 @@ private void analyzeColumns() throws AnalysisException { ImportColumnDesc importColumnDesc = new ImportColumnDesc(column, child1); parsedColumnExprList.add(importColumnDesc); analyzeColumnToHadoopFunction(column, child1); - } } private void analyzeColumnToHadoopFunction(String columnName, Expr child1) throws AnalysisException { + if (!(child1 instanceof FunctionCallExpr)) { + return; + } FunctionCallExpr functionCallExpr = (FunctionCallExpr) child1; String functionName = functionCallExpr.getFnName().getFunction(); - if (!hadoopSupportFunctionName.contains(functionName.toLowerCase())) { + if (!HADOOP_SUPPORT_FUNCTION_NAMES.contains(functionName.toLowerCase())) { return; } List paramExprs = functionCallExpr.getParams().exprs(); @@ -301,6 +311,8 @@ public static void validateMappingFunction(String functionName, List arg validateHllHash(args, columnNameMap); } else if (functionName.equalsIgnoreCase("now")) { validateNowFunction(mappingColumn); + } else if (functionName.equalsIgnoreCase("substitute")) { + validateSubstituteFunction(args, columnNameMap); } else { if (isHadoopLoad) { throw new AnalysisException("Unknown function: " + functionName); @@ -308,6 +320,22 @@ public static void validateMappingFunction(String functionName, List arg } } + // eg: k2 = substitute(k1) + // this is used for creating derivative column from existing column + private static void validateSubstituteFunction(List args, Map columnNameMap) + throws AnalysisException { + if (args.size() != 1) { + throw new AnalysisException("Should has only one argument: " + args); + } + + String argColumn = args.get(0); + if (!columnNameMap.containsKey(argColumn)) { + throw new AnalysisException("Column is not in sources, column: " + argColumn); + } + + args.set(0, columnNameMap.get(argColumn)); + } + private static void validateAlignmentTimestamp(List args, Map columnNameMap) throws AnalysisException { if (args.size() != 2) { @@ -471,6 +499,49 @@ public void analyzeWithoutCheckPriv() throws AnalysisException { analyzeColumns(); } + /* + * If user does not specify COLUMNS in load stmt, we fill it here. + * eg1: + * both COLUMNS and SET clause is empty. after fill: + * (k1,k2,k3) + * + * eg2: + * COLUMNS is empty, SET is not empty + * SET ( k2 = default_value("2") ) + * after fill: + * (k1, k2, k3) + * SET ( k2 = default_value("2") ) + * + * eg3: + * COLUMNS is empty, SET is not empty + * SET (k2 = strftime("%Y-%m-%d %H:%M:%S", k2) + * after fill: + * (k1,k2,k3) + * SET (k2 = strftime("%Y-%m-%d %H:%M:%S", k2) + * + */ + public void fillColumnInfoIfNotSpecified(List baseSchema) throws DdlException { + if (columns != null && !columns.isEmpty()) { + return; + } + + columns = Lists.newArrayList(); + + Set mappingColNames = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + for (ImportColumnDesc importColumnDesc : parsedColumnExprList) { + mappingColNames.add(importColumnDesc.getColumnName()); + } + + for (Column column : baseSchema) { + if (!mappingColNames.contains(column.getName())) { + parsedColumnExprList.add(new ImportColumnDesc(column.getName(), null)); + } + columns.add(column.getName()); + } + + LOG.debug("after fill column info. columns: {}, parsed column exprs: {}", columns, parsedColumnExprList); + } + public String toSql() { StringBuilder sb = new StringBuilder(); sb.append("DATA INFILE ("); diff --git a/fe/src/main/java/org/apache/doris/analysis/InsertStmt.java b/fe/src/main/java/org/apache/doris/analysis/InsertStmt.java index dc3016de419f84..cae3088242f4c7 100644 --- a/fe/src/main/java/org/apache/doris/analysis/InsertStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/InsertStmt.java @@ -17,6 +17,7 @@ package org.apache.doris.analysis; +import org.apache.doris.alter.SchemaChangeHandler; import org.apache.doris.catalog.BrokerTable; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Column; @@ -296,7 +297,7 @@ private void analyzeTargetTable(Analyzer analyzer) throws AnalysisException { // need a descriptor DescriptorTable descTable = analyzer.getDescTbl(); olapTuple = descTable.createTupleDescriptor(); - for (Column col : olapTable.getBaseSchema()) { + for (Column col : olapTable.getFullSchema()) { SlotDescriptor slotDesc = descTable.addSlotDescriptor(olapTuple); slotDesc.setIsMaterialized(true); slotDesc.setType(col.getType()); @@ -319,8 +320,6 @@ private void analyzeTargetTable(Analyzer analyzer) throws AnalysisException { } BrokerTable brokerTable = (BrokerTable) targetTable; - List paths = brokerTable.getPaths(); - if (!brokerTable.isWritable()) { throw new AnalysisException("table " + brokerTable.getName() + "is not writable. path should be an dir"); @@ -346,10 +345,12 @@ private void checkColumnCoverage(Set mentionedCols, List baseCol } } - public void analyzeSubquery(Analyzer analyzer) throws UserException { + private void analyzeSubquery(Analyzer analyzer) throws UserException { // Analyze columns mentioned in the statement. Set mentionedColumns = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); if (targetColumnNames == null) { + // the mentioned columns are columns which are visible to user, so here we use + // getBaseSchema(), not getFullSchema() for (Column col : targetTable.getBaseSchema()) { mentionedColumns.add(col.getName()); targetColumns.add(col); @@ -367,6 +368,34 @@ public void analyzeSubquery(Analyzer analyzer) throws UserException { } } + /* + * When doing schema change, there may be some shadow columns. we should add + * them to the end of targetColumns. And use 'origColIdxsForShadowCols' to save + * the index of column in 'targetColumns' which the shadow column related to. + * eg: origin targetColumns: (A,B,C), shadow column: __doris_shadow_B after + * processing, targetColumns: (A, B, C, __doris_shadow_B), and + * origColIdxsForShadowCols has 1 element: "1", which is the index of column B + * in targetColumns. + * + * Rule A: If the column which the shadow column related to is not mentioned, + * then do not add the shadow column to targetColumns. They will be filled by + * null or default value when loading. + */ + List origColIdxsForShadowCols = Lists.newArrayList(); + for (Column column : targetTable.getFullSchema()) { + if (column.isNameWithPrefix(SchemaChangeHandler.SHADOW_NAME_PRFIX)) { + String origName = Column.removeNamePrefix(column.getName()); + for (int i = 0; i < targetColumns.size(); i++) { + if (targetColumns.get(i).nameEquals(origName, false)) { + // Rule A + origColIdxsForShadowCols.add(i); + targetColumns.add(column); + break; + } + } + } + } + // parse query statement queryStmt.setFromInsert(true); queryStmt.analyze(analyzer); @@ -378,22 +407,46 @@ public void analyzeSubquery(Analyzer analyzer) throws UserException { // Check if all columns mentioned is enough checkColumnCoverage(mentionedColumns, targetTable.getBaseSchema()) ; + + // handle VALUES() or SELECT constant list if (queryStmt instanceof SelectStmt && ((SelectStmt) queryStmt).getTableRefs().isEmpty()) { SelectStmt selectStmt = (SelectStmt) queryStmt; if (selectStmt.getValueList() != null) { + // INSERT INTO VALUES(...) List> rows = selectStmt.getValueList().getRows(); for (int rowIdx = 0; rowIdx < rows.size(); ++rowIdx) { - analyzeRow(analyzer, targetColumns, rows.get(rowIdx), rowIdx + 1); + analyzeRow(analyzer, targetColumns, rows, rowIdx, origColIdxsForShadowCols); } - for (int i = 0; i < selectStmt.getResultExprs().size(); ++i) { - selectStmt.getResultExprs().set(i, selectStmt.getValueList().getFirstRow().get(i)); - selectStmt.getBaseTblResultExprs().set(i, selectStmt.getValueList().getFirstRow().get(i)); + + // clear these 2 structures, rebuild them using VALUES exprs + selectStmt.getResultExprs().clear(); + selectStmt.getBaseTblResultExprs().clear(); + + for (int i = 0; i < selectStmt.getValueList().getFirstRow().size(); ++i) { + selectStmt.getResultExprs().add(selectStmt.getValueList().getFirstRow().get(i)); + selectStmt.getBaseTblResultExprs().add(selectStmt.getValueList().getFirstRow().get(i)); } } else { - analyzeRow(analyzer, targetColumns, selectStmt.getResultExprs(), 1); + // INSERT INTO SELECT 1,2,3 ... + List> rows = Lists.newArrayList(); + rows.add(selectStmt.getResultExprs()); + analyzeRow(analyzer, targetColumns, rows, 0, origColIdxsForShadowCols); + // rows may be changed in analyzeRow(), so rebuild the result exprs + selectStmt.getResultExprs().clear(); + for (Expr expr : rows.get(0)) { + selectStmt.getResultExprs().add(expr); + } } isStreaming = true; } else { + // INSERT INTO SELECT ... FROM tbl + if (!origColIdxsForShadowCols.isEmpty()) { + // extend the result expr by duplicating the related exprs + for (Integer idx : origColIdxsForShadowCols) { + queryStmt.getResultExprs().add(queryStmt.getResultExprs().get(idx)); + } + } + // check compatibility for (int i = 0; i < targetColumns.size(); ++i) { Column column = targetColumns.get(i); if (column.getType().isHllType()) { @@ -402,14 +455,66 @@ public void analyzeSubquery(Analyzer analyzer) throws UserException { } } } + + // expand baseTblResultExprs and colLabels in QueryStmt + if (!origColIdxsForShadowCols.isEmpty()) { + if (queryStmt.getResultExprs().size() != queryStmt.getBaseTblResultExprs().size()) { + for (Integer idx : origColIdxsForShadowCols) { + queryStmt.getBaseTblResultExprs().add(queryStmt.getBaseTblResultExprs().get(idx)); + } + } + + if (queryStmt.getResultExprs().size() != queryStmt.getColLabels().size()) { + for (Integer idx : origColIdxsForShadowCols) { + queryStmt.getColLabels().add(queryStmt.getColLabels().get(idx)); + } + } + } + + if (LOG.isDebugEnabled()) { + for (Expr expr : queryStmt.getResultExprs()) { + LOG.debug("final result expr: {}, {}", expr, System.identityHashCode(expr)); + } + for (Expr expr : queryStmt.getBaseTblResultExprs()) { + LOG.debug("final base table result expr: {}, {}", expr, System.identityHashCode(expr)); + } + for (String colLabel : queryStmt.getColLabels()) { + LOG.debug("final col label: {}", colLabel); + } + } } - private void analyzeRow(Analyzer analyzer, List targetColumns, ArrayList row, int rowIdx) - throws AnalysisException { + private void analyzeRow(Analyzer analyzer, List targetColumns, List> rows, + int rowIdx, List origColIdxsForShadowCols) throws AnalysisException { // 1. check number of fields if equal with first row - if (row.size() != targetColumns.size()) { - throw new AnalysisException("Column count doesn't match value count at row " + rowIdx); + // targetColumns contains some shadow columns, which is added by system, + // so we should minus this + if (rows.get(rowIdx).size() != targetColumns.size() - origColIdxsForShadowCols.size()) { + throw new AnalysisException("Column count doesn't match value count at row " + (rowIdx + 1)); } + + ArrayList row = rows.get(rowIdx); + if (!origColIdxsForShadowCols.isEmpty()) { + /* + * we should extends the row for shadow columns. + * eg: + * the origin row has exprs: (expr1, expr2, expr3), and targetColumns is (A, B, C, __doris_shadow_b) + * after processing, extentedRow is (expr1, expr2, expr3, expr2) + */ + ArrayList extentedRow = Lists.newArrayList(); + for (Expr expr : row) { + extentedRow.add(expr); + } + + for (Integer idx : origColIdxsForShadowCols) { + extentedRow.add(extentedRow.get(idx)); + } + + row = extentedRow; + rows.set(rowIdx, row); + } + + // check the compatibility of expr in row and column in targetColumns for (int i = 0; i < row.size(); ++i) { Expr expr = row.get(i); Column col = targetColumns.get(i); @@ -496,7 +601,7 @@ public void prepareExpressions() throws UserException { exprByName.put(col.getName(), expr); } // reorder resultExprs in table column order - for (Column col : targetTable.getBaseSchema()) { + for (Column col : targetTable.getFullSchema()) { if (exprByName.containsKey(col.getName())) { resultExprs.add(exprByName.get(col.getName())); } else { diff --git a/fe/src/main/java/org/apache/doris/analysis/LoadStmt.java b/fe/src/main/java/org/apache/doris/analysis/LoadStmt.java index 99a2923549bc1a..021b15448c9253 100644 --- a/fe/src/main/java/org/apache/doris/analysis/LoadStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/LoadStmt.java @@ -197,13 +197,13 @@ public static void checkProperties(Map properties) throws DdlExc } - private void analyzeVersion() { + private void analyzeVersion() throws AnalysisException { if (properties == null) { return; } final String versionProperty = properties.get(VERSION); if (versionProperty != null) { - version = Load.VERSION; + throw new AnalysisException("Do not support VERSION property"); } } diff --git a/fe/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java b/fe/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java index 1bba28f30e2a6c..c7fe7880716bbc 100644 --- a/fe/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java @@ -87,7 +87,6 @@ public void analyze(Analyzer analyzer) throws AnalysisException, UserException { Preconditions.checkNotNull(type); // check auth when get job info - handleShowAlterTable(analyzer); } diff --git a/fe/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java b/fe/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java index 9de02136341ca0..dfd5dd516de29c 100644 --- a/fe/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java @@ -36,18 +36,21 @@ public class ShowTabletStmt extends ShowStmt { private String dbName; private String tableName; private long tabletId; + private String indexName; private boolean isShowSingleTablet; - public ShowTabletStmt(TableName dbTableName, long tabletId) { + public ShowTabletStmt(TableName dbTableName, long tabletId, String indexName) { if (dbTableName == null) { this.dbName = null; this.tableName = null; this.isShowSingleTablet = true; + this.indexName = null; } else { this.dbName = dbTableName.getDb(); this.tableName = dbTableName.getTbl(); this.isShowSingleTablet = false; + this.indexName = Strings.emptyToNull(indexName); } this.tabletId = tabletId; } @@ -68,6 +71,10 @@ public boolean isShowSingleTablet() { return isShowSingleTablet; } + public String getIndexName() { + return indexName; + } + @Override public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); @@ -93,7 +100,10 @@ public String toSql() { if (isShowSingleTablet) { sb.append(tabletId); } else { - sb.append("`").append(dbName).append("`.`").append(tableName).append("`"); + sb.append("FROM `").append(dbName).append("`.`").append(tableName).append("`"); + if (!Strings.isNullOrEmpty(indexName)) { + sb.append(" ROLLUP `").append(indexName).append("`"); + } } return sb.toString(); } diff --git a/fe/src/main/java/org/apache/doris/backup/BackupJob.java b/fe/src/main/java/org/apache/doris/backup/BackupJob.java index ee2cf15d9c577d..1ec86fbd0d9bc3 100644 --- a/fe/src/main/java/org/apache/doris/backup/BackupJob.java +++ b/fe/src/main/java/org/apache/doris/backup/BackupJob.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.FsBroker; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Replica; @@ -385,7 +386,7 @@ private void prepareAndSendSnapshotTask() { for (Partition partition : partitions) { long visibleVersion = partition.getVisibleVersion(); long visibleVersionHash = partition.getVisibleVersionHash(); - List indexes = partition.getMaterializedIndices(); + List indexes = partition.getMaterializedIndices(IndexExtState.VISIBLE); for (MaterializedIndex index : indexes) { int schemaHash = tbl.getSchemaHashByIndexId(index.getId()); List tablets = index.getTablets(); diff --git a/fe/src/main/java/org/apache/doris/backup/BackupJobInfo.java b/fe/src/main/java/org/apache/doris/backup/BackupJobInfo.java index c788659e0177aa..9f07fed710da41 100644 --- a/fe/src/main/java/org/apache/doris/backup/BackupJobInfo.java +++ b/fe/src/main/java/org/apache/doris/backup/BackupJobInfo.java @@ -19,6 +19,7 @@ import org.apache.doris.backup.RestoreFileMapping.IdChain; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Table; @@ -251,7 +252,7 @@ public static BackupJobInfo fromCatalog(long backupTime, String label, String db partitionInfo.versionHash = partition.getVisibleVersionHash(); tableInfo.partitions.put(partitionInfo.name, partitionInfo); // indexes - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { BackupIndexInfo idxInfo = new BackupIndexInfo(); idxInfo.id = index.getId(); idxInfo.name = olapTbl.getIndexNameById(index.getId()); diff --git a/fe/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/src/main/java/org/apache/doris/backup/RestoreJob.java index bb850ffc5f409a..8f368c34318d3d 100644 --- a/fe/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -30,6 +30,7 @@ import org.apache.doris.catalog.FsBroker; import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.OlapTable.OlapTableState; import org.apache.doris.catalog.Partition; @@ -610,7 +611,7 @@ private void checkAndPrepareMeta() { Set bfColumns = localTbl.getCopiedBfColumns(); double bfFpp = localTbl.getBfFpp(); - for (MaterializedIndex restoredIdx : restorePart.getMaterializedIndices()) { + for (MaterializedIndex restoredIdx : restorePart.getMaterializedIndices(IndexExtState.VISIBLE)) { short shortKeyColumnCount = localTbl.getShortKeyColumnCountByIndexId(restoredIdx.getId()); int schemaHash = localTbl.getSchemaHashByIndexId(restoredIdx.getId()); KeysType keysType = localTbl.getKeysType(); @@ -643,7 +644,7 @@ private void checkAndPrepareMeta() { for (Partition restorePart : restoreTbl.getPartitions()) { Set bfColumns = restoreTbl.getCopiedBfColumns(); double bfFpp = restoreTbl.getBfFpp(); - for (MaterializedIndex index : restorePart.getMaterializedIndices()) { + for (MaterializedIndex index : restorePart.getMaterializedIndices(IndexExtState.VISIBLE)) { short shortKeyColumnCount = restoreTbl.getShortKeyColumnCountByIndexId(index.getId()); int schemaHash = restoreTbl.getSchemaHashByIndexId(index.getId()); KeysType keysType = restoreTbl.getKeysType(); @@ -826,7 +827,7 @@ private Partition resetPartitionForRestore(OlapTable localTbl, OlapTable remoteT long visibleVersionHash = remotePart.getVisibleVersionHash(); // tablets - for (MaterializedIndex remoteIdx : remotePart.getMaterializedIndices()) { + for (MaterializedIndex remoteIdx : remotePart.getMaterializedIndices(IndexExtState.VISIBLE)) { int schemaHash = remoteTbl.getSchemaHashByIndexId(remoteIdx.getId()); int remotetabletSize = remoteIdx.getTablets().size(); remoteIdx.clearTabletsForRestore(); @@ -860,7 +861,7 @@ private Partition resetPartitionForRestore(OlapTable localTbl, OlapTable remoteT // files in repo to files in local private void genFileMapping(OlapTable localTbl, Partition localPartition, Long remoteTblId, BackupPartitionInfo backupPartInfo, boolean overwrite) { - for (MaterializedIndex localIdx : localPartition.getMaterializedIndices()) { + for (MaterializedIndex localIdx : localPartition.getMaterializedIndices(IndexExtState.VISIBLE)) { LOG.debug("get index id: {}, index name: {}", localIdx.getId(), localTbl.getIndexNameById(localIdx.getId())); BackupIndexInfo backupIdxInfo = backupPartInfo.getIdx(localTbl.getIndexNameById(localIdx.getId())); @@ -923,7 +924,7 @@ private void replayCheckAndPrepareMeta() { localTbl.addPartition(restorePart); // modify tablet inverted index - for (MaterializedIndex restoreIdx : restorePart.getMaterializedIndices()) { + for (MaterializedIndex restoreIdx : restorePart.getMaterializedIndices(IndexExtState.VISIBLE)) { int schemaHash = localTbl.getSchemaHashByIndexId(restoreIdx.getId()); TabletMeta tabletMeta = new TabletMeta(db.getId(), localTbl.getId(), restorePart.getId(), restoreIdx.getId(), schemaHash, TStorageMedium.HDD); @@ -941,7 +942,7 @@ private void replayCheckAndPrepareMeta() { db.createTable(restoreTbl); // modify tablet inverted index for (Partition restorePart : restoreTbl.getPartitions()) { - for (MaterializedIndex restoreIdx : restorePart.getMaterializedIndices()) { + for (MaterializedIndex restoreIdx : restorePart.getMaterializedIndices(IndexExtState.VISIBLE)) { int schemaHash = restoreTbl.getSchemaHashByIndexId(restoreIdx.getId()); TabletMeta tabletMeta = new TabletMeta(db.getId(), restoreTbl.getId(), restorePart.getId(), restoreIdx.getId(), schemaHash, TStorageMedium.HDD); @@ -1207,11 +1208,11 @@ private Status allTabletCommitted(boolean isReplay) { part.updateVersionForRestore(entry.getValue().first, entry.getValue().second); // we also need to update the replica version of these overwritten restored partitions - for (MaterializedIndex idx : part.getMaterializedIndices()) { + for (MaterializedIndex idx : part.getMaterializedIndices(IndexExtState.VISIBLE)) { for (Tablet tablet : idx.getTablets()) { for (Replica replica : tablet.getReplicas()) { if (!replica.checkVersionCatchUp(part.getVisibleVersion(), - part.getVisibleVersionHash())) { + part.getVisibleVersionHash(), false)) { replica.updateVersionInfo(part.getVisibleVersion(), part.getVisibleVersionHash(), replica.getDataSize(), replica.getRowCount()); } @@ -1337,7 +1338,7 @@ public void cancelInternal(boolean isReplay) { for (OlapTable restoreTbl : restoredTbls) { LOG.info("remove restored table when cancelled: {}", restoreTbl.getName()); for (Partition part : restoreTbl.getPartitions()) { - for (MaterializedIndex idx : part.getMaterializedIndices()) { + for (MaterializedIndex idx : part.getMaterializedIndices(IndexExtState.VISIBLE)) { for (Tablet tablet : idx.getTablets()) { Catalog.getCurrentInvertedIndex().deleteTablet(tablet.getId()); } @@ -1354,7 +1355,7 @@ public void cancelInternal(boolean isReplay) { } LOG.info("remove restored partition in table {} when cancelled: {}", restoreTbl.getName(), entry.second.getName()); - for (MaterializedIndex idx : entry.second.getMaterializedIndices()) { + for (MaterializedIndex idx : entry.second.getMaterializedIndices(IndexExtState.VISIBLE)) { for (Tablet tablet : idx.getTablets()) { Catalog.getCurrentInvertedIndex().deleteTablet(tablet.getId()); } diff --git a/fe/src/main/java/org/apache/doris/catalog/BrokerTable.java b/fe/src/main/java/org/apache/doris/catalog/BrokerTable.java index d3251ee81b5771..0e4d8852a5b8fc 100644 --- a/fe/src/main/java/org/apache/doris/catalog/BrokerTable.java +++ b/fe/src/main/java/org/apache/doris/catalog/BrokerTable.java @@ -202,7 +202,7 @@ private void validate(Map properties) throws DdlException { public TTableDescriptor toThrift() { TBrokerTable tBrokerTable = new TBrokerTable(); TTableDescriptor tTableDescriptor = new TTableDescriptor(getId(), TTableType.BROKER_TABLE, - baseSchema.size(), 0, getName(), ""); + fullSchema.size(), 0, getName(), ""); tTableDescriptor.setBrokerTable(tBrokerTable); return tTableDescriptor; } diff --git a/fe/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/src/main/java/org/apache/doris/catalog/Catalog.java index 26330385e36b6f..424aa62ccd1f6e 100644 --- a/fe/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/src/main/java/org/apache/doris/catalog/Catalog.java @@ -20,6 +20,7 @@ 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.RollupHandler; import org.apache.doris.alter.SchemaChangeHandler; @@ -76,6 +77,7 @@ import org.apache.doris.catalog.Database.DbState; import org.apache.doris.catalog.DistributionInfo.DistributionInfoType; import org.apache.doris.catalog.KuduPartition.KuduRange; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.MaterializedIndex.IndexState; import org.apache.doris.catalog.OlapTable.OlapTableState; import org.apache.doris.catalog.Replica.ReplicaState; @@ -1356,7 +1358,7 @@ private void recreateTabletInvertIndex() { long partitionId = partition.getId(); TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty( partitionId).getStorageMedium(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { long indexId = index.getId(); int schemaHash = olapTable.getSchemaHashByIndexId(indexId); TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partitionId, indexId, schemaHash, medium); @@ -1566,18 +1568,20 @@ 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(); + 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 = dis.readInt(); long newChecksum = checksum ^ size; @@ -1612,6 +1616,16 @@ public long loadAlterJob(DataInputStream dis, long checksum, JobType type) throw } } + // alter job v2 + if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_59) { + size = dis.readInt(); + newChecksum ^= size; + for (int i = 0; i < size; i++) { + AlterJobV2 alterJobV2 = AlterJobV2.read(dis); + alterJobsV2.put(alterJobV2.getJobId(), alterJobV2); + } + } + return newChecksum; } @@ -1918,12 +1932,15 @@ public long saveAlterJob(DataOutputStream dos, long checksum) throws IOException public long saveAlterJob(DataOutputStream 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(); @@ -1951,6 +1968,14 @@ public long saveAlterJob(DataOutputStream dos, long checksum, JobType type) thro alterJob.write(dos); } + // alter job v2 + size = alterJobsV2.size(); + checksum ^= size; + dos.writeInt(size); + for (AlterJobV2 alterJobV2 : alterJobsV2.values()) { + alterJobV2.write(dos); + } + return checksum; } @@ -3006,7 +3031,7 @@ public void replayAddPartition(PartitionPersistInfo info) throws DdlException { if (!isCheckpointThread()) { // add to inverted index TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { long indexId = index.getId(); int schemaHash = olapTable.getSchemaHashByIndexId(indexId); TabletMeta tabletMeta = new TabletMeta(info.getDbId(), info.getTableId(), partition.getId(), @@ -3265,6 +3290,7 @@ private Partition createPartitionWithIndices(String clusterName, long dbId, long // estimate timeout long timeout = Config.tablet_create_timeout_second * 1000L * totalTaskNum; + timeout = Math.min(timeout, Config.max_create_table_timeout_second * 1000); try { ok = countDownLatch.await(timeout, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { @@ -3272,19 +3298,25 @@ private Partition createPartitionWithIndices(String clusterName, long dbId, long ok = false; } - if (!ok) { - errMsg = "Failed to create partition[" + partitionName + "]. Timeout"; + if (!ok || !countDownLatch.getStatus().ok()) { + errMsg = "Failed to create partition[" + partitionName + "]. Timeout."; // clear tasks List tasks = batchTask.getAllTasks(); for (AgentTask task : tasks) { AgentTaskQueue.removeTask(task.getBackendId(), TTaskType.CREATE, task.getSignature()); } - List> unfinishedMarks = countDownLatch.getLeftMarks(); - // only show at most 10 results - List> subList = unfinishedMarks.subList(0, Math.min(unfinishedMarks.size(), 10)); - String idStr = Joiner.on(", ").join(subList); - LOG.warn("{}. unfinished marks: {}", errMsg, idStr); + if (!countDownLatch.getStatus().ok()) { + errMsg += " Error: " + countDownLatch.getStatus().getErrorMsg(); + } else { + List> unfinishedMarks = countDownLatch.getLeftMarks(); + // only show at most 3 results + List> subList = unfinishedMarks.subList(0, Math.min(unfinishedMarks.size(), 3)); + if (!subList.isEmpty()) { + errMsg += " Unfinished mark: " + Joiner.on(", ").join(subList); + } + } + LOG.warn(errMsg); throw new DdlException(errMsg); } } else { @@ -3983,7 +4015,7 @@ public void replayCreateTable(String dbName, Table table) { long partitionId = partition.getId(); TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty( partitionId).getStorageMedium(); - for (MaterializedIndex mIndex : partition.getMaterializedIndices()) { + for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { long indexId = mIndex.getId(); int schemaHash = olapTable.getSchemaHashByIndexId(indexId); TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partitionId, indexId, schemaHash, medium); @@ -4181,18 +4213,6 @@ public void replayRecoverTable(RecoverInfo info) { } } - public void handleJobsWhenDeleteReplica(long tableId, long partitionId, long indexId, long tabletId, long replicaId, - long backendId) { - // rollup - getRollupHandler().removeReplicaRelatedTask(tableId, partitionId, indexId, tabletId, backendId); - - // schema change - getSchemaChangeHandler().removeReplicaRelatedTask(tableId, tabletId, replicaId, backendId); - - // task - AgentTaskQueue.removeReplicaRelatedTasks(backendId, tabletId); - } - private void unprotectAddReplica(ReplicaPersistInfo info) { LOG.debug("replay add a replica {}", info); Database db = getDb(info.getDbId()); @@ -5644,7 +5664,7 @@ public Set getMigrations() { for (Partition partition : olapTable.getPartitions()) { final short replicationNum = olapTable.getPartitionInfo() .getReplicationNum(partition.getId()); - for (MaterializedIndex materializedIndex : partition.getMaterializedIndices()) { + for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { if (materializedIndex.getState() != IndexState.NORMAL) { continue; } @@ -6032,7 +6052,7 @@ private void truncateTableInternal(OlapTable olapTable, List newParti for (Partition newPartition : newPartitions) { Partition oldPartition = olapTable.replacePartition(newPartition); // save old tablets to be removed - for (MaterializedIndex index : oldPartition.getMaterializedIndices()) { + for (MaterializedIndex index : oldPartition.getMaterializedIndices(IndexExtState.ALL)) { index.getTablets().stream().forEach(t -> { oldTabletIds.add(t.getId()); }); @@ -6059,7 +6079,7 @@ public void replayTruncateTable(TruncateTableInfo info) { long partitionId = partition.getId(); TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty( partitionId).getStorageMedium(); - for (MaterializedIndex mIndex : partition.getMaterializedIndices()) { + for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { long indexId = mIndex.getId(); int schemaHash = olapTable.getSchemaHashByIndexId(indexId); TabletMeta tabletMeta = new TabletMeta(db.getId(), olapTable.getId(), diff --git a/fe/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java b/fe/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java index d14dfea83f413f..08dfc355608d23 100644 --- a/fe/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java +++ b/fe/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java @@ -17,6 +17,7 @@ package org.apache.doris.catalog; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.Table.TableType; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; @@ -208,7 +209,7 @@ private void onEraseOlapTable(OlapTable olapTable) { // inverted index TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); for (Partition partition : olapTable.getPartitions()) { - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { for (Tablet tablet : index.getTablets()) { invertedIndex.deleteTablet(tablet.getId()); } @@ -218,7 +219,7 @@ private void onEraseOlapTable(OlapTable olapTable) { // drop all replicas AgentBatchTask batchTask = new AgentBatchTask(); for (Partition partition : olapTable.getPartitions()) { - List allIndices = partition.getMaterializedIndices(); + List allIndices = partition.getMaterializedIndices(IndexExtState.ALL); for (MaterializedIndex materializedIndex : allIndices) { long indexId = materializedIndex.getId(); int schemaHash = olapTable.getSchemaHashByIndexId(indexId); @@ -272,7 +273,7 @@ public synchronized void replayEraseTable(long tableId) { // remove tablet from inverted index TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); for (Partition partition : olapTable.getPartitions()) { - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { for (Tablet tablet : index.getTablets()) { invertedIndex.deleteTablet(tablet.getId()); } @@ -297,7 +298,7 @@ private synchronized void erasePartition(long currentTimeMs) { if (isExpire(partitionId, currentTimeMs)) { // remove tablet in inverted index TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { for (Tablet tablet : index.getTablets()) { invertedIndex.deleteTablet(tablet.getId()); } @@ -327,7 +328,7 @@ private synchronized void erasePartitionWithSameName(long dbId, long tableId, St if (partition.getName().equals(partitionName)) { // remove tablet in inverted index TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { for (Tablet tablet : index.getTablets()) { invertedIndex.deleteTablet(tablet.getId()); } @@ -349,7 +350,7 @@ public synchronized void replayErasePartition(long partitionId) { if (!Catalog.isCheckpointThread()) { // remove tablet from inverted index TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { for (Tablet tablet : index.getTablets()) { invertedIndex.deleteTablet(tablet.getId()); } @@ -581,7 +582,7 @@ public void addTabletToInvertedIndex() { for (Partition partition : olapTable.getPartitions()) { long partitionId = partition.getId(); TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty(partitionId).getStorageMedium(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { long indexId = index.getId(); int schemaHash = olapTable.getSchemaHashByIndexId(indexId); TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partitionId, indexId, schemaHash, medium); @@ -633,7 +634,7 @@ public void addTabletToInvertedIndex() { // storage medium should be got from RecyclePartitionInfo, not from olap table. because olap table // does not have this partition any more TStorageMedium medium = partitionInfo.getDataProperty().getStorageMedium(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { long indexId = index.getId(); int schemaHash = olapTable.getSchemaHashByIndexId(indexId); TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partitionId, indexId, schemaHash, medium); diff --git a/fe/src/main/java/org/apache/doris/catalog/Column.java b/fe/src/main/java/org/apache/doris/catalog/Column.java index 52067b9434fded..7fc66369ec5da8 100644 --- a/fe/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/src/main/java/org/apache/doris/catalog/Column.java @@ -17,6 +17,8 @@ package org.apache.doris.catalog; +import org.apache.doris.alter.SchemaChangeHandler; +import org.apache.doris.common.CaseSensibility; import org.apache.doris.common.DdlException; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.io.Text; @@ -118,6 +120,17 @@ public String getName() { return this.name; } + public String getNameWithoutPrefix(String prefix) { + if (isNameWithPrefix(prefix)) { + return name.substring(prefix.length()); + } + return name; + } + + public boolean isNameWithPrefix(String prefix) { + return this.name.startsWith(prefix); + } + public void setIsKey(boolean isKey) { this.isKey = isKey; } @@ -214,24 +227,24 @@ public void checkSchemaChangeAllowed(Column other) throws DdlException { } if (!ColumnType.isSchemaChangeAllowed(type, other.type)) { - throw new DdlException("Cannot change " + getDataType() + " to " + other.getDataType()); + throw new DdlException("Can not change " + getDataType() + " to " + other.getDataType()); } if (this.aggregationType != other.aggregationType) { - throw new DdlException("Cannot change aggregation type"); + throw new DdlException("Can not change aggregation type"); } if (this.isAllowNull && !other.isAllowNull) { - throw new DdlException("Cannot change from null to not null"); + throw new DdlException("Can not change from nullable to non-nullable"); } if (this.getDefaultValue() == null) { if (other.getDefaultValue() != null) { - throw new DdlException("Cannot change default value"); + throw new DdlException("Can not change default value"); } } else { if (!this.getDefaultValue().equals(other.getDefaultValue())) { - throw new DdlException("Cannot change default value"); + throw new DdlException("Can not change default value"); } } @@ -252,6 +265,29 @@ public void checkSchemaChangeAllowed(Column other) throws DdlException { } } + public boolean nameEquals(String otherColName, boolean ignorePrefix) { + if (CaseSensibility.COLUMN.getCaseSensibility()) { + if (!ignorePrefix) { + return name.equals(otherColName); + } else { + return removeNamePrefix(name).equals(removeNamePrefix(otherColName)); + } + } else { + if (!ignorePrefix) { + return name.equalsIgnoreCase(otherColName); + } else { + return removeNamePrefix(name).equalsIgnoreCase(removeNamePrefix(otherColName)); + } + } + } + + public static String removeNamePrefix(String colName) { + if (colName.startsWith(SchemaChangeHandler.SHADOW_NAME_PRFIX)) { + return colName.substring(SchemaChangeHandler.SHADOW_NAME_PRFIX.length()); + } + return colName; + } + public String toSql() { StringBuilder sb = new StringBuilder(); sb.append("`").append(name).append("` "); diff --git a/fe/src/main/java/org/apache/doris/catalog/Database.java b/fe/src/main/java/org/apache/doris/catalog/Database.java index ea206094aad0ea..1b43ce11d16962 100644 --- a/fe/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/src/main/java/org/apache/doris/catalog/Database.java @@ -17,7 +17,7 @@ package org.apache.doris.catalog; -import com.google.common.collect.Lists; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.MaterializedIndex.IndexState; import org.apache.doris.catalog.Replica.ReplicaState; import org.apache.doris.catalog.Table.TableType; @@ -35,6 +35,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.logging.log4j.LogManager; @@ -206,7 +207,7 @@ public long getDataQuotaLeftWithLock() { OlapTable olapTable = (OlapTable) table; for (Partition partition : olapTable.getPartitions()) { - for (MaterializedIndex mIndex : partition.getMaterializedIndices()) { + for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { // skip ROLLUP index if (mIndex.getState() == IndexState.ROLLUP) { continue; diff --git a/fe/src/main/java/org/apache/doris/catalog/EsTable.java b/fe/src/main/java/org/apache/doris/catalog/EsTable.java index 8713ac6ce1271b..bf5c420f804c55 100644 --- a/fe/src/main/java/org/apache/doris/catalog/EsTable.java +++ b/fe/src/main/java/org/apache/doris/catalog/EsTable.java @@ -121,7 +121,7 @@ private void validate(Map properties) throws DdlException { public TTableDescriptor toThrift() { TEsTable tEsTable = new TEsTable(); TTableDescriptor tTableDescriptor = new TTableDescriptor(getId(), TTableType.ES_TABLE, - baseSchema.size(), 0, getName(), ""); + fullSchema.size(), 0, getName(), ""); tTableDescriptor.setEsTable(tEsTable); return tTableDescriptor; } diff --git a/fe/src/main/java/org/apache/doris/catalog/MaterializedIndex.java b/fe/src/main/java/org/apache/doris/catalog/MaterializedIndex.java index 61234be1593890..3324006a90b34b 100644 --- a/fe/src/main/java/org/apache/doris/catalog/MaterializedIndex.java +++ b/fe/src/main/java/org/apache/doris/catalog/MaterializedIndex.java @@ -38,7 +38,18 @@ public class MaterializedIndex extends MetaObject implements Writable { public enum IndexState { NORMAL, ROLLUP, - SCHEMA_CHANGE + SCHEMA_CHANGE, + SHADOW; // index in SHADOW state is visible to load process, but invisible to query + + public boolean isVisible() { + return this == IndexState.NORMAL || this == IndexState.SCHEMA_CHANGE; + } + } + + public enum IndexExtState { + ALL, + VISIBLE, // index state in NORMAL and SCHEMA_CHANGE + SHADOW // index state in SHADOW } private long id; diff --git a/fe/src/main/java/org/apache/doris/catalog/MetadataViewer.java b/fe/src/main/java/org/apache/doris/catalog/MetadataViewer.java index 05e56e85603e5e..55a1657b27b4a2 100644 --- a/fe/src/main/java/org/apache/doris/catalog/MetadataViewer.java +++ b/fe/src/main/java/org/apache/doris/catalog/MetadataViewer.java @@ -20,6 +20,7 @@ import org.apache.doris.analysis.AdminShowReplicaDistributionStmt; import org.apache.doris.analysis.AdminShowReplicaStatusStmt; import org.apache.doris.analysis.BinaryPredicate.Operator; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.Replica.ReplicaStatus; import org.apache.doris.catalog.Table.TableType; import org.apache.doris.common.DdlException; @@ -79,7 +80,7 @@ private static List> getTabletStatus(String dbName, String tblName, long visibleVersion = partition.getVisibleVersion(); short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId()); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { int schemaHash = olapTable.getSchemaHashByIndexId(index.getId()); for (Tablet tablet : index.getTablets()) { long tabletId = tablet.getId(); @@ -210,7 +211,7 @@ private static List> getTabletDistribution(String dbName, String tb int totalReplicaNum = 0; for (String partName : partitions) { Partition partition = olapTable.getPartition(partName); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { for (Tablet tablet : index.getTablets()) { for (Replica replica : tablet.getReplicas()) { if (!countMap.containsKey(replica.getBackendId())) { diff --git a/fe/src/main/java/org/apache/doris/catalog/MysqlTable.java b/fe/src/main/java/org/apache/doris/catalog/MysqlTable.java index e60a8749697c19..a31872faf4c89f 100644 --- a/fe/src/main/java/org/apache/doris/catalog/MysqlTable.java +++ b/fe/src/main/java/org/apache/doris/catalog/MysqlTable.java @@ -144,7 +144,7 @@ public TTableDescriptor toThrift() { TMySQLTable tMySQLTable = new TMySQLTable(host, port, userName, passwd, mysqlDatabaseName, mysqlTableName); TTableDescriptor tTableDescriptor = new TTableDescriptor(getId(), TTableType.MYSQL_TABLE, - baseSchema.size(), 0, getName(), ""); + fullSchema.size(), 0, getName(), ""); tTableDescriptor.setMysqlTable(tMySQLTable); return tTableDescriptor; } diff --git a/fe/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/src/main/java/org/apache/doris/catalog/OlapTable.java index 3467f0984eff30..122da79f88ae49 100644 --- a/fe/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -30,6 +30,7 @@ import org.apache.doris.backup.Status; import org.apache.doris.backup.Status.ErrCode; import org.apache.doris.catalog.DistributionInfo.DistributionInfoType; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.MaterializedIndex.IndexState; import org.apache.doris.catalog.Partition.PartitionState; import org.apache.doris.catalog.Replica.ReplicaState; @@ -214,6 +215,9 @@ public boolean hasMaterializedIndex(String indexName) { return indexNameToId.containsKey(indexName); } + /* + * Set index schema info for specified index. + */ public void setIndexSchemaInfo(Long indexId, String indexName, List schema, int schemaVersion, int schemaHash, short shortKeyColumnCount) { if (indexName == null) { @@ -226,19 +230,40 @@ public void setIndexSchemaInfo(Long indexId, String indexName, List sche indexIdToSchemaHash.put(indexId, schemaHash); indexIdToShortKeyColumnCount.put(indexId, shortKeyColumnCount); } + public void setIndexStorageType(Long indexId, TStorageType newStorageType) { Preconditions.checkState(newStorageType == TStorageType.COLUMN); indexIdToStorageType.put(indexId, newStorageType); } - public void deleteIndexInfo(String indexName) { - long indexId = this.indexNameToId.remove(indexName); + // rebuild the full schema of table + // the order of columns in fullSchema is meaningless + public void rebuildFullSchema() { + fullSchema.clear(); + nameToColumn.clear(); + for (List columns : indexIdToSchema.values()) { + for (Column column : columns) { + if (!nameToColumn.containsKey(column.getName())) { + fullSchema.add(column); + nameToColumn.put(column.getName(), column); + } + } + } + LOG.debug("after rebuild full schema. table {}, schema: {}", id, fullSchema); + } + + public boolean deleteIndexInfo(String indexName) { + if (!indexNameToId.containsKey(indexName)) { + return false; + } + long indexId = this.indexNameToId.remove(indexName); indexIdToSchema.remove(indexId); indexIdToSchemaVersion.remove(indexId); indexIdToSchemaHash.remove(indexId); indexIdToShortKeyColumnCount.remove(indexId); indexIdToStorageType.remove(indexId); + return true; } public Map getIndexNameToId() { @@ -258,6 +283,19 @@ public String getIndexNameById(long indexId) { return null; } + // this is only for schema change. + public void renameIndexForSchemaChange(String name, String newName) { + long idxId = indexNameToId.remove(name); + indexNameToId.put(newName, idxId); + } + + public void renameColumnNamePrefix(long idxId) { + List columns = indexIdToSchema.get(idxId); + for (Column column : columns) { + column.setName(Column.removeNamePrefix(column.getName())); + } + } + public Status resetIdsForRestore(Catalog catalog, Database db, int restoreReplicationNum) { // table id id = catalog.getNextId(); @@ -572,7 +610,7 @@ public boolean shouldLoadToNewRollup() { public TTableDescriptor toThrift() { TOlapTable tOlapTable = new TOlapTable(getName()); TTableDescriptor tTableDescriptor = new TTableDescriptor(id, TTableType.OLAP_TABLE, - baseSchema.size(), 0, getName(), ""); + fullSchema.size(), 0, getName(), ""); tTableDescriptor.setOlapTable(tOlapTable); return tTableDescriptor; } @@ -932,7 +970,7 @@ public OlapTable selectiveCopy(Collection reservedPartNames, boolean res for (Partition partition : copied.getPartitions()) { partition.setState(PartitionState.NORMAL); copied.getPartitionInfo().setDataProperty(partition.getId(), new DataProperty(TStorageMedium.HDD)); - for (MaterializedIndex idx : partition.getMaterializedIndices()) { + for (MaterializedIndex idx : partition.getMaterializedIndices(IndexExtState.ALL)) { idx.setState(IndexState.NORMAL); for (Tablet tablet : idx.getTablets()) { for (Replica replica : tablet.getReplicas()) { @@ -1003,7 +1041,7 @@ public boolean isStable(SystemInfoService infoService, TabletScheduler tabletSch long visibleVersion = partition.getVisibleVersion(); long visibleVersionHash = partition.getVisibleVersionHash(); short replicationNum = partitionInfo.getReplicationNum(partition.getId()); - for (MaterializedIndex mIndex : partition.getMaterializedIndices()) { + for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { for (Tablet tablet : mIndex.getTablets()) { if (tabletScheduler.containsTablet(tablet.getId())) { return false; @@ -1053,11 +1091,11 @@ public long proximateRowCount() { for (Partition partition : getPartitions()) { long version = partition.getVisibleVersion(); long versionHash = partition.getVisibleVersionHash(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { for (Tablet tablet : index.getTablets()) { long tabletRowCount = 0L; for (Replica replica : tablet.getReplicas()) { - if (replica.checkVersionCatchUp(version, versionHash) + if (replica.checkVersionCatchUp(version, versionHash, false) && replica.getRowCount() > tabletRowCount) { tabletRowCount = replica.getRowCount(); } @@ -1068,4 +1106,19 @@ public long proximateRowCount() { } return totalCount; } + + @Override + public List getBaseSchema() { + return indexIdToSchema.get(baseIndexId); + } + + public int getKeysNum() { + int keysNum = 0; + for (Column column : getBaseSchema()) { + if (column.isKey()) { + keysNum += 1; + } + } + return keysNum; + } } diff --git a/fe/src/main/java/org/apache/doris/catalog/Partition.java b/fe/src/main/java/org/apache/doris/catalog/Partition.java index 4dec9661808b64..f14c3fb3d661a8 100644 --- a/fe/src/main/java/org/apache/doris/catalog/Partition.java +++ b/fe/src/main/java/org/apache/doris/catalog/Partition.java @@ -18,20 +18,24 @@ package org.apache.doris.catalog; import org.apache.doris.catalog.DistributionInfo.DistributionInfoType; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; +import org.apache.doris.catalog.MaterializedIndex.IndexState; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.Util; import org.apache.doris.meta.MetaContext; +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; + +import org.apache.kudu.client.shaded.com.google.common.collect.Lists; 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.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -45,6 +49,7 @@ public class Partition extends MetaObject implements Writable { public static final long PARTITION_INIT_VERSION = 1L; public static final long PARTITION_INIT_VERSION_HASH = 0L; + @Deprecated public enum PartitionState { NORMAL, ROLLUP, @@ -56,7 +61,17 @@ public enum PartitionState { private PartitionState state; private MaterializedIndex baseIndex; - private Map idToRollupIndex; + /* + * Visible rollup indexes are indexes which are visible to user. + * User can do query on them, show them in related 'show' stmt. + */ + private Map idToVisibleRollupIndex = Maps.newHashMap(); + /* + * Shadow indexes are indexes which are not visible to user. + * Query will not run on these shadow indexes, and user can not see them neither. + * But load process will load data into these shadow indexes. + */ + private Map idToShadowIndex = Maps.newHashMap(); /* * committed version(hash): after txn is committed, set committed version(hash) @@ -73,8 +88,7 @@ public enum PartitionState { private DistributionInfo distributionInfo; - public Partition() { - this.idToRollupIndex = new HashMap(); + private Partition() { } public Partition(long id, String name, @@ -84,7 +98,6 @@ public Partition(long id, String name, this.state = PartitionState.NORMAL; this.baseIndex = baseIndex; - this.idToRollupIndex = new HashMap(); this.visibleVersion = PARTITION_INIT_VERSION; this.visibleVersionHash = PARTITION_INIT_VERSION_HASH; @@ -168,11 +181,19 @@ public DistributionInfo getDistributionInfo() { } public void createRollupIndex(MaterializedIndex mIndex) { - this.idToRollupIndex.put(mIndex.getId(), mIndex); + if (mIndex.getState().isVisible()) { + this.idToVisibleRollupIndex.put(mIndex.getId(), mIndex); + } else { + this.idToShadowIndex.put(mIndex.getId(), mIndex); + } } public MaterializedIndex deleteRollupIndex(long indexId) { - return this.idToRollupIndex.remove(indexId); + if (this.idToVisibleRollupIndex.containsKey(indexId)) { + return idToVisibleRollupIndex.remove(indexId); + } else { + return idToShadowIndex.remove(indexId); + } } public MaterializedIndex getBaseIndex() { @@ -204,36 +225,40 @@ public long getCommittedVersionHash() { return committedVersionHash; } - public List getRollupIndices() { - List rollupIndices = new ArrayList(idToRollupIndex.size()); - for (Map.Entry entry : idToRollupIndex.entrySet()) { - rollupIndices.add(entry.getValue()); - } - return rollupIndices; - } - public MaterializedIndex getIndex(long indexId) { if (baseIndex.getId() == indexId) { return baseIndex; } - if (idToRollupIndex.containsKey(indexId)) { - return idToRollupIndex.get(indexId); + if (idToVisibleRollupIndex.containsKey(indexId)) { + return idToVisibleRollupIndex.get(indexId); + } else { + return idToShadowIndex.get(indexId); } - return null; } - public List getMaterializedIndices() { - List indices = new ArrayList(); - indices.add(baseIndex); - for (MaterializedIndex rollupIndex : idToRollupIndex.values()) { - indices.add(rollupIndex); + public List getMaterializedIndices(IndexExtState extState) { + List indices = Lists.newArrayList(); + switch (extState) { + case ALL: + indices.add(baseIndex); + indices.addAll(idToVisibleRollupIndex.values()); + indices.addAll(idToShadowIndex.values()); + break; + case VISIBLE: + indices.add(baseIndex); + indices.addAll(idToVisibleRollupIndex.values()); + break; + case SHADOW: + indices.addAll(idToShadowIndex.values()); + default: + break; } return indices; } public long getDataSize() { long dataSize = 0; - for (MaterializedIndex mIndex : getMaterializedIndices()) { + for (MaterializedIndex mIndex : getMaterializedIndices(IndexExtState.VISIBLE)) { dataSize += mIndex.getDataSize(); } return dataSize; @@ -243,6 +268,26 @@ public boolean hasData() { return !(visibleVersion == PARTITION_INIT_VERSION && visibleVersionHash == PARTITION_INIT_VERSION_HASH); } + /* + * Change the index' state from SHADOW to NORMAL + * Also move it to idToVisibleRollupIndex if it is not the base index. + */ + public boolean visualiseShadowIndex(long shadowIndexId, boolean isBaseIndex) { + MaterializedIndex shadowIdx = idToShadowIndex.remove(shadowIndexId); + if (shadowIdx == null) { + return false; + } + Preconditions.checkState(!idToVisibleRollupIndex.containsKey(shadowIndexId), shadowIndexId); + shadowIdx.setState(IndexState.NORMAL); + if (isBaseIndex) { + baseIndex = shadowIdx; + } else { + idToVisibleRollupIndex.put(shadowIndexId, shadowIdx); + } + LOG.info("visualise the shadow index: {}", shadowIndexId); + return true; + } + public static Partition read(DataInput in) throws IOException { Partition partition = new Partition(); partition.readFields(in); @@ -259,14 +304,19 @@ public void write(DataOutput out) throws IOException { baseIndex.write(out); - int rollupCount = (idToRollupIndex != null) ? idToRollupIndex.size() : 0; + int rollupCount = (idToVisibleRollupIndex != null) ? idToVisibleRollupIndex.size() : 0; out.writeInt(rollupCount); - if (idToRollupIndex != null) { - for (Map.Entry entry : idToRollupIndex.entrySet()) { + if (idToVisibleRollupIndex != null) { + for (Map.Entry entry : idToVisibleRollupIndex.entrySet()) { entry.getValue().write(out); } } + out.writeInt(idToShadowIndex.size()); + for (MaterializedIndex shadowIndex : idToShadowIndex.values()) { + shadowIndex.write(out); + } + out.writeLong(visibleVersion); out.writeLong(visibleVersionHash); @@ -291,7 +341,15 @@ public void readFields(DataInput in) throws IOException { int rollupCount = in.readInt(); for (int i = 0; i < rollupCount; ++i) { MaterializedIndex rollupTable = MaterializedIndex.read(in); - idToRollupIndex.put(rollupTable.getId(), rollupTable); + idToVisibleRollupIndex.put(rollupTable.getId(), rollupTable); + } + + if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_59) { + int shadowIndexCount = in.readInt(); + for (int i = 0; i < shadowIndexCount; i++) { + MaterializedIndex shadowIndex = MaterializedIndex.read(in); + idToShadowIndex.put(shadowIndex.getId(), shadowIndex); + } } visibleVersion = in.readLong(); @@ -332,16 +390,16 @@ public boolean equals(Object obj) { } Partition partition = (Partition) obj; - if (idToRollupIndex != partition.idToRollupIndex) { - if (idToRollupIndex.size() != partition.idToRollupIndex.size()) { + if (idToVisibleRollupIndex != partition.idToVisibleRollupIndex) { + if (idToVisibleRollupIndex.size() != partition.idToVisibleRollupIndex.size()) { return false; } - for (Entry entry : idToRollupIndex.entrySet()) { + for (Entry entry : idToVisibleRollupIndex.entrySet()) { long key = entry.getKey(); - if (!partition.idToRollupIndex.containsKey(key)) { + if (!partition.idToVisibleRollupIndex.containsKey(key)) { return false; } - if (!entry.getValue().equals(partition.idToRollupIndex.get(key))) { + if (!entry.getValue().equals(partition.idToVisibleRollupIndex.get(key))) { return false; } } @@ -362,11 +420,11 @@ public String toString() { buffer.append("base_index: ").append(baseIndex.toString()).append("; "); - int rollupCount = (idToRollupIndex != null) ? idToRollupIndex.size() : 0; + int rollupCount = (idToVisibleRollupIndex != null) ? idToVisibleRollupIndex.size() : 0; buffer.append("rollup count: ").append(rollupCount).append("; "); - if (idToRollupIndex != null) { - for (Map.Entry entry : idToRollupIndex.entrySet()) { + if (idToVisibleRollupIndex != null) { + for (Map.Entry entry : idToVisibleRollupIndex.entrySet()) { buffer.append("rollup_index: ").append(entry.getValue().toString()).append("; "); } } diff --git a/fe/src/main/java/org/apache/doris/catalog/Replica.java b/fe/src/main/java/org/apache/doris/catalog/Replica.java index 815a2008979008..5b44ddfdb639c2 100644 --- a/fe/src/main/java/org/apache/doris/catalog/Replica.java +++ b/fe/src/main/java/org/apache/doris/catalog/Replica.java @@ -39,13 +39,20 @@ public class Replica implements Writable { public enum ReplicaState { NORMAL, + @Deprecated ROLLUP, + @Deprecated SCHEMA_CHANGE, CLONE, + ALTER, // replica is under rollup or schema change DECOMMISSION; // replica is ready to be deleted - public boolean isLoadable() { - return this == ReplicaState.NORMAL || this == ReplicaState.SCHEMA_CHANGE; + public boolean canLoad() { + return this == NORMAL || this == SCHEMA_CHANGE || this == ALTER; + } + + public boolean canQuery() { + return this == NORMAL || this == SCHEMA_CHANGE; } } @@ -379,7 +386,20 @@ public synchronized void updateLastFailedVersion(long lastFailedVersion, long la this.lastSuccessVersion, this.lastSuccessVersionHash, dataSize, rowCount); } - public boolean checkVersionCatchUp(long expectedVersion, long expectedVersionHash) { + /* + * Check whether the replica's version catch up with the expected version. + * If ignoreAlter is true, and state is ALTER, and replica's version is PARTITION_INIT_VERSION, just return true, ignore the version. + * This is for the case that when altering table, the newly created replica's version is PARTITION_INIT_VERSION, + * but we need to treat it as a "normal" replica which version is supposed to be "catch-up". + * But if state is ALTER but version larger than PARTITION_INIT_VERSION, which means this replica + * is already updated by load process, so we need to consider its version. + */ + public boolean checkVersionCatchUp(long expectedVersion, long expectedVersionHash, boolean ignoreAlter) { + if (ignoreAlter && state == ReplicaState.ALTER && version == Partition.PARTITION_INIT_VERSION + && versionHash == Partition.PARTITION_INIT_VERSION_HASH) { + return true; + } + if (expectedVersion == Partition.PARTITION_INIT_VERSION && expectedVersionHash == Partition.PARTITION_INIT_VERSION_HASH) { // no data is loaded into this replica, just return true diff --git a/fe/src/main/java/org/apache/doris/catalog/Table.java b/fe/src/main/java/org/apache/doris/catalog/Table.java index aedad489ff059d..69e1ba2c176bc8 100644 --- a/fe/src/main/java/org/apache/doris/catalog/Table.java +++ b/fe/src/main/java/org/apache/doris/catalog/Table.java @@ -24,6 +24,7 @@ import org.apache.doris.thrift.TTableDescriptor; import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.commons.lang.NotImplementedException; @@ -33,7 +34,6 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -57,8 +57,17 @@ public enum TableType { protected long id; protected String name; protected TableType type; - protected List baseSchema; - // tree map for case-insensitive lookup + /* + * fullSchema and nameToColumn should contains all columns, both visible and shadow. + * eg. for OlapTable, when doing schema change, there will be some shadow columns which are not visible + * to query but visible to load process. + * If you want to get all visible columns, you should call getBaseSchema() method, which is override in + * sub classes. + * + * NOTICE: the order of this fullSchema is meaningless to OlapTable + */ + protected List fullSchema; + // tree map for case-insensitive lookup. protected Map nameToColumn; // DO NOT persist this variable. @@ -66,19 +75,21 @@ public enum TableType { public Table(TableType type) { this.type = type; - this.baseSchema = new LinkedList(); + this.fullSchema = Lists.newArrayList(); this.nameToColumn = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); } - public Table(long id, String tableName, TableType type, List baseSchema) { + public Table(long id, String tableName, TableType type, List fullSchema) { this.id = id; this.name = tableName; this.type = type; - this.baseSchema = baseSchema; - + // must copy the list, it should not be the same object as in indexIdToSchmea + if (fullSchema != null) { + this.fullSchema = Lists.newArrayList(fullSchema); + } this.nameToColumn = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); - if (baseSchema != null) { - for (Column col : baseSchema) { + if (this.fullSchema != null) { + for (Column col : this.fullSchema) { nameToColumn.put(col.getName(), col); } } else { @@ -107,24 +118,19 @@ public TableType getType() { return type; } - public int getKeysNum() { - int keysNum = 0; - for (Column column : baseSchema) { - if (column.isKey()) { - keysNum += 1; - } - } - return keysNum; + public List getFullSchema() { + return fullSchema; } + // should override in subclass if necessary public List getBaseSchema() { - return baseSchema; + return fullSchema; } - public void setNewBaseSchema(List newSchema) { - this.baseSchema = newSchema; + public void setNewFullSchema(List newSchema) { + this.fullSchema = newSchema; this.nameToColumn.clear(); - for (Column col : baseSchema) { + for (Column col : fullSchema) { nameToColumn.put(col.getName(), col); } } @@ -182,9 +188,9 @@ public void write(DataOutput out) throws IOException { Text.writeString(out, name); // base schema - int columnCount = baseSchema.size(); + int columnCount = fullSchema.size(); out.writeInt(columnCount); - for (Column column : baseSchema) { + for (Column column : fullSchema) { column.write(out); } } @@ -205,7 +211,7 @@ public void readFields(DataInput in) throws IOException { int columnCount = in.readInt(); for (int i = 0; i < columnCount; i++) { Column column = Column.read(in); - this.baseSchema.add(column); + this.fullSchema.add(column); this.nameToColumn.put(column.getName(), column); } } diff --git a/fe/src/main/java/org/apache/doris/catalog/Tablet.java b/fe/src/main/java/org/apache/doris/catalog/Tablet.java index f3d73f83a011d7..2c6c428d5d1c8a 100644 --- a/fe/src/main/java/org/apache/doris/catalog/Tablet.java +++ b/fe/src/main/java/org/apache/doris/catalog/Tablet.java @@ -176,8 +176,7 @@ public List getNormalReplicaBackendIds() { } ReplicaState state = replica.getState(); - if (infoService.checkBackendAlive(replica.getBackendId()) - && (state == ReplicaState.NORMAL || state == ReplicaState.SCHEMA_CHANGE)) { + if (infoService.checkBackendAlive(replica.getBackendId()) && state.canLoad()) { beIds.add(replica.getBackendId()); } } @@ -193,9 +192,9 @@ public void getQueryableReplicas(List allQuerableReplica, List } ReplicaState state = replica.getState(); - if (state == ReplicaState.NORMAL || state == ReplicaState.SCHEMA_CHANGE) { + if (state.canQuery()) { // replica.getSchemaHash() == -1 is for compatibility - if (replica.checkVersionCatchUp(visibleVersion, visibleVersionHash) + if (replica.checkVersionCatchUp(visibleVersion, visibleVersionHash, false) && (replica.getSchemaHash() == -1 || replica.getSchemaHash() == schemaHash)) { allQuerableReplica.add(replica); if (localBeId != -1 && replica.getBackendId() == localBeId) { diff --git a/fe/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java b/fe/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java index a6006620554bff..9cdee89fbe2668 100644 --- a/fe/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java +++ b/fe/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java @@ -347,6 +347,12 @@ private boolean needSync(Replica replicaInFe, TTabletInfo backendTabletInfo) { // it will be handled in needRecovery() return false; } + + if (replicaInFe.getState() == ReplicaState.ALTER) { + // ignore the replica is ALTER state. its version will be taken care by load process and alter table process + return false; + } + long versionInFe = replicaInFe.getVersion(); long versionHashInFe = replicaInFe.getVersionHash(); diff --git a/fe/src/main/java/org/apache/doris/catalog/TabletStatMgr.java b/fe/src/main/java/org/apache/doris/catalog/TabletStatMgr.java index 67911c78f7b458..a2fcd62d055e92 100644 --- a/fe/src/main/java/org/apache/doris/catalog/TabletStatMgr.java +++ b/fe/src/main/java/org/apache/doris/catalog/TabletStatMgr.java @@ -17,6 +17,7 @@ package org.apache.doris.catalog; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.Table.TableType; import org.apache.doris.common.ClientPool; import org.apache.doris.common.Config; @@ -119,12 +120,12 @@ protected void runOneCycle() { for (Partition partition : olapTable.getPartitions()) { long version = partition.getVisibleVersion(); long versionHash = partition.getVisibleVersionHash(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { long indexRowCount = 0L; for (Tablet tablet : index.getTablets()) { long tabletRowCount = 0L; for (Replica replica : tablet.getReplicas()) { - if (replica.checkVersionCatchUp(version, versionHash) + if (replica.checkVersionCatchUp(version, versionHash, false) && replica.getRowCount() > tabletRowCount) { tabletRowCount = replica.getRowCount(); } diff --git a/fe/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java b/fe/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java index 19707dc884dbfb..60a092c861b4c2 100644 --- a/fe/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java +++ b/fe/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.ColocateTableIndex.GroupId; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Replica; @@ -233,7 +234,7 @@ private long selectSubstituteBackend(int tabletOrderIdx, GroupId groupId, long u } for (Partition partition : tbl.getPartitions()) { - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { long tabletId = index.getTabletIdsInOrder().get(tabletOrderIdx); Tablet tablet = index.getTablet(tabletId); Replica replica = tablet.getReplicaByBackendId(unavailableBeId); @@ -344,7 +345,9 @@ private void matchGroup() { short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId()); long visibleVersion = partition.getVisibleVersion(); long visibleVersionHash = partition.getVisibleVersionHash(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + // Here we only get VISIBLE indexes. All other indexes are not queryable. + // So it does not matter if tablets of other indexes are not matched. + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { Preconditions.checkState(backendBucketsSeq.size() == index.getTablets().size(), backendBucketsSeq.size() + " vs. " + index.getTablets().size()); int idx = 0; diff --git a/fe/src/main/java/org/apache/doris/clone/LoadBalancer.java b/fe/src/main/java/org/apache/doris/clone/LoadBalancer.java index 77d3ac9a2814e2..16baa7a7cface8 100644 --- a/fe/src/main/java/org/apache/doris/clone/LoadBalancer.java +++ b/fe/src/main/java/org/apache/doris/clone/LoadBalancer.java @@ -82,6 +82,9 @@ public List selectAlternativeTablets() { * * Here we only select tablets from high load node, do not set its src or dest, all this will be set * when this tablet is being scheduled in tablet scheduler. + * + * NOTICE that we may select any available tablets here, ignore their state. + * The state will be checked when being scheduled in tablet scheduler. */ private List selectAlternativeTabletsForCluster( String clusterName, ClusterLoadStatistic clusterStat, TStorageMedium medium) { diff --git a/fe/src/main/java/org/apache/doris/clone/TabletChecker.java b/fe/src/main/java/org/apache/doris/clone/TabletChecker.java index d2e9d643a99a2d..9dbedda6758b9f 100644 --- a/fe/src/main/java/org/apache/doris/clone/TabletChecker.java +++ b/fe/src/main/java/org/apache/doris/clone/TabletChecker.java @@ -22,6 +22,7 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Partition.PartitionState; @@ -207,7 +208,13 @@ private void checkTablets() { } boolean isInPrios = isInPrios(dbId, table.getId(), partition.getId()); boolean prioPartIsHealthy = true; - for (MaterializedIndex idx : partition.getMaterializedIndices()) { + /* + * Here we get all ALL indexes, including SHADOW indexes. + * SHADOW index should be treated as a special NORMAL index. + * It can be repaired, but CAN NOT be balanced, added or removed. + * The above restrictions will be checked in tablet scheduler. + */ + for (MaterializedIndex idx : partition.getMaterializedIndices(IndexExtState.ALL)) { for (Tablet tablet : idx.getTablets()) { totalTabletNum++; diff --git a/fe/src/main/java/org/apache/doris/clone/TabletSchedCtx.java b/fe/src/main/java/org/apache/doris/clone/TabletSchedCtx.java index 175cd04be899fe..0898febfd63544 100644 --- a/fe/src/main/java/org/apache/doris/clone/TabletSchedCtx.java +++ b/fe/src/main/java/org/apache/doris/clone/TabletSchedCtx.java @@ -478,7 +478,7 @@ public void chooseSrcReplica(Map backendsWorkingSlots) throws Sc continue; } - if (!replica.checkVersionCatchUp(visibleVersion, visibleVersionHash)) { + if (!replica.checkVersionCatchUp(visibleVersion, visibleVersionHash, false)) { continue; } diff --git a/fe/src/main/java/org/apache/doris/clone/TabletScheduler.java b/fe/src/main/java/org/apache/doris/clone/TabletScheduler.java index 06445cb612a89c..680a22e142f5d7 100644 --- a/fe/src/main/java/org/apache/doris/clone/TabletScheduler.java +++ b/fe/src/main/java/org/apache/doris/clone/TabletScheduler.java @@ -514,7 +514,8 @@ private void scheduleTablet(TabletSchedCtx tabletCtx, AgentBatchTask batchTask) throw new SchedException(Status.UNRECOVERABLE, "table's state is not NORMAL"); } - if (statusPair.first != TabletStatus.VERSION_INCOMPLETE && partition.getState() != PartitionState.NORMAL) { + if (statusPair.first != TabletStatus.VERSION_INCOMPLETE + && (partition.getState() != PartitionState.NORMAL || tableState != OlapTableState.NORMAL)) { // If table is under ALTER process(before FINISHING), do not allow to add or delete replica. // VERSION_INCOMPLETE will repair the replica in place, which is allowed. throw new SchedException(Status.UNRECOVERABLE, @@ -743,7 +744,7 @@ private boolean deleteReplicaWithFailedVersion(TabletSchedCtx tabletCtx, boolean private boolean deleteReplicaWithLowerVersion(TabletSchedCtx tabletCtx, boolean force) throws SchedException { for (Replica replica : tabletCtx.getReplicas()) { - if (!replica.checkVersionCatchUp(tabletCtx.getCommittedVersion(), tabletCtx.getCommittedVersionHash())) { + if (!replica.checkVersionCatchUp(tabletCtx.getCommittedVersion(), tabletCtx.getCommittedVersionHash(), false)) { deleteReplicaInternal(tabletCtx, replica, "lower version", force); return true; } @@ -874,7 +875,7 @@ private void deleteReplicaInternal(TabletSchedCtx tabletCtx, Replica replica, St * 2. Wait for any txns before the watermark txn id to be finished. If all are finished, which means this replica is * safe to be deleted. */ - if (!force && replica.getState().isLoadable() && replica.getWatermarkTxnId() == -1) { + if (!force && replica.getState().canLoad() && replica.getWatermarkTxnId() == -1) { long nextTxnId = Catalog.getCurrentGlobalTransactionMgr().getTransactionIDGenerator().getNextTransactionId(); replica.setWatermarkTxnId(nextTxnId); replica.setState(ReplicaState.DECOMMISSION); diff --git a/fe/src/main/java/org/apache/doris/common/CaseSensibility.java b/fe/src/main/java/org/apache/doris/common/CaseSensibility.java index 170f8b37ec2a76..fa11d1d5390ced 100644 --- a/fe/src/main/java/org/apache/doris/common/CaseSensibility.java +++ b/fe/src/main/java/org/apache/doris/common/CaseSensibility.java @@ -23,7 +23,7 @@ public enum CaseSensibility { TABLE(true), ROLUP(true), PARTITION(true), - COLUMN(true), + COLUMN(false), USER(true), ROLE(false), HOST(false), diff --git a/fe/src/main/java/org/apache/doris/common/Config.java b/fe/src/main/java/org/apache/doris/common/Config.java index ba3074e9063fa5..0bff828a87ba2c 100644 --- a/fe/src/main/java/org/apache/doris/common/Config.java +++ b/fe/src/main/java/org/apache/doris/common/Config.java @@ -263,6 +263,11 @@ public class Config extends ConfigBase { */ @ConfField(mutable = true, masterOnly = true) public static int tablet_create_timeout_second = 1; + /* + * In order not to wait too long for create table(index), set a max timeout. + */ + @ConfField(mutable = true, masterOnly = true) + public static int max_create_table_timeout_second = 60; /* * Maximal waiting time for all publish version tasks of one transaction to be finished diff --git a/fe/src/main/java/org/apache/doris/common/FeConstants.java b/fe/src/main/java/org/apache/doris/common/FeConstants.java index 95683e20ac3693..46bafa7598fcd8 100644 --- a/fe/src/main/java/org/apache/doris/common/FeConstants.java +++ b/fe/src/main/java/org/apache/doris/common/FeConstants.java @@ -28,12 +28,12 @@ public class FeConstants { public static int checkpoint_interval_second = 60; // 1 minutes // dpp version - public static String dpp_version = "3_1_0"; + public static String dpp_version = "3_2_0"; // bloom filter false positive probability public static double default_bloom_filter_fpp = 0.05; // general model // Current meta data version. Use this version to write journals and image - public static int meta_version = FeMetaVersion.VERSION_58; + public static int meta_version = FeMetaVersion.VERSION_59; } diff --git a/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java b/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java index b5bdb2a42d9edd..d45486aa15d858 100644 --- a/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java +++ b/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java @@ -126,4 +126,6 @@ public final class FeMetaVersion { public static final int VERSION_57 = 57; // broker load support function, persist origin stmt in broker load public static final int VERSION_58 = 58; + // for alter job v2 + public static final int VERSION_59 = 59; } diff --git a/fe/src/main/java/org/apache/doris/common/FeNameFormat.java b/fe/src/main/java/org/apache/doris/common/FeNameFormat.java index 722d825a6a230b..f050e1ad2c64fb 100644 --- a/fe/src/main/java/org/apache/doris/common/FeNameFormat.java +++ b/fe/src/main/java/org/apache/doris/common/FeNameFormat.java @@ -17,6 +17,7 @@ package org.apache.doris.common; +import org.apache.doris.alter.SchemaChangeHandler; import org.apache.doris.mysql.privilege.PaloRole; import org.apache.doris.system.SystemInfoService; @@ -63,6 +64,9 @@ public static void checkColumnName(String columnName) throws AnalysisException { if (Strings.isNullOrEmpty(columnName) || !columnName.matches(COMMON_NAME_REGEX)) { ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_COLUMN_NAME, columnName); } + if (columnName.startsWith(SchemaChangeHandler.SHADOW_NAME_PRFIX)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_COLUMN_NAME, columnName); + } } public static void checkLabel(String label) throws AnalysisException { diff --git a/fe/src/main/java/org/apache/doris/common/MarkedCountDownLatch.java b/fe/src/main/java/org/apache/doris/common/MarkedCountDownLatch.java index bc25b3877f2b8f..74a42ee6c76b8b 100644 --- a/fe/src/main/java/org/apache/doris/common/MarkedCountDownLatch.java +++ b/fe/src/main/java/org/apache/doris/common/MarkedCountDownLatch.java @@ -28,6 +28,7 @@ public class MarkedCountDownLatch extends CountDownLatch { private Multimap marks; + private Status st = Status.OK; public MarkedCountDownLatch(int count) { super(count); @@ -50,9 +51,16 @@ public synchronized List> getLeftMarks() { return Lists.newArrayList(marks.entries()); } - public synchronized void countDownToZero() { + public Status getStatus() { + return st; + } + + public synchronized void countDownToZero(Status status) { while(getCount() > 0) { super.countDown(); } + if (st.ok()) { + st = status; + } } } diff --git a/fe/src/main/java/org/apache/doris/common/Status.java b/fe/src/main/java/org/apache/doris/common/Status.java index 13107b3233c63c..01e17d5d46975c 100644 --- a/fe/src/main/java/org/apache/doris/common/Status.java +++ b/fe/src/main/java/org/apache/doris/common/Status.java @@ -23,10 +23,11 @@ public class Status { public static final Status OK = new Status(); - public static final Status CANCELLED = new Status(TStatusCode.CANCELLED, "Cancelled"); + public static final Status CANCELLED = new Status(TStatusCode.CANCELLED, "Cancelled"); public static final Status THRIFT_RPC_ERROR = new Status(TStatusCode.THRIFT_RPC_ERROR, "Thrift RPC failed"); + public TStatusCode getErrorCode() { return errorCode; } diff --git a/fe/src/main/java/org/apache/doris/common/proc/IndicesProcDir.java b/fe/src/main/java/org/apache/doris/common/proc/IndicesProcDir.java index be4986cfca2970..63bf96d2f0c7be 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/IndicesProcDir.java +++ b/fe/src/main/java/org/apache/doris/common/proc/IndicesProcDir.java @@ -19,6 +19,7 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.common.AnalysisException; @@ -63,7 +64,7 @@ public ProcResult fetchResult() throws AnalysisException { db.readLock(); try { result.setNames(TITLE_NAMES); - for (MaterializedIndex materializedIndex : partition.getMaterializedIndices()) { + for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { List indexInfo = new ArrayList(); indexInfo.add(materializedIndex.getId()); indexInfo.add(olapTable.getIndexNameById(materializedIndex.getId())); diff --git a/fe/src/main/java/org/apache/doris/common/proc/RollupJobProcDir.java b/fe/src/main/java/org/apache/doris/common/proc/RollupJobProcDir.java index 8e2096367d7bf1..741e0038a45d6f 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/RollupJobProcDir.java +++ b/fe/src/main/java/org/apache/doris/common/proc/RollupJobProcDir.java @@ -17,23 +17,23 @@ package org.apache.doris.common.proc; -import org.apache.doris.alter.RollupJob; +import org.apache.doris.alter.RollupJobV2; import org.apache.doris.common.AnalysisException; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import java.util.ArrayList; import java.util.List; -public class RollupJobProcDir implements ProcDirInterface { +// Show unfinished rollup tasks of rollup job v2 +public class RollupJobProcDir implements ProcNodeInterface { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() - .add("PartitionId").add("RollupIndexId").add("IndexState") + .add("BackendId").add("BaseTabletId").add("RollupTabletId") .build(); - private RollupJob rollupJob; + private RollupJobV2 rollupJob; - public RollupJobProcDir(RollupJob rollupJob) { + public RollupJobProcDir(RollupJobV2 rollupJob) { this.rollupJob = rollupJob; } @@ -44,32 +44,8 @@ public ProcResult fetchResult() throws AnalysisException { BaseProcResult result = new BaseProcResult(); result.setNames(TITLE_NAMES); - List> rollupJobInfos = rollupJob.getInfos(); - for (List infoStr : rollupJobInfos) { - List oneInfo = new ArrayList(TITLE_NAMES.size()); - for (Comparable element : infoStr) { - oneInfo.add(element.toString()); - } - result.addRow(oneInfo); - } + List> unfinishedRollupTasks = rollupJob.getUnfinishedTasks(2000); + result.setRows(unfinishedRollupTasks); return result; } - - @Override - public boolean register(String name, ProcNodeInterface node) { - return false; - } - - @Override - public ProcNodeInterface lookup(String partitionIdStr) throws AnalysisException { - long partitionId; - try { - partitionId = Long.valueOf(partitionIdStr); - } catch (NumberFormatException e) { - throw new AnalysisException("Invalid table id format: " + partitionIdStr); - } - - return new RollupTabletsProcNode(rollupJob, partitionId); - } - } diff --git a/fe/src/main/java/org/apache/doris/common/proc/RollupProcDir.java b/fe/src/main/java/org/apache/doris/common/proc/RollupProcDir.java index 42c31523c65819..c290f55f0010ca 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/RollupProcDir.java +++ b/fe/src/main/java/org/apache/doris/common/proc/RollupProcDir.java @@ -17,9 +17,9 @@ package org.apache.doris.common.proc; -import org.apache.doris.alter.AlterJob; +import org.apache.doris.alter.AlterJobV2; import org.apache.doris.alter.RollupHandler; -import org.apache.doris.alter.RollupJob; +import org.apache.doris.alter.RollupJobV2; import org.apache.doris.catalog.Database; import org.apache.doris.common.AnalysisException; @@ -34,7 +34,7 @@ public class RollupProcDir implements ProcDirInterface { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("JobId").add("TableName").add("CreateTime").add("FinishedTime") .add("BaseIndexName").add("RollupIndexName").add("RollupId").add("TransactionId") - .add("State").add("Msg") .add("Progress") + .add("State").add("Msg").add("Progress").add("Timeout") .build(); private RollupHandler rollupHandler; @@ -70,25 +70,25 @@ public boolean register(String name, ProcNodeInterface node) { } @Override - public ProcNodeInterface lookup(String tableIdStr) throws AnalysisException { - if (Strings.isNullOrEmpty(tableIdStr)) { + public ProcNodeInterface lookup(String jobIdStr) throws AnalysisException { + if (Strings.isNullOrEmpty(jobIdStr)) { throw new AnalysisException("Table id is null"); } - long tableId = -1L; + long jobId = -1L; try { - tableId = Long.valueOf(tableIdStr); + jobId = Long.valueOf(jobIdStr); } catch (Exception e) { throw new AnalysisException("Table id is invalid"); } - Preconditions.checkState(tableId != -1L); - AlterJob job = rollupHandler.getAlterJob(tableId); + Preconditions.checkState(jobId != -1L); + AlterJobV2 job = rollupHandler.getAlterJobV2(jobId); if (job == null) { return null; } - return new RollupJobProcDir((RollupJob) job); + return new RollupJobProcDir((RollupJobV2) job); } } diff --git a/fe/src/main/java/org/apache/doris/common/proc/RollupTabletsProcNode.java b/fe/src/main/java/org/apache/doris/common/proc/RollupTabletsProcNode.java deleted file mode 100644 index 08ad1ff195bd42..00000000000000 --- a/fe/src/main/java/org/apache/doris/common/proc/RollupTabletsProcNode.java +++ /dev/null @@ -1,62 +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.common.proc; - -import org.apache.doris.alter.RollupJob; -import org.apache.doris.common.AnalysisException; - -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; - -import java.util.ArrayList; -import java.util.List; - -public class RollupTabletsProcNode implements ProcNodeInterface { - public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() - .add("TabletId").add("ReplicaId").add("BackendId").add("Version") - .add("VersionHash").add("DataSize").add("RowCount").add("State") - .build(); - - private RollupJob rollupJob; - private long partitionId; - - public RollupTabletsProcNode(RollupJob rollupJob, long partitionId) { - this.rollupJob = rollupJob; - this.partitionId = partitionId; - } - - @Override - public ProcResult fetchResult() throws AnalysisException { - Preconditions.checkNotNull(rollupJob); - - BaseProcResult result = new BaseProcResult(); - result.setNames(TITLE_NAMES); - - List> tabletInfos = rollupJob.getRollupIndexInfo(partitionId); - for (int i = 0; i < tabletInfos.size(); i++) { - List info = tabletInfos.get(i); - List row = new ArrayList(info.size()); - for (int j = 0; j < info.size(); j++) { - row.add(info.get(j).toString()); - } - result.addRow(row); - } - return result; - } - -} diff --git a/fe/src/main/java/org/apache/doris/common/proc/SchemaChangeProcNode.java b/fe/src/main/java/org/apache/doris/common/proc/SchemaChangeProcNode.java index 3ec874aa8c9c01..a5806beed5c477 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/SchemaChangeProcNode.java +++ b/fe/src/main/java/org/apache/doris/common/proc/SchemaChangeProcNode.java @@ -30,8 +30,8 @@ public class SchemaChangeProcNode implements ProcNodeInterface { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("JobId").add("TableName").add("CreateTime").add("FinishTime") - .add("IndexName").add("IndexId").add("SchemaVersion").add("IndexState") - .add("TransactionId").add("State").add("Progress").add("Msg") + .add("IndexName").add("IndexId").add("OriginIndexId").add("SchemaVersion") + .add("TransactionId").add("State").add("Msg").add("Progress").add("Timeout") .build(); private SchemaChangeHandler schemaChangeHandler; diff --git a/fe/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java b/fe/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java index da46a36c0d36fa..3e320e17180d98 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java +++ b/fe/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java @@ -25,6 +25,7 @@ import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Table.TableType; import org.apache.doris.catalog.Tablet; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.Tablet.TabletStatus; import org.apache.doris.clone.TabletSchedCtx.Priority; import org.apache.doris.common.AnalysisException; @@ -117,7 +118,7 @@ public ProcResult fetchResult() throws AnalysisException { for (Partition partition : olapTable.getPartitions()) { short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId()); ++dbPartitionNum; - for (MaterializedIndex materializedIndex : partition.getMaterializedIndices()) { + for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { ++dbIndexNum; for (Tablet tablet : materializedIndex.getTablets()) { ++dbTabletNum; diff --git a/fe/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java index caa8e6d705bf2a..4dcb15798baa50 100644 --- a/fe/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java +++ b/fe/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java @@ -68,6 +68,8 @@ public class PropertyAnalyzer { public static final String PROPERTIES_COLOCATE_WITH = "colocate_with"; + public static final String PROPERTIES_TIMEOUT = "timeout"; + public static DataProperty analyzeDataProperty(Map properties, DataProperty oldDataProperty) throws AnalysisException { DataProperty dataProperty = oldDataProperty; @@ -353,4 +355,18 @@ public static String analyzeColocate(Map properties) throws Anal } return colocateGroup; } + + public static long analyzeTimeout(Map properties, long defaultTimeout) throws AnalysisException { + long timeout = defaultTimeout; + if (properties != null && properties.containsKey(PROPERTIES_TIMEOUT)) { + String timeoutStr = properties.get(PROPERTIES_TIMEOUT); + try { + timeout = Long.valueOf(timeoutStr); + } catch (NumberFormatException e) { + throw new AnalysisException("Invalid timeout format: " + timeoutStr); + } + properties.remove(PROPERTIES_TIMEOUT); + } + return timeout; + } } diff --git a/fe/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java b/fe/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java index 3e329a5cc531ff..fbd727b4d66815 100644 --- a/fe/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java +++ b/fe/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.MetaObject; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; @@ -304,7 +305,7 @@ private long chooseTablet() { // sort materializedIndices Queue indexQueue = new PriorityQueue(1, COMPARATOR); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { indexQueue.add(index); } diff --git a/fe/src/main/java/org/apache/doris/http/rest/RowCountAction.java b/fe/src/main/java/org/apache/doris/http/rest/RowCountAction.java index 54a420b442d3e3..0ed9bc7e8a0a87 100644 --- a/fe/src/main/java/org/apache/doris/http/rest/RowCountAction.java +++ b/fe/src/main/java/org/apache/doris/http/rest/RowCountAction.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Replica; @@ -92,12 +93,12 @@ public void execute(BaseRequest request, BaseResponse response) throws DdlExcept for (Partition partition : olapTable.getPartitions()) { long version = partition.getVisibleVersion(); long versionHash = partition.getVisibleVersionHash(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { long indexRowCount = 0L; for (Tablet tablet : index.getTablets()) { long tabletRowCount = 0L; for (Replica replica : tablet.getReplicas()) { - if (replica.checkVersionCatchUp(version, versionHash) + if (replica.checkVersionCatchUp(version, versionHash, false) && replica.getRowCount() > tabletRowCount) { tabletRowCount = replica.getRowCount(); } diff --git a/fe/src/main/java/org/apache/doris/http/rest/ShowMetaInfoAction.java b/fe/src/main/java/org/apache/doris/http/rest/ShowMetaInfoAction.java index 9ff93a253a8418..28adf4ef65369a 100644 --- a/fe/src/main/java/org/apache/doris/http/rest/ShowMetaInfoAction.java +++ b/fe/src/main/java/org/apache/doris/http/rest/ShowMetaInfoAction.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Replica; @@ -34,9 +35,8 @@ import org.apache.doris.http.BaseResponse; import org.apache.doris.http.IllegalArgException; import org.apache.doris.persist.Storage; -import com.google.gson.Gson; -import io.netty.handler.codec.http.HttpMethod; +import com.google.gson.Gson; import org.apache.commons.lang.StringUtils; import org.apache.logging.log4j.LogManager; @@ -49,6 +49,8 @@ import java.util.List; import java.util.Map; +import io.netty.handler.codec.http.HttpMethod; + public class ShowMetaInfoAction extends RestBaseAction { private enum Action { SHOW_DB_SIZE, @@ -164,7 +166,7 @@ public Map getDataSize() { long tableSize = 0; for (Partition partition : olapTable.getPartitions()) { long partitionSize = 0; - for (MaterializedIndex mIndex : partition.getMaterializedIndices()) { + for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { long indexSize = 0; for (Tablet tablet : mIndex.getTablets()) { long maxReplicaSize = 0; diff --git a/fe/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/src/main/java/org/apache/doris/journal/JournalEntity.java index a28ebfb7b38ab6..06a8515b4f5861 100644 --- a/fe/src/main/java/org/apache/doris/journal/JournalEntity.java +++ b/fe/src/main/java/org/apache/doris/journal/JournalEntity.java @@ -18,6 +18,7 @@ package org.apache.doris.journal; import org.apache.doris.alter.AlterJob; +import org.apache.doris.alter.AlterJobV2; import org.apache.doris.analysis.UserIdentity; import org.apache.doris.backup.BackupJob; import org.apache.doris.backup.Repository; @@ -411,6 +412,11 @@ public void readFields(DataInput in) throws IOException { needRead = false; break; } + case OperationType.OP_ALTER_JOB_V2: { + data = AlterJobV2.read(in); + needRead = false; + break; + } default: { IOException e = new IOException(); LOG.error("UNKNOWN Operation Type {}", opCode, e); diff --git a/fe/src/main/java/org/apache/doris/load/BrokerFileGroup.java b/fe/src/main/java/org/apache/doris/load/BrokerFileGroup.java index baf25b0cbe1d17..82328db1313b96 100644 --- a/fe/src/main/java/org/apache/doris/load/BrokerFileGroup.java +++ b/fe/src/main/java/org/apache/doris/load/BrokerFileGroup.java @@ -63,11 +63,11 @@ public class BrokerFileGroup implements Writable { private String fileFormat; private boolean isNegative; private List partitionIds; - // this is a compatible param which only happens before the function of broker has been supported. private List fileFieldNames; private List filePaths; // this is a compatible param which only happens before the function of broker has been supported. + @Deprecated private Map exprColumnMap; // this param will be recreated by data desc when the log replay private List columnExprList; @@ -89,6 +89,7 @@ public BrokerFileGroup(BrokerTable table) throws AnalysisException { public BrokerFileGroup(DataDescription dataDescription) { this.dataDescription = dataDescription; this.exprColumnMap = null; + this.fileFieldNames = dataDescription.getColumnNames(); this.columnExprList = dataDescription.getParsedColumnExprList(); } @@ -173,6 +174,10 @@ public List getFilePaths() { return filePaths; } + public List getFileFieldNames() { + return fileFieldNames; + } + public List getColumnExprList() { return columnExprList; } diff --git a/fe/src/main/java/org/apache/doris/load/Load.java b/fe/src/main/java/org/apache/doris/load/Load.java index 5aa11dd32a6f71..37f4ad2c20490d 100644 --- a/fe/src/main/java/org/apache/doris/load/Load.java +++ b/fe/src/main/java/org/apache/doris/load/Load.java @@ -17,11 +17,14 @@ package org.apache.doris.load; +import org.apache.doris.alter.SchemaChangeHandler; import org.apache.doris.analysis.BinaryPredicate; import org.apache.doris.analysis.CancelLoadStmt; import org.apache.doris.analysis.ColumnSeparator; import org.apache.doris.analysis.DataDescription; import org.apache.doris.analysis.DeleteStmt; +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.ImportColumnDesc; import org.apache.doris.analysis.IsNullPredicate; import org.apache.doris.analysis.LabelName; import org.apache.doris.analysis.LiteralExpr; @@ -36,6 +39,7 @@ 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.OlapTable; import org.apache.doris.catalog.OlapTable.OlapTableState; import org.apache.doris.catalog.Partition; @@ -59,12 +63,10 @@ import org.apache.doris.common.FeNameFormat; import org.apache.doris.common.LabelAlreadyUsedException; import org.apache.doris.common.LoadException; -import org.apache.doris.common.MarkedCountDownLatch; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.Pair; import org.apache.doris.common.util.ListComparator; import org.apache.doris.common.util.TimeUtils; -import org.apache.doris.common.util.Util; import org.apache.doris.load.AsyncDeleteJob.DeleteState; import org.apache.doris.load.FailMsg.CancelType; import org.apache.doris.load.LoadJob.JobState; @@ -74,17 +76,13 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.service.FrontendOptions; import org.apache.doris.system.Backend; -import org.apache.doris.task.AgentBatchTask; import org.apache.doris.task.AgentClient; -import org.apache.doris.task.AgentTask; -import org.apache.doris.task.AgentTaskExecutor; import org.apache.doris.task.AgentTaskQueue; import org.apache.doris.task.PushTask; import org.apache.doris.thrift.TEtlState; import org.apache.doris.thrift.TMiniLoadRequest; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TPriority; -import org.apache.doris.thrift.TPushType; import org.apache.doris.transaction.PartitionCommitInfo; import org.apache.doris.transaction.TableCommitInfo; import org.apache.doris.transaction.TransactionState; @@ -115,7 +113,6 @@ import java.util.Map.Entry; import java.util.Set; import java.util.UUID; -import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantReadWriteLock; public class Load { @@ -237,7 +234,8 @@ private void writeUnlock() { // return true if we truly add the load job // return false otherwise (eg: a retry request) - public boolean addLoadJob(TMiniLoadRequest request) throws DdlException { + @Deprecated + public boolean addMiniLoadJob(TMiniLoadRequest request) throws DdlException { // get params String fullDbName = request.getDb(); String tableName = request.getTbl(); @@ -300,8 +298,7 @@ public boolean addLoadJob(TMiniLoadRequest request) throws DdlException { } DataDescription dataDescription = new DataDescription(tableName, partitionNames, filePaths, - columnNames, - columnSeparator, formatType, false, null); + columnNames, columnSeparator, formatType, false, null); dataDescription.setLineDelimiter(lineDelimiter); dataDescription.setBeAddr(beAddr); // parse hll param pair @@ -449,12 +446,7 @@ private LoadJob createLoadJob(LoadStmt stmt, EtlJobType etlJobType, } if (properties.containsKey(LoadStmt.LOAD_DELETE_FLAG_PROPERTY)) { - String flag = properties.get(LoadStmt.LOAD_DELETE_FLAG_PROPERTY); - if (flag.equalsIgnoreCase("true") || flag.equalsIgnoreCase("false")) { - job.setDeleteFlag(Boolean.parseBoolean(flag)); - } else { - throw new DdlException("Value of delete flag is invalid"); - } + throw new DdlException("Do not support load_delete_flag"); } if (properties.containsKey(LoadStmt.EXEC_MEM_LIMIT)) { @@ -472,7 +464,7 @@ private LoadJob createLoadJob(LoadStmt stmt, EtlJobType etlJobType, Map>> tableToPartitionSources = Maps.newHashMap(); for (DataDescription dataDescription : dataDescriptions) { // create source - checkAndCreateSource(db, dataDescription, tableToPartitionSources, job.getDeleteFlag(), etlJobType); + checkAndCreateSource(db, dataDescription, tableToPartitionSources, etlJobType); job.addTableName(dataDescription.getTableName()); } for (Entry>> tableEntry : tableToPartitionSources.entrySet()) { @@ -599,10 +591,11 @@ private LoadJob createLoadJob(LoadStmt stmt, EtlJobType etlJobType, return job; } + /* + * This is used for both hadoop load and broker load v2 + */ public static void checkAndCreateSource(Database db, DataDescription dataDescription, - Map>> tableToPartitionSources, - boolean deleteFlag, EtlJobType jobType) - throws DdlException { + Map>> tableToPartitionSources, EtlJobType jobType) throws DdlException { Source source = new Source(dataDescription.getFilePaths()); long tableId = -1; Set sourcePartitionIds = Sets.newHashSet(); @@ -641,30 +634,24 @@ public static void checkAndCreateSource(Database db, DataDescription dataDescrip throw new DdlException("Load for AGG_KEYS table should not specify NEGATIVE"); } - if (((OlapTable) table).getKeysType() != KeysType.UNIQUE_KEYS && deleteFlag) { - throw new DdlException("Delete flag can only be used for UNIQUE_KEYS table"); - } - // get table schema - List tableSchema = table.getBaseSchema(); - Map nameToTableColumn = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); - for (Column column : tableSchema) { - nameToTableColumn.put(column.getName(), column); - } + List baseSchema = table.getBaseSchema(); + // fill the column info if user does not specify them + dataDescription.fillColumnInfoIfNotSpecified(baseSchema); // source columns List columnNames = Lists.newArrayList(); List assignColumnNames = dataDescription.getColumnNames(); if (assignColumnNames == null) { // use table columns - for (Column column : tableSchema) { + for (Column column : baseSchema) { columnNames.add(column.getName()); } } else { // convert column to schema format for (String assignCol : assignColumnNames) { - if (nameToTableColumn.containsKey(assignCol)) { - columnNames.add(nameToTableColumn.get(assignCol).getName()); + if (table.getColumn(assignCol) != null) { + columnNames.add(table.getColumn(assignCol).getName()); } else { columnNames.add(assignCol); } @@ -672,15 +659,21 @@ public static void checkAndCreateSource(Database db, DataDescription dataDescrip } source.setColumnNames(columnNames); + Map>> columnToHadoopFunction = dataDescription.getColumnToHadoopFunction(); + List parsedColumnExprList = dataDescription.getParsedColumnExprList(); + Map parsedColumnExprMap = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); + for (ImportColumnDesc importColumnDesc : parsedColumnExprList) { + parsedColumnExprMap.put(importColumnDesc.getColumnName(), importColumnDesc.getExpr()); + } + // check default value - Map>> assignColumnToFunction = dataDescription.getColumnToHadoopFunction(); - for (Column column : tableSchema) { + for (Column column : baseSchema) { String columnName = column.getName(); if (columnNames.contains(columnName)) { continue; } - if (assignColumnToFunction != null && assignColumnToFunction.containsKey(columnName)) { + if (parsedColumnExprMap.containsKey(columnName)) { continue; } @@ -688,20 +681,12 @@ public static void checkAndCreateSource(Database db, DataDescription dataDescrip continue; } - if (deleteFlag && !column.isKey()) { - List args = Lists.newArrayList(); - args.add("0"); - Pair> functionPair = new Pair>("default_value", args); - assignColumnToFunction.put(columnName, functionPair); - continue; - } - throw new DdlException("Column has no default value. column: " + columnName); } - // check negative for sum aggreate type + // check negative for sum aggregate type if (dataDescription.isNegative()) { - for (Column column : tableSchema) { + for (Column column : baseSchema) { if (!column.isKey() && column.getAggregationType() != AggregateType.SUM) { throw new DdlException("Column is not SUM AggreateType. column:" + column.getName()); } @@ -709,29 +694,79 @@ public static void checkAndCreateSource(Database db, DataDescription dataDescrip } // check hll - for (Column column : tableSchema) { + for (Column column : baseSchema) { if (column.getDataType() == PrimitiveType.HLL) { - if (assignColumnToFunction != null && !assignColumnToFunction.containsKey(column.getName())) { + if (columnToHadoopFunction != null && !columnToHadoopFunction.containsKey(column.getName())) { throw new DdlException("Hll column is not assigned. column:" + column.getName()); } } } + // check mapping column exist in table // check function // convert mapping column and func arg columns to schema format + + // When doing schema change, there may have some 'shadow' columns, with prefix '__doris_shadow_' in + // their names. These columns are invisible to user, but we need to generate data for these columns. + // So we add column mappings for these column. + // eg1: + // base schema is (A, B, C), and B is under schema change, so there will be a shadow column: '__doris_shadow_B' + // So the final column mapping should looks like: (A, B, C, __doris_shadow_B = substitute(B)); + for (Column column : table.getFullSchema()) { + if (column.isNameWithPrefix(SchemaChangeHandler.SHADOW_NAME_PRFIX)) { + /* + * There is a case that if user does not specify the related origin column, eg: + * COLUMNS (A, C), and B is not specified, but B is being modified so there is a shadow column '__doris_shadow_B'. + * We can not just add a mapping function "__doris_shadow_B = substitute(B)", because Doris can not find column B. + * In this case, __doris_shadow_B can use its default value, so no need to add it to column mapping + */ + String originCol = column.getNameWithoutPrefix(SchemaChangeHandler.SHADOW_NAME_PRFIX); + Expr mappingExpr = parsedColumnExprMap.get(originCol); + if (mappingExpr != null) { + /* + * eg: + * (A, C) SET (B = func(xx)) + * -> + * (A, C) SET (B = func(xx), __doris_shadow_B = func(xxx)) + */ + if (columnToHadoopFunction.containsKey(originCol)) { + columnToHadoopFunction.put(column.getName(), columnToHadoopFunction.get(originCol)); + } + ImportColumnDesc importColumnDesc = new ImportColumnDesc(column.getName(), mappingExpr); + parsedColumnExprList.add(importColumnDesc); + } else { + /* + * eg: + * (A, B, C) + * -> + * (A, B, C) SET (__doris_shadow_B = substitute(B)) + */ + columnToHadoopFunction.put(column.getName(), Pair.create("substitute", Lists.newArrayList(originCol))); + ImportColumnDesc importColumnDesc = new ImportColumnDesc(column.getName(), new SlotRef(null, originCol)); + parsedColumnExprList.add(importColumnDesc); + } + + } + } + + LOG.debug("after add shadow column. parsedColumnExprList: {}, columnToHadoopFunction: {}", + parsedColumnExprList, columnToHadoopFunction); + Map columnNameMap = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); for (String columnName : columnNames) { columnNameMap.put(columnName, columnName); } - if (assignColumnToFunction != null) { + + // validate hadoop functions + if (columnToHadoopFunction != null) { columnToFunction = Maps.newHashMap(); - for (Entry>> entry : assignColumnToFunction.entrySet()) { + for (Entry>> entry : columnToHadoopFunction.entrySet()) { String mappingColumnName = entry.getKey(); - if (!nameToTableColumn.containsKey(mappingColumnName)) { + Column mappingColumn = table.getColumn(mappingColumnName); + if (mappingColumn == null) { throw new DdlException("Mapping column is not in table. column: " + mappingColumnName); } - Column mappingColumn = nameToTableColumn.get(mappingColumnName); Pair> function = entry.getValue(); try { DataDescription.validateMappingFunction(function.first, function.second, columnNameMap, @@ -1637,7 +1672,7 @@ public List> getLoadJobUnfinishedInfo(long jobId) { long versionHash = partitionLoadInfo.getVersionHash(); for (Replica replica : tablet.getReplicas()) { - if (replica.checkVersionCatchUp(version, versionHash)) { + if (replica.checkVersionCatchUp(version, versionHash, false)) { continue; } @@ -1859,8 +1894,8 @@ public void unprotectQuorumLoadJob(LoadJob job, Database db) { partitionLoadInfo.getVersionHash(), jobId); // update table row count - for (MaterializedIndex materializedIndex : partition.getMaterializedIndices()) { - long tableRowCount = 0L; + for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { + long indexRowCount = 0L; for (Tablet tablet : materializedIndex.getTablets()) { long tabletRowCount = 0L; for (Replica replica : tablet.getReplicas()) { @@ -1869,9 +1904,9 @@ public void unprotectQuorumLoadJob(LoadJob job, Database db) { tabletRowCount = replicaRowCount; } } - tableRowCount += tabletRowCount; + indexRowCount += tabletRowCount; } - materializedIndex.setRowCount(tableRowCount); + materializedIndex.setRowCount(indexRowCount); } // end for indices } // end for partitions } // end for tables @@ -2389,7 +2424,7 @@ private boolean processQuorumFinished(LoadJob job, Database db) { updatePartitionVersion(partition, partitionLoadInfo.getVersion(), partitionLoadInfo.getVersionHash(), jobId); - for (MaterializedIndex materializedIndex : partition.getMaterializedIndices()) { + for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { long tableRowCount = 0L; for (Tablet tablet : materializedIndex.getTablets()) { long tabletRowCount = 0L; @@ -2748,11 +2783,11 @@ private void checkDeleteV2(OlapTable table, Partition partition, List slotRef.setCol(column.getName()); } Map> indexIdToSchema = table.getIndexIdToSchema(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { // check table has condition column - Map indexNameToColumn = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); + Map indexColNameToColumn = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); for (Column column : indexIdToSchema.get(index.getId())) { - indexNameToColumn.put(column.getName(), column); + indexColNameToColumn.put(column.getName(), column); } String indexName = table.getIndexNameById(index.getId()); for (Predicate condition : conditions) { @@ -2764,7 +2799,7 @@ private void checkDeleteV2(OlapTable table, Partition partition, List IsNullPredicate isNullPredicate = (IsNullPredicate) condition; columnName = ((SlotRef) isNullPredicate.getChild(0)).getColumnName(); } - Column column = indexNameToColumn.get(columnName); + Column column = indexColNameToColumn.get(columnName); if (column == null) { ErrorReport.reportDdlException(ErrorCode.ERR_BAD_FIELD_ERROR, columnName, indexName); } @@ -2808,221 +2843,6 @@ private void checkDeleteV2(OlapTable table, Partition partition, List } } - private void checkDelete(OlapTable table, Partition partition, List conditions, - long checkVersion, long checkVersionHash, List deleteConditions, - Map> asyncTabletIdToBackends, boolean preCheck) - throws DdlException { - // check partition state - PartitionState state = partition.getState(); - if (state != PartitionState.NORMAL) { - // ErrorReport.reportDdlException(ErrorCode.ERR_BAD_PARTITION_STATE, partition.getName(), state.name()); - throw new DdlException("Partition[" + partition.getName() + "]' state is not NORNAL: " + state.name()); - } - - // check running load job - List quorumFinishedLoadJobs = Lists.newArrayList(); - if (!checkPartitionLoadFinished(partition.getId(), quorumFinishedLoadJobs)) { - // ErrorReport.reportDdlException(ErrorCode.ERR_PARTITION_HAS_LOADING_JOBS, partition.getName()); - throw new DdlException("Partition[" + partition.getName() + "] has unfinished load jobs"); - } - - // get running async delete job - List asyncDeleteJobs = getCopiedAsyncDeleteJobs(); - - // check condition column is key column and condition value - Map nameToColumn = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); - for (Column column : table.getBaseSchema()) { - nameToColumn.put(column.getName(), column); - } - for (Predicate condition : conditions) { - SlotRef slotRef = null; - if (condition instanceof BinaryPredicate) { - BinaryPredicate binaryPredicate = (BinaryPredicate) condition; - slotRef = (SlotRef) binaryPredicate.getChild(0); - } else if (condition instanceof IsNullPredicate) { - IsNullPredicate isNullPredicate = (IsNullPredicate) condition; - slotRef = (SlotRef) isNullPredicate.getChild(0); - } - String columnName = slotRef.getColumnName(); - if (!nameToColumn.containsKey(columnName)) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_FIELD_ERROR, columnName, table.getName()); - } - - Column column = nameToColumn.get(columnName); - if (!column.isKey()) { - // ErrorReport.reportDdlException(ErrorCode.ERR_NOT_KEY_COLUMN, columnName); - throw new DdlException("Column[" + columnName + "] is not key column"); - } - - if (condition instanceof BinaryPredicate) { - String value = null; - try { - BinaryPredicate binaryPredicate = (BinaryPredicate) condition; - value = ((LiteralExpr) binaryPredicate.getChild(1)).getStringValue(); - LiteralExpr.create(value, Type.fromPrimitiveType(column.getDataType())); - } catch (AnalysisException e) { - // ErrorReport.reportDdlException(ErrorCode.ERR_INVALID_VALUE, value); - throw new DdlException("Invalid column value[" + value + "]"); - } - } - - // set schema column name - slotRef.setCol(column.getName()); - } - - long tableId = table.getId(); - long partitionId = partition.getId(); - Map> indexIdToSchema = table.getIndexIdToSchema(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { - // check table has condition column - Map indexNameToColumn = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); - for (Column column : indexIdToSchema.get(index.getId())) { - indexNameToColumn.put(column.getName(), column); - } - String indexName = table.getIndexNameById(index.getId()); - for (Predicate condition : conditions) { - String columnName = null; - if (condition instanceof BinaryPredicate) { - BinaryPredicate binaryPredicate = (BinaryPredicate) condition; - columnName = ((SlotRef) binaryPredicate.getChild(0)).getColumnName(); - } else if (condition instanceof IsNullPredicate) { - IsNullPredicate isNullPredicate = (IsNullPredicate) condition; - columnName = ((SlotRef) isNullPredicate.getChild(0)).getColumnName(); - } - Column column = indexNameToColumn.get(columnName); - if (column == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_FIELD_ERROR, columnName, indexName); - } - - if (table.getKeysType() == KeysType.DUP_KEYS && !column.isKey()) { - throw new DdlException("Column[" + columnName + "] is not key column in index[" + indexName + "]"); - } - } - - // check replica version and backend alive - short replicationNum = table.getPartitionInfo().getReplicationNum(partition.getId()); - for (Tablet tablet : index.getTablets()) { - Set needAsyncBackendIds = Sets.newHashSet(); - for (Replica replica : tablet.getReplicas()) { - if (!Catalog.getCurrentSystemInfo().checkBackendAvailable(replica.getBackendId())) { - LOG.warn("backend[{}] is not alive when delete check. pre: {}", - replica.getBackendId(), preCheck); - needAsyncBackendIds.add(replica.getBackendId()); - continue; - } - - // check replica version. - // here is a little bit confused. the main idea is - // 1. check if replica catch up the version - // 2. if not catch up and this is pre check, make sure there will be right quorum finished load jobs - // to fill the version gap between 'replica committed version' and 'partition committed version'. - // 3. if not catch up and this is after check - // 1) if diff version == 1, some sync delete task may failed. add async delete task. - // 2) if diff version > 1, make sure there will be right quorum finished load jobs - // to fill the version gap between 'replica committed version' and 'delete version - 1'. - // if ok, add async delete task. - if (!replica.checkVersionCatchUp(checkVersion, checkVersionHash)) { - long replicaVersion = replica.getVersion(); - if (replicaVersion == checkVersion) { - // in this case, version is same but version hash is not. - // which mean the current replica version is a non-committed version. - // so the replica's committed version should be the previous one. - --replicaVersion; - } - - // the *diffVersion* is number of versions need to be check - // for now: - // *replicaVersion* : the 'committed version' of the replica - // *checkVersion* : - // 1) if preCheck, this is partition committed version - // 2) if not preCheck, this is delete version - long diffVersion = checkVersion - replicaVersion; - Preconditions.checkState(diffVersion > 0); - for (int i = 1; i <= diffVersion; i++) { - boolean find = false; - long theVersion = replicaVersion + i; - for (LoadJob loadJob : quorumFinishedLoadJobs) { - if (theVersion == loadJob.getPartitionLoadInfo(tableId, partitionId).getVersion()) { - find = true; - break; - } - } - - for (AsyncDeleteJob deleteJob : asyncDeleteJobs) { - if (tableId == deleteJob.getTableId() && partitionId == deleteJob.getPartitionId() - && theVersion == deleteJob.getPartitionVersion()) { - find = true; - break; - } - } - - if (!find) { - if (theVersion == checkVersion && !preCheck) { - // the sync delete task of this replica may failed. - // add async delete task after. - continue; - } else { - // this should not happend. add log to observe. - LOG.error("replica version does not catch up with version: {}-{}. " - + "replica: {}-{}-{}-{}", - checkVersion, checkVersionHash, replica.getId(), tablet.getId(), - replica.getBackendId(), replica.getState()); - throw new DdlException("Replica[" + tablet.getId() + "-" + replica.getId() - + "] is not catch up with version: " + checkVersion + "-" - + replica.getVersion()); - } - } - } - - needAsyncBackendIds.add(replica.getBackendId()); - } // end check replica version - } // end for replicas - - if (replicationNum - needAsyncBackendIds.size() < replicationNum / 2 + 1) { - String backendsStr = Joiner.on(", ").join(needAsyncBackendIds); - LOG.warn("too many unavailable replica in tablet[{}], backends:[{}]", tablet.getId(), backendsStr); - throw new DdlException("Too many replicas are not available. Wait 10 mins and try again." - + " if still not work, contact Palo RD"); - } - - if (!needAsyncBackendIds.isEmpty()) { - LOG.info("add tablet[{}] to async delete. backends: {}", - tablet.getId(), needAsyncBackendIds); - asyncTabletIdToBackends.put(tablet.getId(), needAsyncBackendIds); - } - } // end for tablets - } // end for indices - - if (deleteConditions == null) { - return; - } - - // save delete conditions - for (Predicate condition : conditions) { - if (condition instanceof BinaryPredicate) { - BinaryPredicate binaryPredicate = (BinaryPredicate) condition; - SlotRef slotRef = (SlotRef) binaryPredicate.getChild(0); - String columnName = slotRef.getColumnName(); - StringBuilder sb = new StringBuilder(); - sb.append(columnName).append(" ").append(binaryPredicate.getOp().name()).append(" \"") - .append(((LiteralExpr) binaryPredicate.getChild(1)).getStringValue()).append("\""); - deleteConditions.add(sb.toString()); - } else if (condition instanceof IsNullPredicate) { - IsNullPredicate isNullPredicate = (IsNullPredicate) condition; - SlotRef slotRef = (SlotRef) isNullPredicate.getChild(0); - String columnName = slotRef.getColumnName(); - StringBuilder sb = new StringBuilder(); - sb.append(columnName); - if (isNullPredicate.isNotNull()) { - sb.append(" IS NOT NULL"); - } else { - sb.append(" IS NULL"); - } - deleteConditions.add(sb.toString()); - } - } - } - private boolean checkAndAddRunningSyncDeleteJob(long partitionId, String partitionName) throws DdlException { // check if there are synchronized delete job under going writeLock(); @@ -3137,7 +2957,7 @@ public void delete(DeleteStmt stmt) throws DdlException { loadDeleteJob = new LoadJob(jobId, db.getId(), tableId, partitionId, jobLabel, olapTable.getIndexIdToSchemaHash(), conditions, deleteInfo); Map idToTabletLoadInfo = Maps.newHashMap(); - for (MaterializedIndex materializedIndex : partition.getMaterializedIndices()) { + for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { for (Tablet tablet : materializedIndex.getTablets()) { long tabletId = tablet.getId(); // tabletLoadInfo is empty, because delete load does not need filepath filesize info @@ -3207,227 +3027,6 @@ public void delete(DeleteStmt stmt) throws DdlException { } } - @Deprecated - public void deleteOld(DeleteStmt stmt) throws DdlException { - String dbName = stmt.getDbName(); - String tableName = stmt.getTableName(); - String partitionName = stmt.getPartitionName(); - List conditions = stmt.getDeleteConditions(); - Database db = Catalog.getInstance().getDb(dbName); - if (db == null) { - throw new DdlException("Db does not exist. name: " + dbName); - } - - DeleteInfo deleteInfo = null; - - long tableId = -1; - long partitionId = -1; - long visibleVersion = -1; - long visibleVersionHash = -1; - long newVersion = -1; - long newVersionHash = -1; - AgentBatchTask deleteBatchTask = null; - int totalReplicaNum = 0; - Map> asyncTabletIdToBackends = Maps.newHashMap(); - db.readLock(); - try { - Table table = db.getTable(tableName); - if (table == null) { - throw new DdlException("Table does not exist. name: " + tableName); - } - - if (table.getType() != TableType.OLAP) { - throw new DdlException("Not olap type table. type: " + table.getType().name()); - } - OlapTable olapTable = (OlapTable) table; - - if (olapTable.getState() != OlapTableState.NORMAL) { - throw new DdlException("Table's state is not normal: " + tableName); - } - - tableId = olapTable.getId(); - Partition partition = olapTable.getPartition(partitionName); - if (partition == null) { - throw new DdlException("Partition does not exist. name: " + partitionName); - } - partitionId = partition.getId(); - - // pre check - visibleVersion = partition.getVisibleVersion(); - visibleVersionHash = partition.getVisibleVersionHash(); - checkDelete(olapTable, partition, conditions, visibleVersion, visibleVersionHash, - null, asyncTabletIdToBackends, true); - - newVersion = visibleVersion + 1; - newVersionHash = Util.generateVersionHash(); - deleteInfo = new DeleteInfo(db.getId(), tableId, tableName, - partition.getId(), partitionName, - newVersion, newVersionHash, null); - - checkAndAddRunningSyncDeleteJob(deleteInfo.getPartitionId(), partitionName); - - // create sync delete tasks - deleteBatchTask = new AgentBatchTask(); - for (MaterializedIndex materializedIndex : partition.getMaterializedIndices()) { - int schemaHash = olapTable.getSchemaHashByIndexId(materializedIndex.getId()); - for (Tablet tablet : materializedIndex.getTablets()) { - long tabletId = tablet.getId(); - for (Replica replica : tablet.getReplicas()) { - - if (asyncTabletIdToBackends.containsKey(tabletId) - && asyncTabletIdToBackends.get(tabletId).contains(replica.getBackendId())) { - continue; - } - - AgentTask pushTask = new PushTask(null, replica.getBackendId(), db.getId(), - tableId, partition.getId(), - materializedIndex.getId(), tabletId, replica.getId(), - schemaHash, newVersion, - newVersionHash, null, -1L, 0, -1L, TPushType.DELETE, - conditions, false, TPriority.HIGH); - if (AgentTaskQueue.addTask(pushTask)) { - deleteBatchTask.addTask(pushTask); - ++totalReplicaNum; - } - } - } - } - } finally { - db.readUnlock(); - } - - // send tasks to backends - MarkedCountDownLatch countDownLatch = new MarkedCountDownLatch(totalReplicaNum); - for (AgentTask task : deleteBatchTask.getAllTasks()) { - countDownLatch.addMark(task.getBackendId(), task.getSignature()); - ((PushTask) task).setCountDownLatch(countDownLatch); - } - AgentTaskExecutor.submit(deleteBatchTask); - long timeout = Config.tablet_delete_timeout_second * 1000L * totalReplicaNum; - boolean ok = false; - try { - ok = countDownLatch.await(timeout, TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - LOG.warn("InterruptedException: ", e); - ok = false; - } - - if (!ok) { - // sync delete failed for unknown reason. - // use async delete to try to make up after. - LOG.warn("sync delete failed. try async delete. table: {}, partition: {}", tableName, partitionName); - } - - Partition partition = null; - try { - // after check - db.writeLock(); - try { - OlapTable table = (OlapTable) db.getTable(tableName); - if (table == null) { - throw new DdlException("Table does not exist. name: " + tableName); - } - - partition = table.getPartition(partitionName); - if (partition == null) { - throw new DdlException("Partition does not exist. name: " + partitionName); - } - - // after check - // 1. check partition committed version first - if (partition.getVisibleVersion() > visibleVersion - || (visibleVersion == partition.getVisibleVersion() - && visibleVersionHash != partition.getVisibleVersionHash())) { - LOG.warn("before delete version: {}-{}. after delete version: {}-{}", - visibleVersion, visibleVersionHash, - partition.getVisibleVersion(), partition.getVisibleVersionHash()); - throw new DdlException("There may have some load job done during delete job. Try again"); - } - - // 2. after check - List deleteConditions = Lists.newArrayList(); - checkDelete(table, partition, conditions, newVersion, newVersionHash, deleteConditions, - asyncTabletIdToBackends, false); - deleteInfo.setDeleteConditions(deleteConditions); - - // update partition's version - updatePartitionVersion(partition, newVersion, newVersionHash, -1); - - for (MaterializedIndex materializedIndex : partition.getMaterializedIndices()) { - long indexId = materializedIndex.getId(); - for (Tablet tablet : materializedIndex.getTablets()) { - long tabletId = tablet.getId(); - for (Replica replica : tablet.getReplicas()) { - ReplicaPersistInfo info = - ReplicaPersistInfo.createForCondDelete(indexId, - tabletId, - replica.getId(), - replica.getVersion(), - replica.getVersionHash(), - table.getSchemaHashByIndexId(indexId), - replica.getDataSize(), - replica.getRowCount(), - replica.getLastFailedVersion(), - replica.getLastFailedVersionHash(), - replica.getLastSuccessVersion(), - replica.getLastSuccessVersionHash()); - deleteInfo.addReplicaPersistInfo(info); - } - } - } - - writeLock(); - try { - // handle async delete jobs - if (!asyncTabletIdToBackends.isEmpty()) { - AsyncDeleteJob asyncDeleteJob = new AsyncDeleteJob(db.getId(), tableId, partition.getId(), - newVersion, newVersionHash, - conditions); - for (Long tabletId : asyncTabletIdToBackends.keySet()) { - asyncDeleteJob.addTabletId(tabletId); - } - deleteInfo.setAsyncDeleteJob(asyncDeleteJob); - idToQuorumFinishedDeleteJob.put(asyncDeleteJob.getJobId(), asyncDeleteJob); - LOG.info("finished create async delete job: {}", asyncDeleteJob.getJobId()); - } - - // save delete info - List deleteInfos = dbToDeleteInfos.get(db.getId()); - if (deleteInfos == null) { - deleteInfos = Lists.newArrayList(); - dbToDeleteInfos.put(db.getId(), deleteInfos); - } - deleteInfos.add(deleteInfo); - } finally { - writeUnlock(); - } - - // Write edit log - Catalog.getInstance().getEditLog().logFinishSyncDelete(deleteInfo); - LOG.info("delete job finished at: {}. table: {}, partition: {}", - TimeUtils.longToTimeString(System.currentTimeMillis()), tableName, partitionName); - } finally { - db.writeUnlock(); - } - } finally { - // clear tasks - List tasks = deleteBatchTask.getAllTasks(); - for (AgentTask task : tasks) { - PushTask pushTask = (PushTask) task; - AgentTaskQueue.removePushTask(pushTask.getBackendId(), pushTask.getSignature(), - pushTask.getVersion(), pushTask.getVersionHash(), - pushTask.getPushType(), pushTask.getTaskType()); - } - - writeLock(); - try { - partitionUnderDelete.remove(partitionId); - } finally { - writeUnlock(); - } - } - } - public List> getAsyncDeleteJobInfo(long jobId) { LinkedList> infos = new LinkedList>(); readLock(); @@ -3685,5 +3284,3 @@ public Integer getLoadJobNumByTypeAndState(EtlJobType type, JobState state) { return num; } } - - diff --git a/fe/src/main/java/org/apache/doris/load/LoadChecker.java b/fe/src/main/java/org/apache/doris/load/LoadChecker.java index 83bb0abe3ca7f3..4950246eb0be06 100644 --- a/fe/src/main/java/org/apache/doris/load/LoadChecker.java +++ b/fe/src/main/java/org/apache/doris/load/LoadChecker.java @@ -21,6 +21,7 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; 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.Partition; @@ -395,7 +396,7 @@ private Set submitPushTasks(LoadJob job, Database db) { short replicationNum = table.getPartitionInfo().getReplicationNum(partition.getId()); // check all indices (base + roll up (not include ROLLUP state index)) - List indices = partition.getMaterializedIndices(); + List indices = partition.getMaterializedIndices(IndexExtState.ALL); for (MaterializedIndex index : indices) { long indexId = index.getId(); // if index is in rollup, then not load into it, be will automatically convert the data @@ -451,8 +452,6 @@ private Set submitPushTasks(LoadJob job, Database db) { TPushType type = TPushType.LOAD; if (job.isSyncDeleteJob()) { type = TPushType.DELETE; - } else if (job.getDeleteFlag()) { - type = TPushType.LOAD_DELETE; } // add task to batchTask diff --git a/fe/src/main/java/org/apache/doris/load/LoadJob.java b/fe/src/main/java/org/apache/doris/load/LoadJob.java index cb788a48af0600..d8c288f7f2a1b5 100644 --- a/fe/src/main/java/org/apache/doris/load/LoadJob.java +++ b/fe/src/main/java/org/apache/doris/load/LoadJob.java @@ -83,7 +83,6 @@ public enum JobState { long timestamp; private int timeoutSecond; private double maxFilterRatio; - private boolean deleteFlag; private JobState state; private BrokerDesc brokerDesc; @@ -148,7 +147,6 @@ public LoadJob(long id, long dbId, long tableId, long partitionId, String label, this.transactionId = -1; this.timestamp = -1; this.timeoutSecond = DEFAULT_TIMEOUT_S; - this.deleteFlag = true; this.state = JobState.LOADING; this.progress = 0; this.createTimeMs = System.currentTimeMillis(); @@ -200,7 +198,6 @@ public LoadJob(String label, int timeoutSecond, double maxFilterRatio) { this.timestamp = -1; this.timeoutSecond = timeoutSecond; this.maxFilterRatio = maxFilterRatio; - this.deleteFlag = false; this.state = JobState.PENDING; this.progress = 0; this.createTimeMs = System.currentTimeMillis(); @@ -284,14 +281,6 @@ public void setMaxFilterRatio(double maxFilterRatio) { public double getMaxFilterRatio() { return maxFilterRatio; } - - public void setDeleteFlag(boolean deleteFlag) { - this.deleteFlag = deleteFlag; - } - - public boolean getDeleteFlag() { - return deleteFlag; - } public JobState getState() { return state; @@ -647,7 +636,7 @@ public long getDeleteJobTimeout() { @Override public String toString() { return "LoadJob [id=" + id + ", dbId=" + dbId + ", label=" + label + ", timeoutSecond=" + timeoutSecond - + ", maxFilterRatio=" + maxFilterRatio + ", deleteFlag=" + deleteFlag + ", state=" + state + + ", maxFilterRatio=" + maxFilterRatio + ", state=" + state + ", progress=" + progress + ", createTimeMs=" + createTimeMs + ", etlStartTimeMs=" + etlStartTimeMs + ", etlFinishTimeMs=" + etlFinishTimeMs + ", loadStartTimeMs=" + loadStartTimeMs + ", loadFinishTimeMs=" + loadFinishTimeMs + ", failMsg=" + failMsg + ", etlJobType=" + etlJobType @@ -706,7 +695,7 @@ public void write(DataOutput out) throws IOException { out.writeLong(timestamp); out.writeInt(timeoutSecond); out.writeDouble(maxFilterRatio); - out.writeBoolean(deleteFlag); + out.writeBoolean(true); // delete flag, does not use anymore Text.writeString(out, state.name()); out.writeInt(progress); out.writeLong(createTimeMs); @@ -853,7 +842,7 @@ public void readFields(DataInput in) throws IOException { timeoutSecond = in.readInt(); maxFilterRatio = in.readDouble(); - deleteFlag = false; + boolean deleteFlag = false; if (version >= FeMetaVersion.VERSION_30) { deleteFlag = in.readBoolean(); } diff --git a/fe/src/main/java/org/apache/doris/load/PullLoadSourceInfo.java b/fe/src/main/java/org/apache/doris/load/PullLoadSourceInfo.java index b611467fb0f198..a211185502ae19 100644 --- a/fe/src/main/java/org/apache/doris/load/PullLoadSourceInfo.java +++ b/fe/src/main/java/org/apache/doris/load/PullLoadSourceInfo.java @@ -34,7 +34,6 @@ /** * PullLoadSourceInfo */ -@Deprecated public class PullLoadSourceInfo implements Writable { private static final Logger LOG = LogManager.getLogger(PullLoadSourceInfo.class); diff --git a/fe/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java b/fe/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java index cd0e091b0ed2b8..8192b0576d5e59 100644 --- a/fe/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java +++ b/fe/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java @@ -75,7 +75,6 @@ public class BrokerLoadJob extends LoadJob { private static final Logger LOG = LogManager.getLogger(BrokerLoadJob.class); // input params - private List dataDescriptions = Lists.newArrayList(); private BrokerDesc brokerDesc; // this param is used to persist the expr of columns // the origin stmt is persisted instead of columns expr @@ -92,12 +91,10 @@ public BrokerLoadJob() { this.jobType = EtlJobType.BROKER; } - public BrokerLoadJob(long dbId, String label, BrokerDesc brokerDesc, List dataDescriptions, - String originStmt) + private BrokerLoadJob(long dbId, String label, BrokerDesc brokerDesc, String originStmt) throws MetaNotFoundException { super(dbId, label); this.timeoutSecond = Config.broker_load_default_timeout_second; - this.dataDescriptions = dataDescriptions; this.brokerDesc = brokerDesc; this.originStmt = originStmt; this.jobType = EtlJobType.BROKER; @@ -111,27 +108,31 @@ public static BrokerLoadJob fromLoadStmt(LoadStmt stmt, String originStmt) throw if (db == null) { throw new DdlException("Database[" + dbName + "] does not exist"); } - // check data source info - LoadJob.checkDataSourceInfo(db, stmt.getDataDescriptions(), EtlJobType.BROKER); // create job try { BrokerLoadJob brokerLoadJob = new BrokerLoadJob(db.getId(), stmt.getLabel().getLabelName(), - stmt.getBrokerDesc(), stmt.getDataDescriptions(), - originStmt); + stmt.getBrokerDesc(), originStmt); brokerLoadJob.setJobProperties(stmt.getProperties()); - brokerLoadJob.setDataSourceInfo(db, stmt.getDataDescriptions()); + brokerLoadJob.checkAndDataSourceInfo(db, stmt.getDataDescriptions()); return brokerLoadJob; } catch (MetaNotFoundException e) { throw new DdlException(e.getMessage()); } } - private void setDataSourceInfo(Database db, List dataDescriptions) throws DdlException { - for (DataDescription dataDescription : dataDescriptions) { - BrokerFileGroup fileGroup = new BrokerFileGroup(dataDescription); - fileGroup.parse(db); - dataSourceInfo.addFileGroup(fileGroup); + private void checkAndDataSourceInfo(Database db, List dataDescriptions) throws DdlException { + // check data source info + db.readLock(); + try { + LoadJob.checkDataSourceInfo(db, dataDescriptions, EtlJobType.BROKER); + for (DataDescription dataDescription : dataDescriptions) { + BrokerFileGroup fileGroup = new BrokerFileGroup(dataDescription); + fileGroup.parse(db); + dataSourceInfo.addFileGroup(fileGroup); + } + } finally { + db.readUnlock(); } } @@ -270,7 +271,7 @@ public void analyze() { if (db == null) { throw new DdlException("Database[" + dbId + "] does not exist"); } - setDataSourceInfo(db, stmt.getDataDescriptions()); + checkAndDataSourceInfo(db, stmt.getDataDescriptions()); } catch (Exception e) { LOG.info(new LogBuilder(LogKey.LOAD_JOB, id) .add("origin_stmt", originStmt) @@ -484,7 +485,6 @@ protected void executeReplayOnVisible(TransactionState txnState) { public void write(DataOutput out) throws IOException { super.write(out); brokerDesc.write(out); - dataSourceInfo.write(out); Text.writeString(out, originStmt); } @@ -492,7 +492,9 @@ public void write(DataOutput out) throws IOException { public void readFields(DataInput in) throws IOException { super.readFields(in); brokerDesc = BrokerDesc.read(in); - dataSourceInfo.readFields(in); + if (Catalog.getCurrentCatalogJournalVersion() <= FeMetaVersion.VERSION_58) { + dataSourceInfo.readFields(in); + } if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_58) { originStmt = Text.readString(in); diff --git a/fe/src/main/java/org/apache/doris/load/loadv2/LoadJob.java b/fe/src/main/java/org/apache/doris/load/loadv2/LoadJob.java index bf637998b4ad74..191f9afce26900 100644 --- a/fe/src/main/java/org/apache/doris/load/loadv2/LoadJob.java +++ b/fe/src/main/java/org/apache/doris/load/loadv2/LoadJob.java @@ -266,7 +266,7 @@ protected static void checkDataSourceInfo(Database db, List dat // >> Map>> loadInfo = Maps.newHashMap(); // only support broker load now - Load.checkAndCreateSource(db, dataDescription, loadInfo, false, jobType); + Load.checkAndCreateSource(db, dataDescription, loadInfo, jobType); } } diff --git a/fe/src/main/java/org/apache/doris/load/loadv2/LoadManager.java b/fe/src/main/java/org/apache/doris/load/loadv2/LoadManager.java index c6022a514dbd74..8563a5e8df5fbc 100644 --- a/fe/src/main/java/org/apache/doris/load/loadv2/LoadManager.java +++ b/fe/src/main/java/org/apache/doris/load/loadv2/LoadManager.java @@ -201,6 +201,7 @@ public void createLoadJobV1FromStmt(LoadStmt stmt, EtlJobType jobType, long time * else: return true. * @throws DdlException */ + @Deprecated public boolean createLoadJobV1FromRequest(TMiniLoadRequest request) throws DdlException { String cluster = SystemInfoService.DEFAULT_CLUSTER; if (request.isSetCluster()) { @@ -210,7 +211,7 @@ public boolean createLoadJobV1FromRequest(TMiniLoadRequest request) throws DdlEx writeLock(); try { checkLabelUsed(database.getId(), request.getLabel(), -1); - return Catalog.getCurrentCatalog().getLoadInstance().addLoadJob(request); + return Catalog.getCurrentCatalog().getLoadInstance().addMiniLoadJob(request); } finally { writeUnlock(); } diff --git a/fe/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java b/fe/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java index e45d502dc24327..d1f8990633fbc3 100644 --- a/fe/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java +++ b/fe/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java @@ -95,7 +95,8 @@ public void plan(TUniqueId loadId, List> fileStatusesLis // Generate tuple descriptor List slotRefs = Lists.newArrayList(); TupleDescriptor tupleDesc = descTable.createTupleDescriptor(); - for (Column col : table.getBaseSchema()) { + // use full schema to fill the descriptor table + for (Column col : table.getFullSchema()) { SlotDescriptor slotDesc = descTable.addSlotDescriptor(tupleDesc); slotDesc.setIsMaterialized(true); slotDesc.setColumn(col); diff --git a/fe/src/main/java/org/apache/doris/master/Checkpoint.java b/fe/src/main/java/org/apache/doris/master/Checkpoint.java index b40c77fb82ac88..f356dc6fd30a8b 100644 --- a/fe/src/main/java/org/apache/doris/master/Checkpoint.java +++ b/fe/src/main/java/org/apache/doris/master/Checkpoint.java @@ -25,6 +25,7 @@ import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Table.TableType; import org.apache.doris.catalog.Tablet; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.common.Config; import org.apache.doris.common.util.Daemon; import org.apache.doris.metric.MetricRepo; @@ -249,7 +250,7 @@ private boolean checkMemoryEnoughToDoCheckpoint() { OlapTable olapTable = (OlapTable) table; for (Partition partition : olapTable.getPartitions()) { totalPartitionNum++; - for (MaterializedIndex materializedIndex : partition.getMaterializedIndices()) { + for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { totalIndexNum++; for (Tablet tablet : materializedIndex.getTablets()) { totalTabletNum++; diff --git a/fe/src/main/java/org/apache/doris/master/MasterImpl.java b/fe/src/main/java/org/apache/doris/master/MasterImpl.java index 3da5a9ac74dbe0..e087d75dcc5bba 100644 --- a/fe/src/main/java/org/apache/doris/master/MasterImpl.java +++ b/fe/src/main/java/org/apache/doris/master/MasterImpl.java @@ -18,6 +18,7 @@ package org.apache.doris.master; import org.apache.doris.alter.AlterJob; +import org.apache.doris.alter.AlterJobV2.JobType; import org.apache.doris.alter.RollupHandler; import org.apache.doris.alter.RollupJob; import org.apache.doris.alter.SchemaChangeHandler; @@ -40,6 +41,7 @@ import org.apache.doris.system.Backend; import org.apache.doris.task.AgentTask; import org.apache.doris.task.AgentTaskQueue; +import org.apache.doris.task.AlterReplicaTask; import org.apache.doris.task.CheckConsistencyTask; import org.apache.doris.task.ClearAlterTask; import org.apache.doris.task.ClearTransactionTask; @@ -132,7 +134,8 @@ public TMasterResult finishTask(TFinishTaskRequest request) throws TException { // We start to let FE perceive the task's error msg if (taskType != TTaskType.MAKE_SNAPSHOT && taskType != TTaskType.UPLOAD && taskType != TTaskType.DOWNLOAD && taskType != TTaskType.MOVE - && taskType != TTaskType.CLONE && taskType != TTaskType.PUBLISH_VERSION) { + && taskType != TTaskType.CLONE && taskType != TTaskType.PUBLISH_VERSION + && taskType != TTaskType.CREATE) { return result; } } @@ -143,7 +146,7 @@ public TMasterResult finishTask(TFinishTaskRequest request) throws TException { switch (taskType) { case CREATE: Preconditions.checkState(request.isSetReport_version()); - finishCreateReplica(task, request.getReport_version()); + finishCreateReplica(task, request); break; case PUSH: checkHasTabletInfo(request); @@ -199,6 +202,9 @@ public TMasterResult finishTask(TFinishTaskRequest request) throws TException { case RECOVER_TABLET: finishRecoverTablet(task); break; + case ALTER: + finishAlterTask(task); + break; default: break; } @@ -224,21 +230,27 @@ private void checkHasTabletInfo(TFinishTaskRequest request) throws Exception { } } - private void finishCreateReplica(AgentTask task, long reportVersion) { + private void finishCreateReplica(AgentTask task, TFinishTaskRequest request) { // if we get here, this task will be removed from AgentTaskQueue for certain. // because in this function, the only problem that cause failure is meta missing. // and if meta is missing, we no longer need to resend this task - - CreateReplicaTask createReplicaTask = (CreateReplicaTask) task; - long tabletId = createReplicaTask.getTabletId(); - - // this should be called before 'countDownLatch()' - Catalog.getCurrentSystemInfo().updateBackendReportVersion(task.getBackendId(), reportVersion, task.getDbId()); - - createReplicaTask.countDownLatch(task.getBackendId(), task.getSignature()); - LOG.debug("finish create replica. tablet id: {}, be: {}, report version: {}", - tabletId, task.getBackendId(), reportVersion); - AgentTaskQueue.removeTask(task.getBackendId(), TTaskType.CREATE, task.getSignature()); + try { + CreateReplicaTask createReplicaTask = (CreateReplicaTask) task; + if (request.getTask_status().getStatus_code() != TStatusCode.OK) { + createReplicaTask.countDownToZero(task.getBackendId() + ": " + request.getTask_status().getError_msgs().toString()); + } else { + long tabletId = createReplicaTask.getTabletId(); + + // this should be called before 'countDownLatch()' + Catalog.getCurrentSystemInfo().updateBackendReportVersion(task.getBackendId(), request.getReport_version(), task.getDbId()); + + createReplicaTask.countDownLatch(task.getBackendId(), task.getSignature()); + LOG.debug("finish create replica. tablet id: {}, be: {}, report version: {}", + tabletId, task.getBackendId(), request.getReport_version()); + } + } finally { + AgentTaskQueue.removeTask(task.getBackendId(), TTaskType.CREATE, task.getSignature()); + } } private void finishRealtimePush(AgentTask task, TFinishTaskRequest request) { @@ -759,4 +771,18 @@ public TFetchResourceResult fetchResource() { return Catalog.getInstance().getAuth().toResourceThrift(); } + private void finishAlterTask(AgentTask task) { + AlterReplicaTask alterTask = (AlterReplicaTask) task; + try { + if (alterTask.getJobType() == JobType.ROLLUP) { + Catalog.getCurrentCatalog().getRollupHandler().handleFinishAlterTask(alterTask); + } else if (alterTask.getJobType() == JobType.SCHEMA_CHANGE) { + Catalog.getCurrentCatalog().getSchemaChangeHandler().handleFinishAlterTask(alterTask); + } + alterTask.setFinished(true); + } catch (MetaNotFoundException e) { + LOG.warn("failed to handle finish alter task: {}", e.getMessage()); + } + AgentTaskQueue.removeTask(task.getBackendId(), TTaskType.ALTER, task.getSignature()); + } } diff --git a/fe/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/src/main/java/org/apache/doris/master/ReportHandler.java index 462ddde216144e..5225f54df5f749 100644 --- a/fe/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/src/main/java/org/apache/doris/master/ReportHandler.java @@ -22,6 +22,7 @@ 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.Partition; import org.apache.doris.catalog.Replica; @@ -521,6 +522,11 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta if (index == null) { continue; } + if (index.getState() == IndexState.SHADOW) { + // This index is under schema change or rollup, tablet may not be created on BE. + // ignore it. + continue; + } Tablet tablet = index.getTablet(tabletId); if (tablet == null) { @@ -591,9 +597,8 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta tablet.deleteReplicaByBackendId(backendId); ++deleteCounter; - // handle related task - Catalog.getInstance().handleJobsWhenDeleteReplica(tableId, partitionId, indexId, tabletId, - replica.getId(), backendId); + // remove replica related tasks + AgentTaskQueue.removeReplicaRelatedTasks(backendId, tabletId); // write edit log ReplicaPersistInfo info = ReplicaPersistInfo.createForDelete(dbId, tableId, partitionId, diff --git a/fe/src/main/java/org/apache/doris/persist/EditLog.java b/fe/src/main/java/org/apache/doris/persist/EditLog.java index 3a115ad939900e..9fe20493981176 100644 --- a/fe/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/src/main/java/org/apache/doris/persist/EditLog.java @@ -17,9 +17,12 @@ package org.apache.doris.persist; +import org.apache.doris.alter.AlterJobV2; import org.apache.doris.alter.DecommissionBackendJob; import org.apache.doris.alter.RollupJob; +import org.apache.doris.alter.RollupJobV2; import org.apache.doris.alter.SchemaChangeJob; +import org.apache.doris.alter.SchemaChangeJobV2; import org.apache.doris.analysis.UserIdentity; import org.apache.doris.backup.BackupJob; import org.apache.doris.backup.Repository; @@ -654,28 +657,42 @@ public static void loadJournal(Catalog catalog, JournalEntity journal) { } case OperationType.OP_REMOVE_ROUTINE_LOAD_JOB: { RoutineLoadOperation operation = (RoutineLoadOperation) journal.getData(); - Catalog.getCurrentCatalog().getRoutineLoadManager().replayRemoveOldRoutineLoad(operation); + catalog.getRoutineLoadManager().replayRemoveOldRoutineLoad(operation); break; } case OperationType.OP_CREATE_LOAD_JOB: { org.apache.doris.load.loadv2.LoadJob loadJob = (org.apache.doris.load.loadv2.LoadJob) journal.getData(); - Catalog.getCurrentCatalog().getLoadManager().replayCreateLoadJob(loadJob); + catalog.getLoadManager().replayCreateLoadJob(loadJob); break; } case OperationType.OP_END_LOAD_JOB: { LoadJobFinalOperation operation = (LoadJobFinalOperation) journal.getData(); - Catalog.getCurrentCatalog().getLoadManager().replayEndLoadJob(operation); + catalog.getLoadManager().replayEndLoadJob(operation); break; } case OperationType.OP_CREATE_SMALL_FILE: { SmallFile smallFile = (SmallFile) journal.getData(); - Catalog.getCurrentCatalog().getSmallFileMgr().replayCreateFile(smallFile); + catalog.getSmallFileMgr().replayCreateFile(smallFile); break; } case OperationType.OP_DROP_SMALL_FILE: { SmallFile smallFile = (SmallFile) journal.getData(); - Catalog.getCurrentCatalog().getSmallFileMgr().replayRemoveFile(smallFile); + catalog.getSmallFileMgr().replayRemoveFile(smallFile); + break; + } + case OperationType.OP_ALTER_JOB_V2: { + AlterJobV2 alterJob = (AlterJobV2) journal.getData(); + switch (alterJob.getType()) { + case ROLLUP: + catalog.getRollupHandler().replayAlterJobV2((RollupJobV2) alterJob); + break; + case SCHEMA_CHANGE: + catalog.getSchemaChangeHandler().replayAlterJobV2((SchemaChangeJobV2) alterJob); + break; + default: + break; + } break; } default: { @@ -1167,4 +1184,8 @@ public void logCreateSmallFile(SmallFile info) { public void logDropSmallFile(SmallFile info) { logEdit(OperationType.OP_DROP_SMALL_FILE, info); } + + public void logAlterJob(AlterJobV2 alterJob) { + logEdit(OperationType.OP_ALTER_JOB_V2, alterJob); + } } diff --git a/fe/src/main/java/org/apache/doris/persist/OperationType.java b/fe/src/main/java/org/apache/doris/persist/OperationType.java index 1eb8108b0e9c4d..5041cbc793ab28 100644 --- a/fe/src/main/java/org/apache/doris/persist/OperationType.java +++ b/fe/src/main/java/org/apache/doris/persist/OperationType.java @@ -54,6 +54,7 @@ public class OperationType { public static final short OP_CLEAR_ROLLUP_INFO = 28; public static final short OP_FINISH_CONSISTENCY_CHECK = 29; public static final short OP_RENAME_ROLLUP = 120; + public static final short OP_ALTER_JOB_V2 = 121; // 30~39 130~139 230~239 ... // load job for only hadoop load diff --git a/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java b/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java index 3cda72adf7425d..87cd0095b6dee5 100644 --- a/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java +++ b/fe/src/main/java/org/apache/doris/planner/BrokerScanNode.java @@ -17,6 +17,7 @@ package org.apache.doris.planner; +import org.apache.doris.alter.SchemaChangeHandler; import org.apache.doris.analysis.Analyzer; import org.apache.doris.analysis.ArithmeticExpr; import org.apache.doris.analysis.BinaryPredicate; @@ -252,61 +253,68 @@ private void initParams(ParamCreateContext context) throws AnalysisException, Us * @throws UserException */ private void initColumns(ParamCreateContext context) throws UserException { - // This tuple descriptor is used for origin file - TupleDescriptor srcTupleDesc = analyzer.getDescTbl().createTupleDescriptor(); - context.tupleDescriptor = srcTupleDesc; - Map slotDescByName = Maps.newHashMap(); - context.slotDescByName = slotDescByName; - - TBrokerScanRangeParams params = context.params; - // there are no columns transform + List sourceFileColumns = context.fileGroup.getFileFieldNames(); List originColumnNameToExprList = context.fileGroup.getColumnExprList(); - if (originColumnNameToExprList == null || originColumnNameToExprList.isEmpty()) { - for (Column column : targetTable.getBaseSchema()) { - SlotDescriptor slotDesc = analyzer.getDescTbl().addSlotDescriptor(srcTupleDesc); - slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR)); - slotDesc.setIsMaterialized(true); - // ISSUE A: src slot should be nullable even if the column is not nullable. - // because src slot is what we read from file, not represent to real column value. - // If column is not nullable, error will be thrown when filling the dest slot, - // which is not nullable - slotDesc.setIsNullable(true); - slotDescByName.put(column.getName(), slotDesc); - params.addToSrc_slot_ids(slotDesc.getId().asInt()); - } - params.setSrc_tuple_id(srcTupleDesc.getId().asInt()); - return; - } - - // there are columns expr which belong to load - Map columnNameToExpr = Maps.newHashMap(); - context.exprMap = columnNameToExpr; + // originColumnNameToExprList must has emlements. because it is always filled by user or by system + Preconditions.checkState(originColumnNameToExprList != null && !originColumnNameToExprList.isEmpty()); + + // 1. create source slots + context.tupleDescriptor = analyzer.getDescTbl().createTupleDescriptor(); + context.slotDescByName = Maps.newHashMap(); + for (String sourceColName : sourceFileColumns) { + SlotDescriptor slotDesc = analyzer.getDescTbl().addSlotDescriptor(context.tupleDescriptor); + slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR)); + slotDesc.setIsMaterialized(true); + // src slot should be nullable even if the column is not nullable. + // because src slot is what we read from file, not represent to real column value. + // If column is not nullable, error will be thrown when filling the dest slot, + // which is not nullable + slotDesc.setIsNullable(true); + context.params.addToSrc_slot_ids(slotDesc.getId().asInt()); + String realColName = targetTable.getColumn(sourceColName) == null ? sourceColName + : targetTable.getColumn(sourceColName).getName(); + context.slotDescByName.put(realColName, slotDesc); + } + context.params.setSrc_tuple_id(context.tupleDescriptor.getId().asInt()); + + // 2. handle column mapping exprs + context.exprMap = Maps.newHashMap(); for (ImportColumnDesc originColumnNameToExpr : originColumnNameToExprList) { - // make column name case match with real column name String columnName = originColumnNameToExpr.getColumnName(); Expr columnExpr = originColumnNameToExpr.getExpr(); - String realColName = targetTable.getColumn(columnName) == null ? columnName - : targetTable.getColumn(columnName).getName(); + Column col = targetTable.getColumn(columnName); + if (col == null) { + // maybe 1) shadow column, 2) unknown column + if (columnName.startsWith(SchemaChangeHandler.SHADOW_NAME_PRFIX)) { + /* + * The shadow column mapping expr is added when creating load job. + * But the load job is being actually scheduled, the schema change may already finished. + * So the shadow column may not be found here. + * We can just ignore this shadow column's mapping expr, like it does not exist. + */ + continue; + } else if (columnExpr == null) { + // this is a unknown column, but the column expr is null, so we just consider it as + // a placeholder column, ignore it + continue; + } + // unknown column but has column expr, which is not allowed. + throw new UserException("Unknown column(" + columnName + ")"); + } + Preconditions.checkNotNull(col, columnName); + String realColName = col.getName(); if (columnExpr != null) { columnExpr = transformHadoopFunctionExpr(columnName, columnExpr); - columnNameToExpr.put(realColName, columnExpr); - } else { - SlotDescriptor slotDesc = analyzer.getDescTbl().addSlotDescriptor(srcTupleDesc); - slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR)); - slotDesc.setIsMaterialized(true); - // same as ISSUE A - slotDesc.setIsNullable(true); - params.addToSrc_slot_ids(slotDesc.getId().asInt()); - slotDescByName.put(realColName, slotDesc); + context.exprMap.put(realColName, columnExpr); } } - // analyze all exprs - for (Map.Entry entry : columnNameToExpr.entrySet()) { + // analyze all column mapping exprs + for (Map.Entry entry : context.exprMap.entrySet()) { ExprSubstitutionMap smap = new ExprSubstitutionMap(); List slots = Lists.newArrayList(); entry.getValue().collect(SlotRef.class, slots); for (SlotRef slot : slots) { - SlotDescriptor slotDesc = slotDescByName.get(slot.getColumnName()); + SlotDescriptor slotDesc = context.slotDescByName.get(slot.getColumnName()); if (slotDesc == null) { throw new UserException("unknown reference column, column=" + entry.getKey() + ", reference=" + slot.getColumnName()); @@ -325,11 +333,9 @@ private void initColumns(ParamCreateContext context) throws UserException { throw new AnalysisException("Don't support aggregation function in load expression"); } } - - columnNameToExpr.put(entry.getKey(), expr); + context.exprMap.put(entry.getKey(), expr); } - params.setSrc_tuple_id(srcTupleDesc.getId().asInt()); - + LOG.debug("after init column, exprMap: {}", context.exprMap); } /** @@ -432,6 +438,8 @@ private Expr transformHadoopFunctionExpr(String columnName, Expr originExpr) thr FunctionName nowFunctionName = new FunctionName("NOW"); FunctionCallExpr newFunc = new FunctionCallExpr(nowFunctionName, new FunctionParams(null)); return newFunc; + } else if (funcName.equalsIgnoreCase("substitute")) { + return funcExpr.getChild(0); } } return originExpr; @@ -469,7 +477,7 @@ private void finalizeParams(ParamCreateContext context) throws UserException, An expr = NullLiteral.create(column.getType()); } else { throw new UserException("Unknown slot ref(" - + destSlotDesc.getColumn().getName() + ") in source file"); + + destSlotDesc.getColumn().getName() + ") in source file"); } } } diff --git a/fe/src/main/java/org/apache/doris/planner/DataSplitSink.java b/fe/src/main/java/org/apache/doris/planner/DataSplitSink.java index 8e8f25ba8f0def..d478d02d42eb6a 100644 --- a/fe/src/main/java/org/apache/doris/planner/DataSplitSink.java +++ b/fe/src/main/java/org/apache/doris/planner/DataSplitSink.java @@ -62,7 +62,7 @@ // This class used to split data read from file to batch @Deprecated public class DataSplitSink extends DataSink { - private static final Logger LOG = LogManager.getLogger(Planner.class); + private static final Logger LOG = LogManager.getLogger(DataSplitSink.class); private final OlapTable targetTable; diff --git a/fe/src/main/java/org/apache/doris/planner/OlapTableSink.java b/fe/src/main/java/org/apache/doris/planner/OlapTableSink.java index f69754afe6c9e5..97462e2be4f3aa 100644 --- a/fe/src/main/java/org/apache/doris/planner/OlapTableSink.java +++ b/fe/src/main/java/org/apache/doris/planner/OlapTableSink.java @@ -24,6 +24,7 @@ import org.apache.doris.catalog.DistributionInfo; import org.apache.doris.catalog.HashDistributionInfo; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.PartitionKey; @@ -77,11 +78,6 @@ public class OlapTableSink extends DataSink { // set after init called private TDataSink tDataSink; - public OlapTableSink(OlapTable dstTable, TupleDescriptor tupleDescriptor) { - this.dstTable = dstTable; - this.tupleDescriptor = tupleDescriptor; - } - public OlapTableSink(OlapTable dstTable, TupleDescriptor tupleDescriptor, String partitions) { this.dstTable = dstTable; this.tupleDescriptor = tupleDescriptor; @@ -179,8 +175,9 @@ private TOlapTableSchemaParam createSchema(long dbId, OlapTable table) { for (Map.Entry> pair : table.getIndexIdToSchema().entrySet()) { List columns = Lists.newArrayList(); columns.addAll(pair.getValue().stream().map(Column::getName).collect(Collectors.toList())); - schemaParam.addToIndexes(new TOlapTableIndexSchema(pair.getKey(), columns, - table.getSchemaHashByIndexId(pair.getKey()))); + TOlapTableIndexSchema indexSchema = new TOlapTableIndexSchema(pair.getKey(), columns, + table.getSchemaHashByIndexId(pair.getKey())); + schemaParam.addToIndexes(indexSchema); } return schemaParam; } @@ -242,8 +239,8 @@ private TOlapTablePartitionParam createPartition(long dbId, OlapTable table) thr tPartition.addToEnd_keys(range.upperEndpoint().getKeys().get(i).treeToThrift().getNodes().get(0)); } } - - for (MaterializedIndex index : partition.getMaterializedIndices()) { + + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { tPartition.addToIndexes(new TOlapTableIndexTablets(index.getId(), Lists.newArrayList( index.getTablets().stream().map(Tablet::getId).collect(Collectors.toList())))); tPartition.setNum_buckets(index.getTablets().size()); @@ -273,7 +270,7 @@ private TOlapTablePartitionParam createPartition(long dbId, OlapTable table) thr TOlapTablePartition tPartition = new TOlapTablePartition(); tPartition.setId(partition.getId()); // No lowerBound and upperBound for this range - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { tPartition.addToIndexes(new TOlapTableIndexTablets(index.getId(), Lists.newArrayList( index.getTablets().stream().map(Tablet::getId).collect(Collectors.toList())))); tPartition.setNum_buckets(index.getTablets().size()); @@ -294,7 +291,7 @@ private TOlapTableLocationParam createLocation(OlapTable table) throws UserExcep TOlapTableLocationParam locationParam = new TOlapTableLocationParam(); for (Partition partition : table.getPartitions()) { int quorum = table.getPartitionInfo().getReplicationNum(partition.getId()) / 2 + 1; - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { // we should ensure the replica backend is alive // otherwise, there will be a 'unknown node id, id=xxx' error for stream load for (Tablet tablet : index.getTablets()) { diff --git a/fe/src/main/java/org/apache/doris/planner/Planner.java b/fe/src/main/java/org/apache/doris/planner/Planner.java index 857fc9e923c570..3b34fc8989a2bd 100644 --- a/fe/src/main/java/org/apache/doris/planner/Planner.java +++ b/fe/src/main/java/org/apache/doris/planner/Planner.java @@ -145,7 +145,6 @@ public void createPlanFragments(StatementBase statment, Analyzer analyzer, TQuer singleNodePlanner = new SingleNodePlanner(plannerContext); PlanNode singleNodePlan = singleNodePlanner.createSingleNodePlan(); - List resultExprs = queryStmt.getResultExprs(); if (statment instanceof InsertStmt) { InsertStmt insertStmt = (InsertStmt) statment; insertStmt.prepareExpressions(); diff --git a/fe/src/main/java/org/apache/doris/planner/RollupSelector.java b/fe/src/main/java/org/apache/doris/planner/RollupSelector.java index b9abc376f15a4c..610040552efa02 100644 --- a/fe/src/main/java/org/apache/doris/planner/RollupSelector.java +++ b/fe/src/main/java/org/apache/doris/planner/RollupSelector.java @@ -17,22 +17,25 @@ package org.apache.doris.planner; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; import org.apache.doris.analysis.Analyzer; +import org.apache.doris.analysis.BinaryPredicate; import org.apache.doris.analysis.CastExpr; import org.apache.doris.analysis.Expr; -import org.apache.doris.analysis.BinaryPredicate; import org.apache.doris.analysis.InPredicate; import org.apache.doris.analysis.SlotDescriptor; import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.common.UserException; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -104,9 +107,7 @@ private List selectBestPrefixIndexRollup( outputColumns.add(col.getName()); } - final List rollups = Lists.newArrayList(); - rollups.add(partition.getBaseIndex()); - rollups.addAll(partition.getRollupIndices()); + final List rollups = partition.getMaterializedIndices(IndexExtState.VISIBLE); LOG.debug("num of rollup(base included): {}, pre aggr: {}", rollups.size(), isPreAggregation); // 1. find all rollup indexes which contains all tuple columns diff --git a/fe/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java b/fe/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java index bb6b28de23f543..e38d660d62714d 100644 --- a/fe/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java +++ b/fe/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java @@ -85,7 +85,8 @@ public TExecPlanFragmentParams plan(TUniqueId loadId) throws UserException { // construct tuple descriptor, used for scanNode and dataSink TupleDescriptor tupleDesc = descTable.createTupleDescriptor("DstTableTuple"); boolean negative = streamLoadTask.getNegative(); - for (Column col : destTable.getBaseSchema()) { + // here we should be full schema to fill the descriptor table + for (Column col : destTable.getFullSchema()) { SlotDescriptor slotDesc = descTable.addSlotDescriptor(tupleDesc); slotDesc.setIsMaterialized(true); slotDesc.setColumn(col); diff --git a/fe/src/main/java/org/apache/doris/planner/StreamLoadScanNode.java b/fe/src/main/java/org/apache/doris/planner/StreamLoadScanNode.java index ad64c514b895b5..0565012f9405de 100644 --- a/fe/src/main/java/org/apache/doris/planner/StreamLoadScanNode.java +++ b/fe/src/main/java/org/apache/doris/planner/StreamLoadScanNode.java @@ -17,6 +17,7 @@ package org.apache.doris.planner; +import org.apache.doris.alter.SchemaChangeHandler; import org.apache.doris.analysis.Analyzer; import org.apache.doris.analysis.ArithmeticExpr; import org.apache.doris.analysis.Expr; @@ -49,6 +50,7 @@ import org.apache.doris.thrift.TScanRangeLocations; import org.apache.doris.thrift.TUniqueId; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -123,67 +125,85 @@ public void init(Analyzer analyzer) throws UserException { // columns: k1, k2, v1, v2=k1 + k2 // this means that there are three columns(k1, k2, v1) in source file, // and v2 is derived from (k1 + k2) - if (streamLoadTask.getColumnExprDesc() != null && !streamLoadTask.getColumnExprDesc().isEmpty()) { - for (ImportColumnDesc importColumnDesc : streamLoadTask.getColumnExprDesc()) { - // make column name case match with real column name - String columnName = importColumnDesc.getColumnName(); - String realColName = dstTable.getColumn(columnName) == null ? columnName - : dstTable.getColumn(columnName).getName(); - if (importColumnDesc.getExpr() != null) { - exprsByName.put(realColName, importColumnDesc.getExpr()); - } else { - SlotDescriptor slotDesc = analyzer.getDescTbl().addSlotDescriptor(srcTupleDesc); - slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR)); - slotDesc.setIsMaterialized(true); - // ISSUE A: src slot should be nullable even if the column is not nullable. - // because src slot is what we read from file, not represent to real column value. - // If column is not nullable, error will be thrown when filling the dest slot, - // which is not nullable - slotDesc.setIsNullable(true); - params.addToSrc_slot_ids(slotDesc.getId().asInt()); - slotDescByName.put(realColName, slotDesc); - } - } - - // analyze all exprs - for (Map.Entry entry : exprsByName.entrySet()) { - ExprSubstitutionMap smap = new ExprSubstitutionMap(); - List slots = Lists.newArrayList(); - entry.getValue().collect(SlotRef.class, slots); - for (SlotRef slot : slots) { - SlotDescriptor slotDesc = slotDescByName.get(slot.getColumnName()); - if (slotDesc == null) { - throw new UserException("unknown reference column, column=" + entry.getKey() - + ", reference=" + slot.getColumnName()); - } - smap.getLhs().add(slot); - smap.getRhs().add(new SlotRef(slotDesc)); - } - Expr expr = entry.getValue().clone(smap); - expr.analyze(analyzer); - // check if contain aggregation - List funcs = Lists.newArrayList(); - expr.collect(FunctionCallExpr.class, funcs); - for (FunctionCallExpr fn : funcs) { - if (fn.isAggregateFunction()) { - throw new AnalysisException("Don't support aggregation function in load expression"); - } - } + // If user does not specify the column expr descs, generate it by using base schema of table. + // So that the following process can be unified + if (streamLoadTask.getColumnExprDescs() == null || streamLoadTask.getColumnExprDescs().isEmpty()) { + List columns = dstTable.getBaseSchema(); + for (Column column : columns) { + ImportColumnDesc columnDesc = new ImportColumnDesc(column.getName()); + LOG.debug("add base column {} to stream load task", column.getName()); + streamLoadTask.addColumnExprDesc(columnDesc); + } + } - exprsByName.put(entry.getKey(), expr); + // When doing schema change, there may have some 'shadow' columns, with prefix '__doris_shadow_' in + // their names. These columns are visible to user, but we need to generate data for these columns. + // So we add column mappings for these column. + // eg: + // base schema is (A, B, C), and B is under schema change, so there will be a shadow column: '__doris_shadow_B' + // So the final column mapping should looks like: (A, B, C, __doris_shadow_B = B); + List fullSchema = dstTable.getFullSchema(); + for (Column column : fullSchema) { + if (column.isNameWithPrefix(SchemaChangeHandler.SHADOW_NAME_PRFIX)) { + String baseColName = column.getNameWithoutPrefix(SchemaChangeHandler.SHADOW_NAME_PRFIX); + ImportColumnDesc columnDesc = new ImportColumnDesc(column.getName(), new SlotRef(null, baseColName)); + LOG.debug("add shadow column {} to stream load task, base name: {}", column.getName(), baseColName); + streamLoadTask.addColumnExprDesc(columnDesc); } - } else { - for (Column column : dstTable.getBaseSchema()) { + } + Preconditions.checkState(streamLoadTask.getColumnExprDescs() != null); + Preconditions.checkState(!streamLoadTask.getColumnExprDescs().isEmpty()); + + for (ImportColumnDesc importColumnDesc : streamLoadTask.getColumnExprDescs()) { + // make column name case match with real column name + String columnName = importColumnDesc.getColumnName(); + String realColName = dstTable.getColumn(columnName) == null ? columnName + : dstTable.getColumn(columnName).getName(); + if (importColumnDesc.getExpr() != null) { + exprsByName.put(realColName, importColumnDesc.getExpr()); + } else { SlotDescriptor slotDesc = analyzer.getDescTbl().addSlotDescriptor(srcTupleDesc); slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR)); slotDesc.setIsMaterialized(true); - // same as ISSUE A + // ISSUE A: src slot should be nullable even if the column is not nullable. + // because src slot is what we read from file, not represent to real column value. + // If column is not nullable, error will be thrown when filling the dest slot, + // which is not nullable. slotDesc.setIsNullable(true); params.addToSrc_slot_ids(slotDesc.getId().asInt()); + slotDescByName.put(realColName, slotDesc); + } + } + + LOG.debug("slotDescByName: {}, exprsByName: {}", slotDescByName, exprsByName); - slotDescByName.put(column.getName(), slotDesc); + // analyze all exprs + for (Map.Entry entry : exprsByName.entrySet()) { + ExprSubstitutionMap smap = new ExprSubstitutionMap(); + List slots = Lists.newArrayList(); + entry.getValue().collect(SlotRef.class, slots); + for (SlotRef slot : slots) { + SlotDescriptor slotDesc = slotDescByName.get(slot.getColumnName()); + if (slotDesc == null) { + throw new UserException("unknown reference column, column=" + entry.getKey() + + ", reference=" + slot.getColumnName()); + } + smap.getLhs().add(slot); + smap.getRhs().add(new SlotRef(slotDesc)); + } + Expr expr = entry.getValue().clone(smap); + expr.analyze(analyzer); + + // check if contain aggregation + List funcs = Lists.newArrayList(); + expr.collect(FunctionCallExpr.class, funcs); + for (FunctionCallExpr fn : funcs) { + if (fn.isAggregateFunction()) { + throw new AnalysisException("Don't support aggregation function in load expression"); + } } + exprsByName.put(entry.getKey(), expr); } // analyze where statement diff --git a/fe/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/src/main/java/org/apache/doris/qe/Coordinator.java index 9d072e1180ee18..4302715d1ce947 100644 --- a/fe/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/src/main/java/org/apache/doris/qe/Coordinator.java @@ -45,8 +45,8 @@ import org.apache.doris.planner.ResultSink; import org.apache.doris.planner.ScanNode; import org.apache.doris.planner.UnionNode; -import org.apache.doris.proto.PPlanFragmentCancelReason; import org.apache.doris.proto.PExecPlanFragmentResult; +import org.apache.doris.proto.PPlanFragmentCancelReason; import org.apache.doris.rpc.BackendServiceProxy; import org.apache.doris.rpc.RpcException; import org.apache.doris.service.FrontendOptions; @@ -672,7 +672,7 @@ private void cancelInternal(PPlanFragmentCancelReason cancelReason) { cancelRemoteFragmentsAsync(cancelReason); if (profileDoneSignal != null) { // count down to zero to notify all objects waiting for this - profileDoneSignal.countDownToZero(); + profileDoneSignal.countDownToZero(new Status()); LOG.info("unfinished instance: {}", profileDoneSignal.getLeftMarks().stream().map(e->DebugUtil.printId(e.getKey())).toArray()); } } diff --git a/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java b/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java index e6364302e63bec..56137aeadcb58c 100644 --- a/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java @@ -115,7 +115,7 @@ public static void execute(Catalog catalog, DdlStmt ddlStmt, String origStmt) th } jobType = EtlJobType.HADOOP; } - if (loadStmt.getVersion().equals(Load.VERSION) || loadStmt.getBrokerDesc() == null) { + if (loadStmt.getVersion().equals(Load.VERSION) || jobType == EtlJobType.HADOOP) { catalog.getLoadManager().createLoadJobV1FromStmt(loadStmt, jobType, System.currentTimeMillis()); } else { catalog.getLoadManager().createLoadJobFromStmt(loadStmt, origStmt); diff --git a/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java index 18c90b067757f9..d93afb0f5dde8e 100644 --- a/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -70,9 +70,11 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Function; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.MetadataViewer; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.ScalarFunction; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Tablet; @@ -1118,6 +1120,21 @@ private void handleShowTablet() throws AnalysisException { isSync = false; break; } + + List replicas = tablet.getReplicas(); + for (Replica replica : replicas) { + Replica tmp = invertedIndex.getReplica(tabletId, replica.getBackendId()); + if (tmp == null) { + isSync = false; + break; + } + // use !=, not equals(), because this should be the same object. + if (tmp != replica) { + isSync = false; + break; + } + } + } finally { db.readUnlock(); } @@ -1146,9 +1163,13 @@ private void handleShowTablet() throws AnalysisException { } OlapTable olapTable = (OlapTable) table; - + String indexName = showStmt.getIndexName(); for (Partition partition : olapTable.getPartitions()) { - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { + if (!Strings.isNullOrEmpty(indexName) && !olapTable.getIndexNameById(index.getId()).equalsIgnoreCase(indexName)) { + // only show the specified index + continue; + } TabletsProcDir procDir = new TabletsProcDir(db, index); rows.addAll(procDir.fetchResult().getRows()); } diff --git a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java index ede891439b0e1f..916f7b2f64c141 100644 --- a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -639,12 +639,6 @@ private void handleInsertStmt() throws Exception { return; } - if (insertStmt.getQueryStmt() != null && (coord.getCommitInfos() == null || coord.getCommitInfos().isEmpty())) { - Catalog.getCurrentGlobalTransactionMgr().abortTransaction(insertStmt.getTransactionId(), "select stmt return empty set when insert"); - context.getState().setOk(); - return; - } - Catalog.getCurrentGlobalTransactionMgr().commitAndPublishTransaction( insertStmt.getDbObj(), insertStmt.getTransactionId(), TabletCommitInfo.fromThrift(coord.getCommitInfos()), diff --git a/fe/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index ccdba2aef1f0c9..c25fde486ed0f6 100644 --- a/fe/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -335,6 +335,7 @@ public TFetchResourceResult fetchResource() throws TException { return masterImpl.fetchResource(); } + @Deprecated @Override public TFeResult miniLoad(TMiniLoadRequest request) throws TException { LOG.info("receive mini load request: label: {}, db: {}, tbl: {}, backend: {}", @@ -812,7 +813,8 @@ private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request) } StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request); StreamLoadPlanner planner = new StreamLoadPlanner(db, (OlapTable) table, streamLoadTask); - return planner.plan(streamLoadTask.getId()); + TExecPlanFragmentParams plan = planner.plan(streamLoadTask.getId()); + return plan; } finally { db.readUnlock(); } diff --git a/fe/src/main/java/org/apache/doris/task/AgentBatchTask.java b/fe/src/main/java/org/apache/doris/task/AgentBatchTask.java index 6f09c3016f4099..ea5077b475e6eb 100644 --- a/fe/src/main/java/org/apache/doris/task/AgentBatchTask.java +++ b/fe/src/main/java/org/apache/doris/task/AgentBatchTask.java @@ -24,6 +24,7 @@ import org.apache.doris.thrift.TAgentServiceVersion; import org.apache.doris.thrift.TAgentTaskRequest; import org.apache.doris.thrift.TAlterTabletReq; +import org.apache.doris.thrift.TAlterTabletReqV2; import org.apache.doris.thrift.TCheckConsistencyReq; import org.apache.doris.thrift.TClearAlterTaskRequest; import org.apache.doris.thrift.TClearTransactionTaskRequest; @@ -44,6 +45,8 @@ import org.apache.doris.thrift.TUpdateTabletMetaInfoReq; import org.apache.doris.thrift.TUploadReq; +import com.google.common.collect.Lists; + import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -101,6 +104,47 @@ public int getTaskNum() { return num; } + // return true only if all tasks are finished. + // NOTICE that even if AgentTask.isFinished() return false, it does not mean that task is not finished. + // this depends on caller's logic. See comments on 'isFinished' member. + public boolean isFinished() { + for (List tasks : this.backendIdToTasks.values()) { + for (AgentTask agentTask : tasks) { + if (!agentTask.isFinished()) { + return false; + } + } + } + return true; + } + + // return the limit number of unfinished tasks. + public List getUnfinishedTasks(int limit) { + List res = Lists.newArrayList(); + for (List tasks : this.backendIdToTasks.values()) { + for (AgentTask agentTask : tasks) { + if (!agentTask.isFinished()) { + if (res.size() < limit) { + res.add(agentTask); + } + } + } + } + return res; + } + + public int getFinishedTaskNum() { + int count = 0; + for (List tasks : this.backendIdToTasks.values()) { + for (AgentTask agentTask : tasks) { + if (agentTask.isFinished()) { + count++; + } + } + } + return count; + } + @Override public void run() { for (Long backendId : this.backendIdToTasks.keySet()) { @@ -315,6 +359,14 @@ private TAgentTaskRequest toAgentTaskRequest(AgentTask task) { LOG.debug(request.toString()); } tAgentTaskRequest.setUpdate_tablet_meta_info_req(request); + } + case ALTER: { + AlterReplicaTask createRollupTask = (AlterReplicaTask) task; + TAlterTabletReqV2 request = createRollupTask.toThrift(); + if (LOG.isDebugEnabled()) { + LOG.debug(request.toString()); + } + tAgentTaskRequest.setAlter_tablet_req_v2(request); return tAgentTaskRequest; } default: diff --git a/fe/src/main/java/org/apache/doris/task/AgentTask.java b/fe/src/main/java/org/apache/doris/task/AgentTask.java index 807d859c557d3e..60486e73791d77 100644 --- a/fe/src/main/java/org/apache/doris/task/AgentTask.java +++ b/fe/src/main/java/org/apache/doris/task/AgentTask.java @@ -34,6 +34,10 @@ public abstract class AgentTask { protected TResourceInfo resourceInfo; protected int failedTimes; + // some of process may use this member to check if the task is finished. + // some of are not. + // so whether the task is finished depends on caller's logic, not the value of this member. + protected boolean isFinished = false; public AgentTask(TResourceInfo resourceInfo, long backendId, TTaskType taskType, long dbId, long tableId, long partitionId, long indexId, long tabletId, long signature) { @@ -101,6 +105,14 @@ public int getFailedTimes() { return this.failedTimes; } + public void setFinished(boolean isFinished) { + this.isFinished = isFinished; + } + + public boolean isFinished() { + return isFinished; + } + @Override public String toString() { return "[" + taskType + "], signature: " + signature + ", backendId: " + backendId + ", tablet id: " + tabletId; diff --git a/fe/src/main/java/org/apache/doris/task/AgentTaskQueue.java b/fe/src/main/java/org/apache/doris/task/AgentTaskQueue.java index 38e4fdfab3a8b9..48e02054ebd6bf 100644 --- a/fe/src/main/java/org/apache/doris/task/AgentTaskQueue.java +++ b/fe/src/main/java/org/apache/doris/task/AgentTaskQueue.java @@ -44,6 +44,12 @@ public class AgentTaskQueue { // backend id -> (task type -> (signature -> agent task)) private static Table> tasks = HashBasedTable.create(); private static int taskNum = 0; + + public static synchronized void addBatchTask(AgentBatchTask batchTask) { + for (AgentTask task : batchTask.getAllTasks()) { + addTask(task); + } + } public static synchronized boolean addTask(AgentTask task) { long backendId = task.getBackendId(); @@ -70,6 +76,14 @@ public static synchronized boolean addTask(AgentTask task) { return true; } + // remove all task in AgentBatchTask. + // the caller should make sure all tasks in AgentBatchTask is type of 'type' + public static synchronized void removeBatchTask(AgentBatchTask batchTask, TTaskType type) { + for (AgentTask task : batchTask.getAllTasks()) { + removeTask(task.getBackendId(), type, task.getSignature()); + } + } + public static synchronized void removeTask(long backendId, TTaskType type, long signature) { if (!tasks.contains(backendId, type)) { return; diff --git a/fe/src/main/java/org/apache/doris/task/AlterReplicaTask.java b/fe/src/main/java/org/apache/doris/task/AlterReplicaTask.java new file mode 100644 index 00000000000000..d682383b14c38c --- /dev/null +++ b/fe/src/main/java/org/apache/doris/task/AlterReplicaTask.java @@ -0,0 +1,98 @@ +// 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.task; + +import org.apache.doris.alter.AlterJobV2; +import org.apache.doris.thrift.TAlterTabletReqV2; +import org.apache.doris.thrift.TTaskType; + +/* + * This task is used for alter table process, such as rollup and schema change + * The task will do data transformation from base replica to new replica. + * The new replica should be created before. + * The new replica can be a rollup replica, or a shadow replica of schema change. + */ +public class AlterReplicaTask extends AgentTask { + + private long baseTabletId; + private long newReplicaId; + private int baseSchemaHash; + private int newSchemaHash; + private long version; + private long versionHash; + private long jobId; + private AlterJobV2.JobType jobType; + + public AlterReplicaTask(long backendId, long dbId, long tableId, + long partitionId, long rollupIndexId, long baseIndexId, long rollupTabletId, + long baseTabletId, long newReplicaId, int newSchemaHash, int baseSchemaHash, + long version, long versionHash, long jobId, AlterJobV2.JobType jobType) { + super(null, backendId, TTaskType.ALTER, dbId, tableId, partitionId, rollupIndexId, rollupTabletId); + + this.baseTabletId = baseTabletId; + this.newReplicaId = newReplicaId; + + this.newSchemaHash = newSchemaHash; + this.baseSchemaHash = baseSchemaHash; + + this.version = version; + this.versionHash = versionHash; + this.jobId = jobId; + + this.jobType = jobType; + } + + public long getBaseTabletId() { + return baseTabletId; + } + + public long getNewReplicaId() { + return newReplicaId; + } + + public int getNewSchemaHash() { + return newSchemaHash; + } + + public int getBaseSchemaHash() { + return baseSchemaHash; + } + + public long getVersion() { + return version; + } + + public long getVersionHash() { + return versionHash; + } + + public long getJobId() { + return jobId; + } + + public AlterJobV2.JobType getJobType() { + return jobType; + } + + public TAlterTabletReqV2 toThrift() { + TAlterTabletReqV2 req = new TAlterTabletReqV2(baseTabletId, signature, baseSchemaHash, newSchemaHash); + req.setAlter_version(version); + req.setAlter_version_hash(versionHash); + return req; + } +} diff --git a/fe/src/main/java/org/apache/doris/task/CreateReplicaTask.java b/fe/src/main/java/org/apache/doris/task/CreateReplicaTask.java index 4cb938d698f9db..9da5f4f66af7fb 100644 --- a/fe/src/main/java/org/apache/doris/task/CreateReplicaTask.java +++ b/fe/src/main/java/org/apache/doris/task/CreateReplicaTask.java @@ -17,11 +17,14 @@ package org.apache.doris.task; +import org.apache.doris.alter.SchemaChangeHandler; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.KeysType; import org.apache.doris.common.MarkedCountDownLatch; +import org.apache.doris.common.Status; import org.apache.doris.thrift.TColumn; import org.apache.doris.thrift.TCreateTabletReq; +import org.apache.doris.thrift.TStatusCode; import org.apache.doris.thrift.TStorageMedium; import org.apache.doris.thrift.TStorageType; import org.apache.doris.thrift.TTabletSchema; @@ -54,15 +57,19 @@ public class CreateReplicaTask extends AgentTask { private double bfFpp; // used for synchronous process - private MarkedCountDownLatch latch; + private MarkedCountDownLatch latch; private boolean inRestoreMode = false; + // if base tablet id is set, BE will create the replica on same disk as this base tablet + private long baseTabletId = -1; + private int baseSchemaHash = -1; + public CreateReplicaTask(long backendId, long dbId, long tableId, long partitionId, long indexId, long tabletId, short shortKeyColumnCount, int schemaHash, long version, long versionHash, KeysType keysType, TStorageType storageType, TStorageMedium storageMedium, List columns, - Set bfColumns, double bfFpp, MarkedCountDownLatch latch) { + Set bfColumns, double bfFpp, MarkedCountDownLatch latch) { super(null, backendId, TTaskType.CREATE, dbId, tableId, partitionId, indexId, tabletId); this.shortKeyColumnCount = shortKeyColumnCount; @@ -92,7 +99,15 @@ public void countDownLatch(long backendId, long tabletId) { } } - public void setLatch(MarkedCountDownLatch latch) { + // call this always means one of tasks is failed. count down to zero to finish entire task + public void countDownToZero(String errMsg) { + if (this.latch != null) { + latch.countDownToZero(new Status(TStatusCode.CANCELLED, errMsg)); + LOG.debug("CreateReplicaTask download to zero. error msg: {}", errMsg); + } + } + + public void setLatch(MarkedCountDownLatch latch) { this.latch = latch; } @@ -100,6 +115,11 @@ public void setInRestoreMode(boolean inRestoreMode) { this.inRestoreMode = inRestoreMode; } + public void setBaseTablet(long baseTabletId, int baseSchemaHash) { + this.baseTabletId = baseTabletId; + this.baseSchemaHash = baseSchemaHash; + } + public TCreateTabletReq toThrift() { TCreateTabletReq createTabletReq = new TCreateTabletReq(); createTabletReq.setTablet_id(tabletId); @@ -117,6 +137,11 @@ public TCreateTabletReq toThrift() { if (bfColumns != null && bfColumns.contains(column.getName())) { tColumn.setIs_bloom_filter_column(true); } + // when doing schema change, some modified column has a prefix in name. + // this prefix is only used in FE, not visible to BE, so we should remove this prefix. + if(column.getName().startsWith(SchemaChangeHandler.SHADOW_NAME_PRFIX)) { + tColumn.setColumn_name(column.getName().substring(SchemaChangeHandler.SHADOW_NAME_PRFIX.length())); + } tColumns.add(tColumn); } tSchema.setColumns(tColumns); @@ -136,6 +161,11 @@ public TCreateTabletReq toThrift() { createTabletReq.setTable_id(tableId); createTabletReq.setPartition_id(partitionId); + if (baseTabletId != -1) { + createTabletReq.setBase_tablet_id(baseTabletId); + createTabletReq.setBase_schema_hash(baseSchemaHash); + } + return createTabletReq; } } diff --git a/fe/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java b/fe/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java index 4d8334b08020ef..fb11b3c407ce62 100644 --- a/fe/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java +++ b/fe/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java @@ -204,7 +204,7 @@ private Map createEtlIndices(OlapTable table, long partitionId } else { aggregation = aggregateType.name(); } - } else if ("UNIQUE_KEYS" == table.getKeysType().name()) { + } else if (table.getKeysType().name().equalsIgnoreCase("UNIQUE_KEYS")) { aggregation = "REPLACE"; } dppColumn.put("aggregation_method", aggregation); diff --git a/fe/src/main/java/org/apache/doris/task/LoadEtlTask.java b/fe/src/main/java/org/apache/doris/task/LoadEtlTask.java index 236775cde2d56a..4f37819c4800b1 100644 --- a/fe/src/main/java/org/apache/doris/task/LoadEtlTask.java +++ b/fe/src/main/java/org/apache/doris/task/LoadEtlTask.java @@ -22,6 +22,7 @@ import org.apache.doris.catalog.DistributionInfo; import org.apache.doris.catalog.DistributionInfo.DistributionInfoType; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Tablet; @@ -264,7 +265,7 @@ protected Map getTabletLoadInfos(Map> fileStatusesList, int filesAdded) // Generate tuple descriptor List slotRefs = Lists.newArrayList(); TupleDescriptor tupleDesc = descTable.createTupleDescriptor(); - for (Column col : table.getBaseSchema()) { + for (Column col : table.getFullSchema()) { SlotDescriptor slotDesc = descTable.addSlotDescriptor(tupleDesc); slotDesc.setIsMaterialized(true); slotDesc.setColumn(col); diff --git a/fe/src/main/java/org/apache/doris/task/StreamLoadTask.java b/fe/src/main/java/org/apache/doris/task/StreamLoadTask.java index 57059fec8e4f51..e9e3e88538d3f1 100644 --- a/fe/src/main/java/org/apache/doris/task/StreamLoadTask.java +++ b/fe/src/main/java/org/apache/doris/task/StreamLoadTask.java @@ -34,6 +34,7 @@ import org.apache.doris.thrift.TUniqueId; import com.google.common.base.Joiner; +import com.google.common.collect.Lists; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -51,7 +52,7 @@ public class StreamLoadTask { private TFileFormatType formatType; // optional - private List columnExprDesc; + private List columnExprDescs; private Expr whereExpr; private ColumnSeparator columnSeparator; private String partitions; @@ -82,8 +83,15 @@ public TFileFormatType getFormatType() { return formatType; } - public List getColumnExprDesc() { - return columnExprDesc; + public List getColumnExprDescs() { + return columnExprDescs; + } + + public void addColumnExprDesc(ImportColumnDesc columnExprDesc) { + if (columnExprDescs == null) { + columnExprDescs = Lists.newArrayList(); + } + this.columnExprDescs.add(columnExprDesc); } public Expr getWhereExpr() { @@ -154,7 +162,7 @@ public static StreamLoadTask fromRoutineLoadJob(RoutineLoadJob routineLoadJob) { } private void setOptionalFromRoutineLoadJob(RoutineLoadJob routineLoadJob) { - columnExprDesc = routineLoadJob.getColumnDescs(); + columnExprDescs = routineLoadJob.getColumnDescs(); whereExpr = routineLoadJob.getWhereExpr(); columnSeparator = routineLoadJob.getColumnSeparator(); partitions = routineLoadJob.getPartitions() == null ? null : Joiner.on(",").join(routineLoadJob.getPartitions()); @@ -184,7 +192,7 @@ private void setColumnToColumnExpr(String columns) throws UserException { } if (columnsStmt.getColumns() != null && !columnsStmt.getColumns().isEmpty()) { - columnExprDesc = columnsStmt.getColumns(); + columnExprDescs = columnsStmt.getColumns(); } } diff --git a/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java b/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java index b024ccc1268c11..99f0b2a1da8a29 100644 --- a/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java +++ b/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java @@ -17,11 +17,10 @@ package org.apache.doris.transaction; -import org.apache.doris.alter.RollupJob; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; -import org.apache.doris.catalog.MaterializedIndex.IndexState; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.OlapTable.OlapTableState; import org.apache.doris.catalog.Partition; @@ -326,15 +325,7 @@ public void commitTransaction(long dbId, long transactionId, List allIndices = new ArrayList<>(); - allIndices.addAll(partition.getMaterializedIndices()); - MaterializedIndex rollingUpIndex = null; - RollupJob rollupJob = null; - if (table.getState() == OlapTableState.ROLLUP) { - rollupJob = (RollupJob) catalog.getRollupHandler().getAlterJob(tableId); - rollingUpIndex = rollupJob.getRollupIndex(partition.getId()); - } - + List allIndices = partition.getMaterializedIndices(IndexExtState.ALL); if (table.getState() == OlapTableState.ROLLUP || table.getState() == OlapTableState.SCHEMA_CHANGE) { /* * This is just a optimization that do our best to not let publish version tasks @@ -352,15 +343,7 @@ public void commitTransaction(long dbId, long transactionId, List errorReplicaIds) thr continue; } int quorumReplicaNum = partitionInfo.getReplicationNum(partitionId) / 2 + 1; - MaterializedIndex baseIndex = partition.getBaseIndex(); - MaterializedIndex rollingUpIndex = null; - RollupJob rollupJob = null; - if (table.getState() == OlapTableState.ROLLUP) { - rollupJob = (RollupJob) catalog.getRollupHandler().getAlterJob(tableId); - rollingUpIndex = rollupJob.getRollupIndex(partitionId); - } - List allInices = new ArrayList<>(); - allInices.addAll(partition.getMaterializedIndices()); - if (rollingUpIndex != null) { - allInices.add(rollingUpIndex); - } + + List allInices = partition.getMaterializedIndices(IndexExtState.ALL); for (MaterializedIndex index : allInices) { for (Tablet tablet : index.getTablets()) { int healthReplicaNum = 0; @@ -685,7 +648,7 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) thr // it is healthy in the past and does not have error in current load if (replica.checkVersionCatchUp(partition.getVisibleVersion(), - partition.getVisibleVersionHash())) { + partition.getVisibleVersionHash(), true)) { // during rollup, the rollup replica's last failed version < 0, // it may be treated as a normal replica. // the replica is not failed during commit or publish @@ -723,17 +686,9 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) thr errorReplicaIds.remove(replica.getId()); ++healthReplicaNum; } - if (replica.getLastFailedVersion() > 0) { - // if this error replica is a base replica and it is under rollup - // then remove the rollup task and rollup job will remove the rollup replica automatically - if (index.getId() == baseIndex.getId() && rollupJob != null) { - LOG.info("base replica [{}] has errors during load, remove rollup task on related replica", replica); - rollupJob.removeReplicaRelatedTask(partition.getId(), - tablet.getId(), replica.getId(), replica.getBackendId()); - } - } } - if (index.getState() != IndexState.ROLLUP && healthReplicaNum < quorumReplicaNum) { + + if (healthReplicaNum < quorumReplicaNum) { LOG.info("publish version failed for transaction {} on tablet {}, with only {} replicas less than quorum {}", transactionState, tablet, healthReplicaNum, quorumReplicaNum); hasError = true; @@ -830,6 +785,7 @@ public void removeOldTransactions() { } // 3. use dbIdToTxnIds to remove old transactions, without holding load locks again + List abortedTxns = Lists.newArrayList(); writeLock(); try { List transactionsToDelete = Lists.newArrayList(); @@ -856,6 +812,7 @@ public void removeOldTransactions() { transactionState.setFinishTime(System.currentTimeMillis()); transactionState.setReason("transaction is timeout and is cancelled automatically"); unprotectUpsertTransactionState(transactionState); + abortedTxns.add(transactionState); } } } @@ -868,10 +825,19 @@ public void removeOldTransactions() { } finally { writeUnlock(); } + + for (TransactionState abortedTxn : abortedTxns) { + try { + abortedTxn.afterStateTransform(TransactionStatus.ABORTED, true, abortedTxn.getReason()); + } catch (UserException e) { + // just print a log, it does not matter. + LOG.warn("after abort timeout txn failed. txn id: {}", abortedTxn.getTransactionId(), e); + } + } } private boolean checkTxnHasRelatedJob(TransactionState txnState, Map> dbIdToTxnIds) { - // TODO: put checkTxnHasRelaredJob into Load + // TODO: put checkTxnHasRelatedJob into Load Set txnIds = dbIdToTxnIds.get(txnState.getDbId()); if (txnIds == null) { // We can't find the related load job of this database. @@ -1027,18 +993,7 @@ private void updateCatalogAfterCommitted(TransactionState transactionState, Data for (PartitionCommitInfo partitionCommitInfo : tableCommitInfo.getIdToPartitionCommitInfo().values()) { long partitionId = partitionCommitInfo.getPartitionId(); Partition partition = table.getPartition(partitionId); - List allIndices = new ArrayList<>(); - allIndices.addAll(partition.getMaterializedIndices()); - MaterializedIndex baseIndex = partition.getBaseIndex(); - MaterializedIndex rollingUpIndex = null; - RollupJob rollupJob = null; - if (table.getState() == OlapTableState.ROLLUP) { - rollupJob = (RollupJob) catalog.getRollupHandler().getAlterJob(tableId); - rollingUpIndex = rollupJob.getRollupIndex(partition.getId()); - } - if (rollingUpIndex != null) { - allIndices.add(rollingUpIndex); - } + List allIndices = partition.getMaterializedIndices(IndexExtState.ALL); for (MaterializedIndex index : allIndices) { List tablets = index.getTablets(); for (Tablet tablet : tablets) { @@ -1048,14 +1003,6 @@ private void updateCatalogAfterCommitted(TransactionState transactionState, Data // should get from transaction state replica.updateLastFailedVersion(partitionCommitInfo.getVersion(), partitionCommitInfo.getVersionHash()); - // if this error replica is a base replica and it is under rollup - // then remove the rollup task and rollup job will remove the rollup replica automatically - if (index.getId() == baseIndex.getId() && rollupJob != null) { - LOG.debug("the base replica [{}] has error, remove the related rollup replica from rollupjob [{}]", - replica, rollupJob); - rollupJob.removeReplicaRelatedTask(partition.getId(), - tablet.getId(), replica.getId(), replica.getBackendId()); - } } } } @@ -1083,18 +1030,7 @@ private boolean updateCatalogAfterVisible(TransactionState transactionState, Dat long newCommitVersion = partitionCommitInfo.getVersion(); long newCommitVersionHash = partitionCommitInfo.getVersionHash(); Partition partition = table.getPartition(partitionId); - MaterializedIndex baseIndex = partition.getBaseIndex(); - MaterializedIndex rollingUpIndex = null; - RollupJob rollupJob = null; - if (table.getState() == OlapTableState.ROLLUP) { - rollupJob = (RollupJob) catalog.getRollupHandler().getAlterJob(tableId); - rollingUpIndex = rollupJob.getRollupIndex(partitionId); - } - List allIndices = new ArrayList<>(); - allIndices.addAll(partition.getMaterializedIndices()); - if (rollingUpIndex != null) { - allIndices.add(rollingUpIndex); - } + List allIndices = partition.getMaterializedIndices(IndexExtState.ALL); for (MaterializedIndex index : allIndices) { for (Tablet tablet : index.getTablets()) { for (Replica replica : tablet.getReplicas()) { @@ -1110,7 +1046,7 @@ private boolean updateCatalogAfterVisible(TransactionState transactionState, Dat newVersion = replica.getVersion(); newVersionHash = replica.getVersionHash(); } else if (!replica.checkVersionCatchUp(partition.getVisibleVersion(), - partition.getVisibleVersionHash())) { + partition.getVisibleVersionHash(), true)) { // this means the replica has error in the past, but we did not observe it // during upgrade, one job maybe in quorum finished state, for example, A,B,C 3 replica // A,B 's version is 10, C's version is 10 but C' 10 is abnormal should be rollback @@ -1139,15 +1075,6 @@ private boolean updateCatalogAfterVisible(TransactionState transactionState, Dat } } replica.updateVersionInfo(newVersion, newVersionHash, lastFailedVersion, lastFailedVersionHash, lastSucessVersion, lastSuccessVersionHash); - // if this error replica is a base replica and it is under rollup - // then remove the rollup task and rollup job will remove the rollup replica automatically - if (index.getId() == baseIndex.getId() - && replica.getLastFailedVersion() > 0 - && rollupJob != null) { - LOG.debug("base replica [{}] has errors during load, remove rollup task on related replica", replica); - rollupJob.removeReplicaRelatedTask(partition.getId(), - tablet.getId(), replica.getId(), replica.getBackendId()); - } } } } // end for indices diff --git a/fe/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java b/fe/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java index 129c03948bd531..1e4c40799205b4 100644 --- a/fe/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java +++ b/fe/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java @@ -40,7 +40,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; public class PublishVersionDaemon extends Daemon { @@ -134,7 +133,7 @@ private void publishVersion() throws UserException { continue; } Map transTasks = transactionState.getPublishVersionTasks(); - Set transErrorReplicas = Sets.newHashSet(); + Set publishErrorReplicaIds = Sets.newHashSet(); List unfinishedTasks = Lists.newArrayList(); for (PublishVersionTask publishVersionTask : transTasks.values()) { if (publishVersionTask.isFinished()) { @@ -153,7 +152,7 @@ private void publishVersion() throws UserException { } Replica replica = tabletInvertedIndex.getReplica(tabletId, publishVersionTask.getBackendId()); if (replica != null) { - transErrorReplicas.add(replica); + publishErrorReplicaIds.add(replica.getId()); } else { LOG.info("could not find related replica with tabletid={}, backendid={}", tabletId, publishVersionTask.getBackendId()); @@ -189,7 +188,7 @@ private void publishVersion() throws UserException { Replica replica = tabletInvertedIndex.getReplica(tabletId, unfinishedTask.getBackendId()); if (replica != null) { - transErrorReplicas.add(replica); + publishErrorReplicaIds.add(replica.getId()); } else { LOG.info("could not find related replica with tabletid={}, backendid={}", tabletId, unfinishedTask.getBackendId()); @@ -207,14 +206,13 @@ private void publishVersion() throws UserException { } if (shouldFinishTxn) { - Set allErrorReplicas = transErrorReplicas.stream().map(v -> v.getId()).collect(Collectors.toSet()); - globalTransactionMgr.finishTransaction(transactionState.getTransactionId(), allErrorReplicas); + globalTransactionMgr.finishTransaction(transactionState.getTransactionId(), publishErrorReplicaIds); if (transactionState.getTransactionStatus() != TransactionStatus.VISIBLE) { // if finish transaction state failed, then update publish version time, should check // to finish after some interval transactionState.updateSendTaskTime(); LOG.debug("publish version for transation {} failed, has {} error replicas during publish", - transactionState, transErrorReplicas.size()); + transactionState, publishErrorReplicaIds.size()); } } diff --git a/fe/src/test/java/org/apache/doris/alter/RollupJobTest.java b/fe/src/test/java/org/apache/doris/alter/RollupJobTest.java index 87b49af93f8ac2..59d4c28de454ab 100644 --- a/fe/src/test/java/org/apache/doris/alter/RollupJobTest.java +++ b/fe/src/test/java/org/apache/doris/alter/RollupJobTest.java @@ -30,6 +30,7 @@ import org.apache.doris.catalog.FakeCatalog; import org.apache.doris.catalog.FakeEditLog; 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; @@ -116,7 +117,7 @@ public void testAddRollup() throws Exception { alterClauses.add(clause); Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); - rollupHandler.process(alterClauses, db, olapTable, false); + rollupHandler.process(alterClauses, db.getClusterName(), db, olapTable); RollupJob rollupJob = (RollupJob) rollupHandler.getAlterJob(CatalogTestUtil.testTableId1); Assert.assertEquals(CatalogTestUtil.testIndexId1, rollupJob.getBaseIndexId()); Assert.assertEquals(CatalogTestUtil.testRollupIndex2, rollupJob.getRollupIndexName()); @@ -134,7 +135,7 @@ public void testRollup1() throws Exception { Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); - rollupHandler.process(alterClauses, db, olapTable, false); + rollupHandler.process(alterClauses, db.getClusterName(), db, olapTable); RollupJob rollupJob = (RollupJob) rollupHandler.getAlterJob(CatalogTestUtil.testTableId1); Assert.assertEquals(CatalogTestUtil.testIndexId1, rollupJob.getBaseIndexId()); Assert.assertEquals(CatalogTestUtil.testRollupIndex2, rollupJob.getRollupIndexName()); @@ -226,7 +227,7 @@ public void testRollup2() throws Exception { Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); - rollupHandler.process(alterClauses, db, olapTable, false); + rollupHandler.process(alterClauses, db.getClusterName(), db, olapTable); RollupJob rollupJob = (RollupJob) rollupHandler.getAlterJob(CatalogTestUtil.testTableId1); Assert.assertEquals(CatalogTestUtil.testIndexId1, rollupJob.getBaseIndexId()); Assert.assertEquals(CatalogTestUtil.testRollupIndex2, rollupJob.getRollupIndexName()); @@ -284,7 +285,7 @@ public void testRollup3() throws Exception { Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); - rollupHandler.process(alterClauses, db, olapTable, false); + rollupHandler.process(alterClauses, db.getClusterName(), db, olapTable); RollupJob rollupJob = (RollupJob) rollupHandler.getAlterJob(CatalogTestUtil.testTableId1); Assert.assertEquals(CatalogTestUtil.testIndexId1, rollupJob.getBaseIndexId()); Assert.assertEquals(CatalogTestUtil.testRollupIndex2, rollupJob.getRollupIndexName()); @@ -346,6 +347,6 @@ public void testRollup3() throws Exception { // rollup hander run one cycle again, the rollup job is finishing rollupHandler.runOneCycle(); Assert.assertEquals(JobState.CANCELLED, rollupJob.getState()); - assertEquals(1, testPartition.getMaterializedIndices().size()); + assertEquals(1, testPartition.getMaterializedIndices(IndexExtState.ALL).size()); } } diff --git a/fe/src/test/java/org/apache/doris/backup/BackupHandlerTest.java b/fe/src/test/java/org/apache/doris/backup/BackupHandlerTest.java index db90cba79c5d9b..2a4c22cf0f8f29 100644 --- a/fe/src/test/java/org/apache/doris/backup/BackupHandlerTest.java +++ b/fe/src/test/java/org/apache/doris/backup/BackupHandlerTest.java @@ -29,6 +29,7 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Table; @@ -205,7 +206,7 @@ public Status getSnapshotInfoFile(String label, String backupTimestamp, List snapshotInfos = Maps.newHashMap(); for (Partition part : tbl.getPartitions()) { - for (MaterializedIndex idx : part.getMaterializedIndices()) { + for (MaterializedIndex idx : part.getMaterializedIndices(IndexExtState.VISIBLE)) { for (Tablet tablet : idx.getTablets()) { List files = Lists.newArrayList(); SnapshotInfo sinfo = new SnapshotInfo(db.getId(), tbl.getId(), part.getId(), idx.getId(), diff --git a/fe/src/test/java/org/apache/doris/backup/RestoreJobTest.java b/fe/src/test/java/org/apache/doris/backup/RestoreJobTest.java index 304cf58e0d43eb..07d8c770c6c5cd 100644 --- a/fe/src/test/java/org/apache/doris/backup/RestoreJobTest.java +++ b/fe/src/test/java/org/apache/doris/backup/RestoreJobTest.java @@ -25,6 +25,7 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Table; @@ -214,7 +215,7 @@ boolean await(long timeout, TimeUnit unit) { partInfo.name = partition.getName(); tblInfo.partitions.put(partInfo.name, partInfo); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { BackupIndexInfo idxInfo = new BackupIndexInfo(); idxInfo.id = index.getId(); idxInfo.name = expectedRestoreTbl.getIndexNameById(index.getId()); diff --git a/fe/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java b/fe/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java index d8548a967d6680..648b898971abe4 100644 --- a/fe/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java +++ b/fe/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java @@ -19,6 +19,7 @@ import org.apache.doris.analysis.PartitionKeyDesc; import org.apache.doris.analysis.SingleRangePartitionDesc; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.MaterializedIndex.IndexState; import org.apache.doris.catalog.Replica.ReplicaState; import org.apache.doris.common.DdlException; @@ -122,7 +123,7 @@ public static boolean compareCatalog(Catalog masterCatalog, Catalog slaveCatalog || masterPartition.getCommittedVersionHash() != slavePartition.getCommittedVersionHash()) { return false; } - List allMaterializedIndices = masterPartition.getMaterializedIndices(); + List allMaterializedIndices = masterPartition.getMaterializedIndices(IndexExtState.ALL); for (MaterializedIndex masterIndex : allMaterializedIndices) { MaterializedIndex slaveIndex = slavePartition.getIndex(masterIndex.getId()); if (slaveIndex == null) { diff --git a/fe/src/test/java/org/apache/doris/catalog/ReplicaTest.java b/fe/src/test/java/org/apache/doris/catalog/ReplicaTest.java index 4f0e210d6c00ef..2b396c8a90fcad 100644 --- a/fe/src/test/java/org/apache/doris/catalog/ReplicaTest.java +++ b/fe/src/test/java/org/apache/doris/catalog/ReplicaTest.java @@ -85,9 +85,9 @@ public void getMethodTest() { Assert.assertEquals(newRowCount, replica.getRowCount()); // check version catch up - Assert.assertFalse(replica.checkVersionCatchUp(5, 98765)); - Assert.assertFalse(replica.checkVersionCatchUp(newVersion, 76543)); - Assert.assertTrue(replica.checkVersionCatchUp(newVersion, newVersionHash)); + Assert.assertFalse(replica.checkVersionCatchUp(5, 98765, false)); + Assert.assertFalse(replica.checkVersionCatchUp(newVersion, 76543, false)); + Assert.assertTrue(replica.checkVersionCatchUp(newVersion, newVersionHash, false)); } @Test diff --git a/fe/src/test/java/org/apache/doris/load/LoadCheckerTest.java b/fe/src/test/java/org/apache/doris/load/LoadCheckerTest.java index f69ca373f7c5bb..1b99e0dec09fb7 100644 --- a/fe/src/test/java/org/apache/doris/load/LoadCheckerTest.java +++ b/fe/src/test/java/org/apache/doris/load/LoadCheckerTest.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Replica; @@ -254,7 +255,7 @@ public void testRunLoadingJobs() throws Exception { // set tablet load infos int replicaNum = 0; Map tabletLoadInfos = new HashMap(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { for (Tablet tablet : index.getTablets()) { replicaNum += tablet.getReplicas().size(); TabletLoadInfo tabletLoadInfo = new TabletLoadInfo("/label/path", 1L); @@ -285,7 +286,7 @@ public void testRunLoadingJobs() throws Exception { Assert.assertEquals(0, AgentTaskQueue.getTaskNum()); // update replica to new version - for (MaterializedIndex olapIndex : partition.getMaterializedIndices()) { + for (MaterializedIndex olapIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { for (Tablet tablet : olapIndex.getTablets()) { for (Replica replica : tablet.getReplicas()) { replica.updateVersionInfo(newVersion, newVersionHash, 0L, 0L); @@ -327,7 +328,7 @@ public void testRunQuorumFinishedJobs() throws Exception { job.setIdToTableLoadInfo(idToTableLoadInfo); // set tablet load infos Map tabletLoadInfos = new HashMap(); - for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { for (Tablet tablet : index.getTablets()) { for (Replica replica : tablet.getReplicas()) { replica.updateVersionInfo(newVersion, newVersionHash, 0L, 0L); diff --git a/fe/src/test/java/org/apache/doris/planner/OlapTableSinkTest.java b/fe/src/test/java/org/apache/doris/planner/OlapTableSinkTest.java index e6e3a3b1d0383b..3fb8f82b98f7cf 100644 --- a/fe/src/test/java/org/apache/doris/planner/OlapTableSinkTest.java +++ b/fe/src/test/java/org/apache/doris/planner/OlapTableSinkTest.java @@ -97,7 +97,7 @@ public void testSinglePartition() throws UserException { dstTable.getPartitions(); result = Lists.newArrayList(partition); }}; - OlapTableSink sink = new OlapTableSink(dstTable, tuple); + OlapTableSink sink = new OlapTableSink(dstTable, tuple, ""); sink.init(new TUniqueId(1, 2), 3, 4); sink.finalize(); LOG.info("sink is {}", sink.toThrift()); diff --git a/fe/src/test/java/org/apache/doris/qe/ShowExecutorTest.java b/fe/src/test/java/org/apache/doris/qe/ShowExecutorTest.java index 62ee101b1deceb..f89fae150002d8 100644 --- a/fe/src/test/java/org/apache/doris/qe/ShowExecutorTest.java +++ b/fe/src/test/java/org/apache/doris/qe/ShowExecutorTest.java @@ -95,7 +95,6 @@ public void setUp() throws Exception { // mock partition Partition partition = EasyMock.createMock(Partition.class); - EasyMock.expect(partition.getRollupIndices()).andReturn(Lists.newArrayList(index1, index2)).anyTimes(); EasyMock.expect(partition.getBaseIndex()).andReturn(index1).anyTimes(); EasyMock.replay(partition); diff --git a/fe/src/test/java/org/apache/doris/task/LoadEtlTaskTest.java b/fe/src/test/java/org/apache/doris/task/LoadEtlTaskTest.java index 32cd8fa9f7ac63..392d6ce243825e 100644 --- a/fe/src/test/java/org/apache/doris/task/LoadEtlTaskTest.java +++ b/fe/src/test/java/org/apache/doris/task/LoadEtlTaskTest.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Tablet; @@ -175,7 +176,7 @@ public void testRunEtlTask() throws Exception { .getIdToPartitionLoadInfo().get(paritionId).getVersion()); int tabletNum = 0; Map tabletLoadInfos = job.getIdToTabletLoadInfo(); - for (MaterializedIndex olapTable : partition.getMaterializedIndices()) { + for (MaterializedIndex olapTable : partition.getMaterializedIndices(IndexExtState.ALL)) { for (Tablet tablet : olapTable.getTablets()) { ++tabletNum; Assert.assertTrue(tabletLoadInfos.containsKey(tablet.getId())); diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift index a382fbde4a65b0..240b72c2edc6f0 100644 --- a/gensrc/thrift/AgentService.thrift +++ b/gensrc/thrift/AgentService.thrift @@ -65,7 +65,7 @@ struct TDropTabletReq { 2: optional Types.TSchemaHash schema_hash } -struct TAlterTabletReq{ +struct TAlterTabletReq { 1: required Types.TTabletId base_tablet_id 2: required Types.TSchemaHash base_schema_hash 3: required TCreateTabletReq new_tablet_req diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift index 659f179d79ca36..fbd0c502f55d97 100644 --- a/gensrc/thrift/Types.thrift +++ b/gensrc/thrift/Types.thrift @@ -161,7 +161,8 @@ enum TTaskType { RECOVER_TABLET, STREAM_LOAD, UPDATE_TABLET_META_INFO, - ALTER_TASK + // this type of task will replace both ROLLUP and SCHEMA_CHANGE + ALTER } enum TStmtType {