From a982111ccca73b5109127e5f61d7da4cfe5e1dfd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 2 Jun 2020 16:46:18 +0800 Subject: [PATCH 01/50] Support table level read and write lock --- .../org/apache/doris/alter/AlterJobV2.java | 18 +-- .../doris/alter/MaterializedViewHandler.java | 15 +-- .../org/apache/doris/alter/RollupJob.java | 77 +++++++------ .../org/apache/doris/alter/RollupJobV2.java | 53 ++++----- .../doris/alter/SchemaChangeHandler.java | 40 +++---- .../apache/doris/alter/SchemaChangeJob.java | 51 +++++---- .../apache/doris/alter/SchemaChangeJobV2.java | 62 +++++------ .../apache/doris/analysis/DescribeStmt.java | 15 +-- .../org/apache/doris/analysis/ExportStmt.java | 16 ++- .../apache/doris/analysis/ShowDataStmt.java | 84 ++++++++------ .../doris/analysis/ShowPartitionsStmt.java | 16 +-- .../apache/doris/backup/BackupHandler.java | 41 +++---- .../org/apache/doris/backup/BackupJob.java | 72 ++++++------ .../org/apache/doris/catalog/Catalog.java | 4 +- .../org/apache/doris/catalog/Database.java | 60 ++++++++-- .../java/org/apache/doris/catalog/Table.java | 44 ++++++++ .../doris/service/FrontendServiceImpl.java | 105 +++++++----------- .../org/apache/doris/task/StreamLoadTask.java | 8 +- .../doris/planner/StreamLoadPlannerTest.java | 2 +- .../doris/planner/StreamLoadScanNodeTest.java | 11 +- 20 files changed, 410 insertions(+), 384 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java index f7578caa3f145b..cb8f590b9750a7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java @@ -187,22 +187,12 @@ public final boolean cancel(String errMsg) { * return false if table is not stable. */ protected boolean checkTableStable(Database db) throws AlterCancelException { - OlapTable tbl = null; - boolean isStable = false; - db.readLock(); - try { - tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); - } + OlapTable tbl = (OlapTable) db.getTable(tableId); - isStable = tbl.isStable(Catalog.getCurrentSystemInfo(), + boolean isStable = tbl.isStable(Catalog.getCurrentSystemInfo(), Catalog.getCurrentCatalog().getTabletScheduler(), db.getClusterName()); - } finally { - db.readUnlock(); - } - db.writeLock(); + tbl.writeLock(); try { if (!isStable) { errMsg = "table is unstable"; @@ -216,7 +206,7 @@ protected boolean checkTableStable(Database db) throws AlterCancelException { return true; } } finally { - db.writeUnlock(); + tbl.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java index 917e8006b005c0..2eff0999091790 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java @@ -1128,18 +1128,13 @@ private void getOldAlterJobInfos(Database db, List> rollupJobIn unlock(); } - db.readLock(); - try { - for (AlterJob selectedJob : jobs) { - OlapTable olapTable = (OlapTable) db.getTable(selectedJob.getTableId()); - if (olapTable == null) { - continue; - } - selectedJob.getJobInfo(rollupJobInfos, olapTable); + for (AlterJob selectedJob : jobs) { + OlapTable olapTable = (OlapTable) db.getTable(selectedJob.getTableId()); + if (olapTable == null) { + continue; } - } finally { - db.readUnlock(); + selectedJob.getJobInfo(rollupJobInfos, olapTable); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java index 15d07f76fc3ab1..b5f352ed88b8d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java @@ -348,15 +348,16 @@ public int checkOrResendClearTasks() { } batchClearAlterTask = new AgentBatchTask(); - db.readLock(); - try { - synchronized (this) { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - cancelMsg = "table[" + tableId + "] does not exist"; - LOG.warn(cancelMsg); - return -1; - } + + synchronized (this) { + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + cancelMsg = "table[" + tableId + "] does not exist"; + LOG.warn(cancelMsg); + return -1; + } + olapTable.readLock(); + try { boolean allAddSuccess = true; LOG.info("sending clear rollup job tasks for table [{}]", tableId); for (Partition partition : olapTable.getPartitions()) { @@ -392,9 +393,10 @@ public int checkOrResendClearTasks() { } batchClearAlterTask = null; } + + } finally { + olapTable.readUnlock(); } - } finally { - db.readUnlock(); } LOG.info("successfully sending clear rollup job[{}]", tableId); return 0; @@ -413,29 +415,29 @@ public boolean sendTasks() { return false; } - db.readLock(); - try { - synchronized (this) { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - cancelMsg = "table[" + tableId + "] does not exist"; - LOG.warn(cancelMsg); - return false; - } - LOG.info("sending rollup job[{}] tasks.", tableId); - // in palo 3.2, the rollup keys type is not serialized, when a fe follower change to fe master - // the rollup keys type == null, so that send tasks will report error - if (rollupKeysType == null) { - rollupKeysType = olapTable.getKeysType().toThrift(); - } + synchronized (this) { + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + cancelMsg = "table[" + tableId + "] does not exist"; + LOG.warn(cancelMsg); + return false; + } + + LOG.info("sending rollup job[{}] tasks.", tableId); + // in palo 3.2, the rollup keys type is not serialized, when a fe follower change to fe master + // the rollup keys type == null, so that send tasks will report error + if (rollupKeysType == null) { + rollupKeysType = olapTable.getKeysType().toThrift(); + } + olapTable.readLock(); + try { for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { long partitionId = entry.getKey(); Partition partition = olapTable.getPartition(partitionId); if (partition == null) { continue; } - MaterializedIndex rollupIndex = entry.getValue(); Map tabletIdMap = this.partitionIdToBaseRollupTabletIdMap.get(partitionId); @@ -448,14 +450,14 @@ public boolean sendTasks() { Preconditions.checkNotNull(tabletIdMap.get(rollupTabletId)); // baseTabletId CreateRollupTask createRollupTask = new CreateRollupTask(resourceInfo, backendId, dbId, tableId, - partitionId, rollupIndexId, baseIndexId, - rollupTabletId, tabletIdMap.get(rollupTabletId), - rollupReplicaId, - rollupShortKeyColumnCount, - rollupSchemaHash, baseSchemaHash, - rollupStorageType, rollupSchema, - olapTable.getCopiedBfColumns(), olapTable.getBfFpp(), - rollupKeysType); + partitionId, rollupIndexId, baseIndexId, + rollupTabletId, tabletIdMap.get(rollupTabletId), + rollupReplicaId, + rollupShortKeyColumnCount, + rollupSchemaHash, baseSchemaHash, + rollupStorageType, rollupSchema, + olapTable.getCopiedBfColumns(), olapTable.getBfFpp(), + rollupKeysType); AgentTaskQueue.addTask(createRollupTask); addReplicaId(partitionId, rollupReplicaId, backendId); @@ -464,9 +466,10 @@ public boolean sendTasks() { } this.state = JobState.RUNNING; + } finally { + olapTable.readUnlock(); } - } finally { - db.readUnlock(); + } Preconditions.checkState(this.state == JobState.RUNNING); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java index 81c25a2e214b66..19230a5750f6e5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java @@ -198,13 +198,13 @@ protected void runPendingJob() throws AlterCancelException { } } MarkedCountDownLatch countDownLatch = new MarkedCountDownLatch(totalReplicaNum); - db.readLock(); - try { - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); - } + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + throw new AlterCancelException("Table " + tableId + " does not exist"); + } + tbl.readLock(); + try { Preconditions.checkState(tbl.getState() == OlapTableState.ROLLUP); for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { long partitionId = entry.getKey(); @@ -244,7 +244,7 @@ protected void runPendingJob() throws AlterCancelException { } // end for rollupTablets } } finally { - db.readUnlock(); + tbl.readUnlock(); } if (!FeConstants.runningUnitTest) { @@ -281,16 +281,12 @@ protected void runPendingJob() throws AlterCancelException { // create all rollup replicas success. // add rollup index to catalog - db.writeLock(); + tbl.writeLock(); try { - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); - } Preconditions.checkState(tbl.getState() == OlapTableState.ROLLUP); addRollupIndexToCatalog(tbl); } finally { - db.writeUnlock(); + tbl.writeUnlock(); } this.watershedTxnId = Catalog.getCurrentGlobalTransactionMgr().getTransactionIDGenerator().getNextTransactionId(); @@ -340,12 +336,12 @@ protected void runWaitingTxnJob() throws AlterCancelException { throw new AlterCancelException("Databasee " + dbId + " does not exist"); } - db.readLock(); + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + throw new AlterCancelException("Table " + tableId + " does not exist"); + } + tbl.readLock(); try { - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); - } Preconditions.checkState(tbl.getState() == OlapTableState.ROLLUP); for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { long partitionId = entry.getKey(); @@ -382,7 +378,7 @@ protected void runWaitingTxnJob() throws AlterCancelException { } } } finally { - db.readUnlock(); + tbl.readUnlock(); } AgentTaskQueue.addBatchTask(rollupBatchTask); @@ -412,14 +408,9 @@ protected void runRunningJob() throws AlterCancelException { throw new AlterCancelException("Databasee " + dbId + " does not exist"); } - db.readLock(); - try { - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); - } - } finally { - db.readUnlock(); + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + throw new AlterCancelException("Table " + tableId + " does not exist"); } if (!rollupBatchTask.isFinished()) { @@ -437,12 +428,8 @@ protected void runRunningJob() throws AlterCancelException { * all tasks are finished. check the integrity. * we just check whether all rollup replicas are healthy. */ - db.writeLock(); + tbl.writeLock(); try { - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); - } Preconditions.checkState(tbl.getState() == OlapTableState.ROLLUP); for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { long partitionId = entry.getKey(); @@ -476,7 +463,7 @@ protected void runRunningJob() throws AlterCancelException { onFinished(tbl); } finally { - db.writeUnlock(); + tbl.writeUnlock(); } this.jobState = JobState.FINISHED; diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index 1801007410850b..2102ca866a2b6c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -1564,17 +1564,12 @@ private void getOldAlterJobInfos(Database db, List> schemaChang unlock(); } - db.readLock(); - try { - for (AlterJob selectedJob : selectedJobs) { - OlapTable olapTable = (OlapTable) db.getTable(selectedJob.getTableId()); - if (olapTable == null) { - continue; - } - selectedJob.getJobInfo(schemaChangeJobInfos, olapTable); + for (AlterJob selectedJob : selectedJobs) { + OlapTable olapTable = (OlapTable) db.getTable(selectedJob.getTableId()); + if (olapTable == null) { + continue; } - } finally { - db.readUnlock(); + selectedJob.getJobInfo(schemaChangeJobInfos, olapTable); } } @@ -1708,7 +1703,7 @@ public void processExternalTable(List alterClauses, Database db, Ta private void sendClearAlterTask(Database db, OlapTable olapTable) { AgentBatchTask batchTask = new AgentBatchTask(); - db.readLock(); + olapTable.readLock(); try { for (Partition partition : olapTable.getPartitions()) { for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { @@ -1723,7 +1718,7 @@ private void sendClearAlterTask(Database db, OlapTable olapTable) { } } } finally { - db.readUnlock(); + olapTable.readUnlock(); } AgentTaskExecutor.submit(batchTask); @@ -1733,15 +1728,14 @@ private void sendClearAlterTask(Database db, OlapTable olapTable) { /** * Update all partitions' in-memory property of table */ - public void updateTableInMemoryMeta(Database db, String tableName, Map properties) throws DdlException { + public void updateTableInMemoryMeta(Database db, String tableName, Map properties) throws UserException { List partitions = Lists.newArrayList(); - OlapTable olapTable; - db.readLock(); + OlapTable olapTable = (OlapTable)db.getTableOrThrowException(tableName, Table.TableType.OLAP); + olapTable.readLock(); try { - olapTable = (OlapTable)db.getTable(tableName); partitions.addAll(olapTable.getPartitions()); } finally { - db.readUnlock(); + olapTable.readUnlock(); } boolean isInMemory = Boolean.parseBoolean(properties.get(PropertyAnalyzer.PROPERTIES_INMEMORY)); @@ -1753,11 +1747,11 @@ public void updateTableInMemoryMeta(Database db, String tableName, Map Map>> beIdToTabletIdWithHash = Maps.newHashMap(); - db.readLock(); + OlapTable olapTable = (OlapTable)db.getTableOrThrowException(tableName, Table.TableType.OLAP); + olapTable.readLock(); try { - OlapTable olapTable = (OlapTable)db.getTable(tableName); Partition partition = olapTable.getPartition(partitionName); if (partition == null) { throw new DdlException( @@ -1829,7 +1823,7 @@ public void updatePartitionInMemoryMeta(Database db, } } } finally { - db.readUnlock(); + olapTable.readUnlock(); } int totalTaskNum = beIdToTabletIdWithHash.keySet().size(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java index df8b982d96a688..ef4d151d1049c0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java @@ -224,13 +224,13 @@ public void deleteAllTableHistorySchema() { return; } - db.readLock(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + LOG.warn("table[{}] does not exist in db[{}]", tableId, dbId); + return; + } + olapTable.readLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - LOG.warn("table[{}] does not exist in db[{}]", tableId, dbId); - return; - } // drop all replicas with old schemaHash for (Partition partition : olapTable.getPartitions()) { long partitionId = partition.getId(); @@ -252,9 +252,8 @@ public void deleteAllTableHistorySchema() { } } // end for partitions } finally { - db.readUnlock(); + olapTable.readUnlock(); } - return; } @Override @@ -321,15 +320,15 @@ public int checkOrResendClearTasks() { } batchClearAlterTask = new AgentBatchTask(); - db.readLock(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + cancelMsg = "could not find table[" + tableId + "] in db [" + dbId + "]"; + LOG.warn(cancelMsg); + return -1; + } + + olapTable.readLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - cancelMsg = "could not find table[" + tableId + "] in db [" + dbId + "]"; - LOG.warn(cancelMsg); - return -1; - } - boolean allAddSuccess = true; LOG.info("sending clear schema change job tasks for table [{}]", tableId); OUTER_LOOP: @@ -360,7 +359,7 @@ public int checkOrResendClearTasks() { batchClearAlterTask = null; } } finally { - db.readUnlock(); + olapTable.readUnlock(); } LOG.info("successfully sending clear schema change job [{}]", tableId); @@ -383,16 +382,16 @@ public boolean sendTasks() { return false; } - db.readLock(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + cancelMsg = "table[" + tableId + "] does not exist"; + LOG.warn(cancelMsg); + return false; + } + + olapTable.readLock(); try { synchronized (this) { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - cancelMsg = "table[" + tableId + "] does not exist"; - LOG.warn(cancelMsg); - return false; - } - Preconditions.checkNotNull(this.unfinishedReplicaIds); List tasks = new LinkedList(); @@ -489,7 +488,7 @@ public boolean sendTasks() { this.state = JobState.RUNNING; } // end synchronized block } finally { - db.readUnlock(); + olapTable.readUnlock(); } Preconditions.checkState(this.state == JobState.RUNNING); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java index e7a401570c7251..8bcd16497a2840 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java @@ -215,12 +215,14 @@ protected void runPendingJob() throws AlterCancelException { } } MarkedCountDownLatch countDownLatch = new MarkedCountDownLatch<>(totalReplicaNum); - db.readLock(); + + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + throw new AlterCancelException("Table " + tableId + " does not exist"); + } + + tbl.readLock(); try { - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); - } Preconditions.checkState(tbl.getState() == OlapTableState.SCHEMA_CHANGE); for (long partitionId : partitionIndexMap.rowKeySet()) { @@ -267,7 +269,7 @@ protected void runPendingJob() throws AlterCancelException { } } } finally { - db.readUnlock(); + tbl.readUnlock(); } if (!FeConstants.runningUnitTest) { @@ -304,16 +306,12 @@ protected void runPendingJob() throws AlterCancelException { // create all replicas success. // add all shadow indexes to catalog - db.writeLock(); + tbl.writeLock(); try { - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); - } Preconditions.checkState(tbl.getState() == OlapTableState.SCHEMA_CHANGE); addShadowIndexToCatalog(tbl); } finally { - db.writeUnlock(); + tbl.writeUnlock(); } this.watershedTxnId = Catalog.getCurrentGlobalTransactionMgr().getTransactionIDGenerator().getNextTransactionId(); @@ -372,13 +370,14 @@ protected void runWaitingTxnJob() throws AlterCancelException { if (db == null) { throw new AlterCancelException("Databasee " + dbId + " does not exist"); } - - db.readLock(); + + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + throw new AlterCancelException("Table " + tableId + " does not exist"); + } + + tbl.readLock(); try { - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); - } Preconditions.checkState(tbl.getState() == OlapTableState.SCHEMA_CHANGE); for (long partitionId : partitionIndexMap.rowKeySet()) { @@ -415,7 +414,7 @@ protected void runWaitingTxnJob() throws AlterCancelException { } } // end for partitions } finally { - db.readUnlock(); + tbl.readUnlock(); } AgentTaskQueue.addBatchTask(schemaChangeBatchTask); @@ -446,16 +445,12 @@ protected void runRunningJob() throws AlterCancelException { throw new AlterCancelException("Database " + dbId + " does not exist"); } - db.readLock(); - try { - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); - } - } finally { - db.readUnlock(); + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + throw new AlterCancelException("Table " + tableId + " does not exist"); } + if (!schemaChangeBatchTask.isFinished()) { LOG.info("schema change tasks not finished. job: {}", jobId); List tasks = schemaChangeBatchTask.getUnfinishedTasks(2000); @@ -467,16 +462,17 @@ protected void runRunningJob() throws AlterCancelException { return; } + /* * all tasks are finished. check the integrity. * we just check whether all new replicas are healthy. */ - db.writeLock(); + tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + throw new AlterCancelException("Table " + tableId + " does not exist"); + } + tbl.writeLock(); try { - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); - } Preconditions.checkState(tbl.getState() == OlapTableState.SCHEMA_CHANGE); for (long partitionId : partitionIndexMap.rowKeySet()) { @@ -514,7 +510,7 @@ protected void runRunningJob() throws AlterCancelException { // all partitions are good onFinished(tbl); } finally { - db.writeUnlock(); + tbl.writeUnlock(); } pruneMeta(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DescribeStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DescribeStmt.java index 2fd4865e72e56a..e4ab7318a216dd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DescribeStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DescribeStmt.java @@ -113,13 +113,14 @@ public void analyze(Analyzer analyzer) throws AnalysisException, UserException { if (db == null) { ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbTableName.getDb()); } - db.readLock(); - try { - Table table = db.getTable(dbTableName.getTbl()); - if (table == null) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, dbTableName.getTbl()); - } + Table table = db.getTable(dbTableName.getTbl()); + if (table == null) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, dbTableName.getTbl()); + } + + table.readLock(); + try { if (!isAllTables) { // show base table schema only String procString = "/dbs/" + db.getId() + "/" + table.getId() + "/" + TableProcDir.INDEX_SCHEMA @@ -220,7 +221,7 @@ public void analyze(Analyzer analyzer) throws AnalysisException, UserException { } } } finally { - db.readUnlock(); + table.readUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java index dafa22d5a93df9..9af531996c97bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java @@ -177,13 +177,13 @@ private void checkTable(Catalog catalog) throws AnalysisException { throw new AnalysisException("Db does not exist. name: " + tblName.getDb()); } - db.readLock(); - try { - Table table = db.getTable(tblName.getTbl()); - if (table == null) { - throw new AnalysisException("Table[" + tblName.getTbl() + "] does not exist"); - } + Table table = db.getTable(tblName.getTbl()); + if (table == null) { + throw new AnalysisException("Table[" + tblName.getTbl() + "] does not exist"); + } + table.readLock(); + try { if (partitions == null) { return; } @@ -212,10 +212,8 @@ private void checkTable(Catalog catalog) throws AnalysisException { } } } finally { - db.readUnlock(); + table.readUnlock(); } - - return; } private static void checkPath(String path) throws AnalysisException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java index fd34bf4d25ffb8..160628c2836dcb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java @@ -93,9 +93,10 @@ public void analyze(Analyzer analyzer) throws AnalysisException, UserException { if (db == null) { ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName); } - db.readLock(); - try { - if (tableName == null) { + + if (tableName == null) { + db.readLock(); + try { long totalSize = 0; long totalReplicaCount = 0; @@ -110,8 +111,8 @@ public int compare(Table t1, Table t2) { for (Table table : tables) { if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), dbName, - table.getName(), - PrivPredicate.SHOW)) { + table.getName(), + PrivPredicate.SHOW)) { continue; } sortedTables.add(table); @@ -123,9 +124,15 @@ public int compare(Table t1, Table t2) { } OlapTable olapTable = (OlapTable) table; - long tableSize = olapTable.getDataSize(); - long replicaCount = olapTable.getReplicaCount(); - + long tableSize = 0; + long replicaCount = 0; + olapTable.readLock(); + try { + tableSize = olapTable.getDataSize(); + replicaCount = olapTable.getReplicaCount(); + } finally { + olapTable.readUnlock(); + } Pair tableSizePair = DebugUtil.getByteUint(tableSize); String readableSize = DebugUtil.DECIMAL_FORMAT_SCALE_3.format(tableSizePair.first) + " " + tableSizePair.second; @@ -161,30 +168,35 @@ public int compare(Table t1, Table t2) { + leftPair.second; List leftRow = Arrays.asList("Left", readableLeft, String.valueOf(replicaCountLeft)); totalRows.add(leftRow); - } else { - if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), dbName, - tableName, - PrivPredicate.SHOW)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "SHOW DATA", - ConnectContext.get().getQualifiedUser(), - ConnectContext.get().getRemoteIP(), - tableName); - } + } finally { + db.readUnlock(); + } + } else { + if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), dbName, + tableName, + PrivPredicate.SHOW)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "SHOW DATA", + ConnectContext.get().getQualifiedUser(), + ConnectContext.get().getRemoteIP(), + tableName); + } - Table table = db.getTable(tableName); - if (table == null) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } + Table table = db.getTable(tableName); + if (table == null) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } - if (table.getType() != TableType.OLAP) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_NOT_OLAP_TABLE, tableName); - } + if (table.getType() != TableType.OLAP) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NOT_OLAP_TABLE, tableName); + } - OlapTable olapTable = (OlapTable) table; - int i = 0; - long totalSize = 0; - long totalReplicaCount = 0; + OlapTable olapTable = (OlapTable) table; + int i = 0; + long totalSize = 0; + long totalReplicaCount = 0; + olapTable.readLock(); + try { // sort by index name Map indexNames = olapTable.getIndexNameToId(); Map sortedIndexNames = new TreeMap(); @@ -210,14 +222,14 @@ public int compare(Table t1, Table t2) { List row = null; if (i == 0) { row = Arrays.asList(tableName, - olapTable.getIndexNameById(indexId), - readableSize, String.valueOf(indexReplicaCount), - String.valueOf(indexRowCount)); + olapTable.getIndexNameById(indexId), + readableSize, String.valueOf(indexReplicaCount), + String.valueOf(indexRowCount)); } else { row = Arrays.asList("", - olapTable.getIndexNameById(indexId), - readableSize, String.valueOf(indexReplicaCount), - String.valueOf(indexRowCount)); + olapTable.getIndexNameById(indexId), + readableSize, String.valueOf(indexReplicaCount), + String.valueOf(indexRowCount)); } totalSize += indexSize; @@ -232,9 +244,9 @@ public int compare(Table t1, Table t2) { + totalSizePair.second; List row = Arrays.asList("", "Total", readableSize, String.valueOf(totalReplicaCount), ""); totalRows.add(row); + } finally { + olapTable.readUnlock(); } - } finally { - db.readUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java index c0cca51359177d..954fe5c5e277af 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java @@ -20,7 +20,6 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Type; @@ -114,14 +113,9 @@ public void analyze(Analyzer analyzer) throws UserException { if (db == null) { ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName); } - - db.readLock(); + Table table = db.getTableOrThrowException(tableName, Table.TableType.OLAP); + table.readLock(); try { - Table table = db.getTable(tableName); - if (!(table instanceof OlapTable)) { - throw new AnalysisException("Table[" + tableName + "] does not exists or is not OLAP table"); - } - // build proc path StringBuilder stringBuilder = new StringBuilder(); stringBuilder.append("/dbs/"); @@ -133,11 +127,13 @@ public void analyze(Analyzer analyzer) throws UserException { stringBuilder.append("/partitions"); } - LOG.debug("process SHOW PROC '{}';", stringBuilder.toString()); + if (LOG.isDebugEnabled()) { + LOG.debug("process SHOW PROC '{}';", stringBuilder.toString()); + } node = ProcService.getInstance().open(stringBuilder.toString()); } finally { - db.readUnlock(); + table.readUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java index 058dc49fb5f98a..445dcc77d78b39 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java @@ -274,25 +274,25 @@ private void backup(Repository repository, Database db, BackupStmt stmt) throws // This is just a pre-check to avoid most of invalid backup requests. // Also calculate the signature for incremental backup check. List tblRefs = stmt.getTableRefs(); - BackupMeta curBackupMeta = null; - db.readLock(); - try { - List backupTbls = Lists.newArrayList(); - for (TableRef tblRef : tblRefs) { - String tblName = tblRef.getName().getTbl(); - Table tbl = db.getTable(tblName); - if (tbl == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tblName); - } - if (tbl.getType() != TableType.OLAP) { - ErrorReport.reportDdlException(ErrorCode.ERR_NOT_OLAP_TABLE, tblName); - } - OlapTable olapTbl = (OlapTable) tbl; + List
backupTbls = Lists.newArrayList(); + for (TableRef tblRef : tblRefs) { + String tblName = tblRef.getName().getTbl(); + Table tbl = db.getTable(tblName); + if (tbl == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tblName); + } + if (tbl.getType() != TableType.OLAP) { + ErrorReport.reportDdlException(ErrorCode.ERR_NOT_OLAP_TABLE, tblName); + } + + OlapTable olapTbl = (OlapTable) tbl; + tbl.readLock(); + try { if (olapTbl.existTempPartitions()) { ErrorReport.reportDdlException(ErrorCode.ERR_COMMON_ERROR, "Do not support backup table with temp partitions"); } - + PartitionNames partitionNames = tblRef.getPartitionNames(); if (partitionNames != null) { if (partitionNames.isTemp()) { @@ -303,7 +303,7 @@ private void backup(Repository repository, Database db, BackupStmt stmt) throws Partition partition = olapTbl.getPartition(partName); if (partition == null) { ErrorReport.reportDdlException(ErrorCode.ERR_COMMON_ERROR, - "Unknown partition " + partName + " in table" + tblName); + "Unknown partition " + partName + " in table" + tblName); } } } @@ -313,15 +313,16 @@ private void backup(Repository repository, Database db, BackupStmt stmt) throws OlapTable copiedTbl = olapTbl.selectiveCopy(reservedPartitions, true, IndexExtState.VISIBLE); if (copiedTbl == null) { ErrorReport.reportDdlException(ErrorCode.ERR_COMMON_ERROR, - "Failed to copy table " + tblName + " with selected partitions"); + "Failed to copy table " + tblName + " with selected partitions"); } backupTbls.add(copiedTbl); + } finally { + tbl.readUnlock(); } - curBackupMeta = new BackupMeta(backupTbls); - } finally { - db.readUnlock(); } + BackupMeta curBackupMeta = new BackupMeta(backupTbls); + // Check if label already be used List existSnapshotNames = Lists.newArrayList(); Status st = repository.listSnapshots(existSnapshotNames); diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java index 5a24cbf9fdabec..914ba56d8074fc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java @@ -342,22 +342,29 @@ private void prepareAndSendSnapshotTask() { // generate job id jobId = catalog.getNextId(); AgentBatchTask batchTask = new AgentBatchTask(); - db.readLock(); - try { - // check all backup tables again - for (TableRef tableRef : tableRefs) { - String tblName = tableRef.getName().getTbl(); - Table tbl = db.getTable(tblName); - if (tbl == null) { - status = new Status(ErrCode.NOT_FOUND, "table " + tblName + " does not exist"); - return; - } - if (tbl.getType() != TableType.OLAP) { - status = new Status(ErrCode.COMMON_ERROR, "table " + tblName + " is not OLAP table"); - return; - } - OlapTable olapTbl = (OlapTable) tbl; + unfinishedTaskIds.clear(); + taskProgress.clear(); + taskErrMsg.clear(); + + List
copiedTables = Lists.newArrayList(); + + for (TableRef tableRef : tableRefs) { + String tblName = tableRef.getName().getTbl(); + Table tbl = db.getTable(tblName); + if (tbl == null) { + status = new Status(ErrCode.NOT_FOUND, "table " + tblName + " does not exist"); + return; + } + if (tbl.getType() != TableType.OLAP) { + status = new Status(ErrCode.COMMON_ERROR, "table " + tblName + " is not OLAP table"); + return; + } + + OlapTable olapTbl = (OlapTable) tbl; + olapTbl.readLock(); + try { + // check backup table again if (tableRef.getPartitionNames() != null) { for (String partName : tableRef.getPartitionNames().getPartitionNames()) { Partition partition = olapTbl.getPartition(partName); @@ -368,20 +375,13 @@ private void prepareAndSendSnapshotTask() { } } } - } - unfinishedTaskIds.clear(); - taskProgress.clear(); - taskErrMsg.clear(); - // create snapshot tasks - for (TableRef tblRef : tableRefs) { - String tblName = tblRef.getName().getTbl(); - OlapTable tbl = (OlapTable) db.getTable(tblName); + // create snapshot tasks List partitions = Lists.newArrayList(); - if (tblRef.getPartitionNames() == null) { - partitions.addAll(tbl.getPartitions()); + if (tableRef.getPartitionNames() == null) { + partitions.addAll(olapTbl.getPartitions()); } else { - for (String partName : tblRef.getPartitionNames().getPartitionNames()) { + for (String partName : tableRef.getPartitionNames().getPartitionNames()) { Partition partition = tbl.getPartition(partName); partitions.add(partition); } @@ -393,7 +393,7 @@ private void prepareAndSendSnapshotTask() { long visibleVersionHash = partition.getVisibleVersionHash(); List indexes = partition.getMaterializedIndices(IndexExtState.VISIBLE); for (MaterializedIndex index : indexes) { - int schemaHash = tbl.getSchemaHashByIndexId(index.getId()); + int schemaHash = olapTbl.getSchemaHashByIndexId(index.getId()); List tablets = index.getTablets(); for (Tablet tablet : tablets) { Replica replica = chooseReplica(tablet, visibleVersion, visibleVersionHash); @@ -415,30 +415,26 @@ private void prepareAndSendSnapshotTask() { } LOG.info("snapshot for partition {}, version: {}, version hash: {}", - partition.getId(), visibleVersion, visibleVersionHash); + partition.getId(), visibleVersion, visibleVersionHash); } - } - // copy all related schema at this moment - List
copiedTables = Lists.newArrayList(); - for (TableRef tableRef : tableRefs) { - String tblName = tableRef.getName().getTbl(); - OlapTable tbl = (OlapTable) db.getTable(tblName); // only copy visible indexes List reservedPartitions = tableRef.getPartitionNames() == null ? null : tableRef.getPartitionNames().getPartitionNames(); - OlapTable copiedTbl = tbl.selectiveCopy(reservedPartitions, true, IndexExtState.VISIBLE); + OlapTable copiedTbl = olapTbl.selectiveCopy(reservedPartitions, true, IndexExtState.VISIBLE); if (copiedTbl == null) { status = new Status(ErrCode.COMMON_ERROR, "failed to copy table: " + tblName); return; } copiedTables.add(copiedTbl); + } finally { + olapTbl.readUnlock(); } - backupMeta = new BackupMeta(copiedTables); - } finally { - db.readUnlock(); + } + backupMeta = new BackupMeta(copiedTables); + // send tasks for (AgentTask task : batchTask.getAllTasks()) { AgentTaskQueue.addTask(task); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index 885c86fa04f70e..4860d689b5df2a 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -5436,9 +5436,9 @@ public void modifyTableDefaultReplicationNum(Database db, OlapTable table, Map properties) { - Preconditions.checkArgument(db.isWriteLockHeldByCurrentThread()); + Preconditions.checkArgument(table.isWriteLockHeldByCurrentThread()); TableProperty tableProperty = table.getTableProperty(); if (tableProperty == null) { tableProperty = new TableProperty(properties); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 4d2f9633aefbe4..6c2ded979e2b19 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -23,6 +23,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.FeConstants; import org.apache.doris.common.FeMetaVersion; +import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.io.Text; @@ -44,13 +45,11 @@ import java.io.IOException; import java.io.UnsupportedEncodingException; import java.util.ArrayList; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -110,8 +109,8 @@ public Database(long id, String name) { this.fullQualifiedName = ""; } this.rwLock = new ReentrantReadWriteLock(true); - this.idToTable = new ConcurrentHashMap<>(); - this.nameToTable = new HashMap<>(); + this.idToTable = Maps.newConcurrentMap(); + this.nameToTable = Maps.newConcurrentMap(); this.dataQuotaBytes = Config.default_db_data_quota_bytes; this.replicaQuotaSize = FeConstants.default_db_replica_quota_size; this.dbState = DbState.NORMAL; @@ -140,6 +139,10 @@ public void writeLock() { this.rwLock.writeLock().lock(); } + public void writeUnlock() { + this.rwLock.writeLock().unlock(); + } + public boolean tryWriteLock(long timeout, TimeUnit unit) { try { return this.rwLock.writeLock().tryLock(timeout, unit); @@ -149,10 +152,6 @@ public boolean tryWriteLock(long timeout, TimeUnit unit) { } } - public void writeUnlock() { - this.rwLock.writeLock().unlock(); - } - public boolean isWriteLockHeldByCurrentThread() { return this.rwLock.writeLock().isHeldByCurrentThread(); } @@ -375,11 +374,30 @@ public Set getTableNamesWithLock() { } } + /** + * This is a thread-safe method when nameToTable is a concurrent hash map + * @param tableName + * @return + */ public Table getTable(String tableName) { - if (nameToTable.containsKey(tableName)) { - return nameToTable.get(tableName); + return nameToTable.get(tableName); + } + + /** + * This is a thread-safe method when nameToTable is a concurrent hash map + * @param tableName + * @param tableType + * @return + */ + public Table getTableOrThrowException(String tableName, TableType tableType) throws UserException { + Table table = nameToTable.get(tableName); + if(table == null) { + throw new MetaNotFoundException("unknown table, table=" + tableName); } - return null; + if (table.getType() != tableType) { + throw new UserException("table type is not " + tableType + ", type=" + table.getClass()); + } + return table; } /** @@ -391,6 +409,24 @@ public Table getTable(long tableId) { return idToTable.get(tableId); } + + /** + * This is a thread-safe method when idToTable is a concurrent hash map + * @param tableId + * @param tableType + * @return + */ + public Table getTableOrThrowException(long tableId, TableType tableType) throws UserException { + Table table = idToTable.get(tableId); + if(table == null) { + throw new MetaNotFoundException("unknown table, tableId=" + tableId); + } + if (table.getType() != tableType) { + throw new UserException("table type is not " + tableType + ", type=" + table.getClass()); + } + return table; + } + public int getMaxReplicationNum() { int ret = 0; readLock(); @@ -464,6 +500,7 @@ public void write(DataOutput out) throws IOException { out.writeLong(replicaQuotaSize); } + @Override public void readFields(DataInput in) throws IOException { super.readFields(in); @@ -512,6 +549,7 @@ public void readFields(DataInput in) throws IOException { } } + @Override public boolean equals(Object obj) { if (this == obj) { return true; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java index ae8f0b32df9123..d3ce8bb873758a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java @@ -37,6 +37,8 @@ import java.io.DataOutput; import java.io.IOException; import java.time.Instant; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -63,6 +65,8 @@ public enum TableType { protected String name; protected TableType type; protected long createTime; + protected ReentrantReadWriteLock rwLock; + /* * 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 @@ -101,6 +105,7 @@ public Table(TableType type) { this.type = type; this.fullSchema = Lists.newArrayList(); this.nameToColumn = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); + this.rwLock = new ReentrantReadWriteLock(true); } public Table(long id, String tableName, TableType type, List fullSchema) { @@ -120,9 +125,48 @@ public Table(long id, String tableName, TableType type, List fullSchema) // Only view in with-clause have null base Preconditions.checkArgument(type == TableType.VIEW, "Table has no columns"); } + this.rwLock = new ReentrantReadWriteLock(); this.createTime = Instant.now().getEpochSecond(); } + public void readLock() { + this.rwLock.readLock().lock(); + } + + public boolean tryReadLock(long timeout, TimeUnit unit) { + try { + return this.rwLock.readLock().tryLock(timeout, unit); + } catch (InterruptedException e) { + LOG.warn("failed to try read lock at table[" + name + "]", e); + return false; + } + } + + public void readUnlock() { + this.rwLock.readLock().unlock(); + } + + public void writeLock() { + this.rwLock.writeLock().lock(); + } + + public boolean tryWriteLock(long timeout, TimeUnit unit) { + try { + return this.rwLock.writeLock().tryLock(timeout, unit); + } catch (InterruptedException e) { + LOG.warn("failed to try write lock at table[" + name + "]", e); + return false; + } + } + + public void writeUnlock() { + this.rwLock.writeLock().unlock(); + } + + public boolean isWriteLockHeldByCurrentThread() { + return this.rwLock.writeLock().isHeldByCurrentThread(); + } + public boolean isTypeRead() { return isTypeRead; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index 7cd1c64cc4bfbc..e55464a99290b5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -251,42 +251,37 @@ public TListTableStatusResult listTableStatus(TGetTablesParams params) throws TE currentUser = UserIdentity.createAnalyzedUserIdentWithIp(params.user, params.user_ip); } if (db != null) { - db.readLock(); - try { - List
tables = null; - if (!params.isSetType() || params.getType() == null || params.getType().isEmpty()) { - tables = db.getTables(); - } else { - switch (params.getType()) { - case "VIEW": - tables = db.getViews(); - break; - default: - tables = db.getTables(); - } + List
tables = null; + if (!params.isSetType() || params.getType() == null || params.getType().isEmpty()) { + tables = db.getTables(); + } else { + switch (params.getType()) { + case "VIEW": + tables = db.getViews(); + break; + default: + tables = db.getTables(); + } + } + for (Table table : tables) { + if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(currentUser, params.db, + table.getName(), PrivPredicate.SHOW)) { + continue; } - for (Table table : tables) { - if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(currentUser, params.db, - table.getName(), PrivPredicate.SHOW)) { - continue; - } - if (matcher != null && !matcher.match(table.getName())) { - continue; - } - TTableStatus status = new TTableStatus(); - status.setName(table.getName()); - status.setType(table.getMysqlType()); - status.setEngine(table.getEngine()); - status.setComment(table.getComment()); - status.setCreateTime(table.getCreateTime()); - status.setLastCheckTime(table.getLastCheckTime()); - status.setDdlSql(table.getDdlSql()); - - tablesResult.add(status); + if (matcher != null && !matcher.match(table.getName())) { + continue; } - } finally { - db.readUnlock(); + TTableStatus status = new TTableStatus(); + status.setName(table.getName()); + status.setType(table.getMysqlType()); + status.setEngine(table.getEngine()); + status.setComment(table.getComment()); + status.setCreateTime(table.getCreateTime()); + status.setLastCheckTime(table.getLastCheckTime()); + status.setDdlSql(table.getDdlSql()); + + tablesResult.add(status); } } return result; @@ -372,11 +367,11 @@ public TDescribeTableResult describeTable(TDescribeTableParams params) throws TE Database db = Catalog.getCurrentCatalog().getDb(params.db); if (db != null) { - db.readLock(); - try { - Table table = db.getTable(params.getTableName()); - if (table != null) { - for (Column column : table.getBaseSchema(params.isShowHiddenColumns())) { + Table table = db.getTable(params.getTableName()); + if (table != null) { + table.readLock(); + try { + for (Column column : table.getBaseSchema()) { final TColumnDesc desc = new TColumnDesc(column.getName(), column.getDataType().toThrift()); final Integer precision = column.getOriginType().getPrecision(); if (precision != null) { @@ -398,9 +393,9 @@ public TDescribeTableResult describeTable(TDescribeTableParams params) throws TE } columns.add(colDef); } + } finally { + table.readUnlock(); } - } finally { - db.readUnlock(); } } return result; @@ -757,17 +752,7 @@ private long loadTxnBeginImpl(TLoadTxnBeginRequest request, String clientIp) thr throw new UserException("unknown database, database=" + dbName); } - Table table = null; - db.readLock(); - try { - table = db.getTable(request.tbl); - if (table == null || table.getType() != TableType.OLAP) { - throw new UserException("unknown table, table=" + request.tbl); - } - } finally { - db.readUnlock(); - } - + Table table = db.getTableOrThrowException(request.tbl, TableType.OLAP); // begin long timeoutSecond = request.isSetTimeout() ? request.getTimeout() : Config.stream_load_default_timeout_second; MetricRepo.COUNTER_LOAD_ADD.increase(1L); @@ -927,18 +912,13 @@ private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request) throw new UserException("unknown database, database=" + dbName); } long timeoutMs = request.isSetThriftRpcTimeoutMs() ? request.getThriftRpcTimeoutMs() : 5000; - if (!db.tryReadLock(timeoutMs, TimeUnit.MILLISECONDS)) { - throw new UserException("get database read lock timeout, database=" + fullDbName); + Table table = db.getTableOrThrowException(request.getTbl(), TableType.OLAP); + table.readLock(); + if (!table.tryReadLock(timeoutMs, TimeUnit.MILLISECONDS)) { + throw new UserException("get table read lock timeout, database=" + fullDbName + ",table=" + table.getName()); } try { - Table table = db.getTable(request.getTbl()); - if (table == null) { - throw new UserException("unknown table, table=" + request.getTbl()); - } - if (!(table instanceof OlapTable)) { - throw new UserException("load table type is not OlapTable, type=" + table.getClass()); - } - StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, db); + StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request); StreamLoadPlanner planner = new StreamLoadPlanner(db, (OlapTable) table, streamLoadTask); TExecPlanFragmentParams plan = planner.plan(streamLoadTask.getId()); // add table indexes to transaction state @@ -947,10 +927,9 @@ private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request) throw new UserException("txn does not exist: " + request.getTxnId()); } txnState.addTableIndexes((OlapTable) table); - return plan; } finally { - db.readUnlock(); + table.readUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/StreamLoadTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/StreamLoadTask.java index 626d92d88b3389..93f6ee53690599 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/StreamLoadTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/StreamLoadTask.java @@ -28,7 +28,6 @@ import org.apache.doris.analysis.PartitionNames; import org.apache.doris.analysis.SqlParser; import org.apache.doris.analysis.SqlScanner; -import org.apache.doris.catalog.Database; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.UserException; @@ -196,14 +195,14 @@ public String getSequenceCol() { return sequenceCol; } - public static StreamLoadTask fromTStreamLoadPutRequest(TStreamLoadPutRequest request, Database db) throws UserException { + public static StreamLoadTask fromTStreamLoadPutRequest(TStreamLoadPutRequest request) throws UserException { StreamLoadTask streamLoadTask = new StreamLoadTask(request.getLoadId(), request.getTxnId(), request.getFileType(), request.getFormatType()); - streamLoadTask.setOptionalFromTSLPutRequest(request, db); + streamLoadTask.setOptionalFromTSLPutRequest(request); return streamLoadTask; } - private void setOptionalFromTSLPutRequest(TStreamLoadPutRequest request, Database db) throws UserException { + private void setOptionalFromTSLPutRequest(TStreamLoadPutRequest request) throws UserException { if (request.isSetColumns()) { setColumnToColumnExpr(request.getColumns()); } @@ -331,6 +330,7 @@ private void setColumnSeparator(String oriSeparator) throws AnalysisException { columnSeparator.analyze(); } + @Override public long getMemLimit() { return execMemLimit; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/StreamLoadPlannerTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/StreamLoadPlannerTest.java index 7dae8ede507447..eadeca9dab7a40 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/StreamLoadPlannerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/StreamLoadPlannerTest.java @@ -98,7 +98,7 @@ public void testNormalPlan() throws UserException { request.setLoadId(new TUniqueId(2, 3)); request.setFileType(TFileType.FILE_STREAM); request.setFormatType(TFileFormatType.FORMAT_CSV_PLAIN); - StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, db); + StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request); StreamLoadPlanner planner = new StreamLoadPlanner(db, destTable, streamLoadTask); planner.plan(streamLoadTask.getId()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/StreamLoadScanNodeTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/StreamLoadScanNodeTest.java index 4c4a8c74f64edb..a7a07d0d18e1f8 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/StreamLoadScanNodeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/StreamLoadScanNodeTest.java @@ -174,7 +174,7 @@ List getSequenceColSchema() { private StreamLoadScanNode getStreamLoadScanNode(TupleDescriptor dstDesc, TStreamLoadPutRequest request) throws UserException { - StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, db); + StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request); StreamLoadScanNode scanNode = new StreamLoadScanNode(streamLoadTask.getId(), new PlanNodeId(1), dstDesc, dstTable, streamLoadTask); return scanNode; } @@ -238,7 +238,6 @@ public void testLostV2() throws UserException { TStreamLoadPutRequest request = getBaseRequest(); request.setColumns("k1, k2, v1"); - StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, db); StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request); scanNode.init(analyzer); @@ -268,7 +267,6 @@ public void testBadColumns() throws UserException, UserException { TStreamLoadPutRequest request = getBaseRequest(); request.setColumns("k1 k2 v1"); - StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, null); StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request); scanNode.init(analyzer); @@ -314,7 +312,6 @@ public void testColumnsNormal() throws UserException, UserException { TStreamLoadPutRequest request = getBaseRequest(); request.setColumns("k1,k2,v1, v2=k2"); - StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, db); StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request); scanNode.init(analyzer); scanNode.finalize(analyzer); @@ -361,8 +358,8 @@ public void testHllColumnsNormal() throws UserException { TStreamLoadPutRequest request = getBaseRequest(); request.setFileType(TFileType.FILE_STREAM); + request.setColumns("k1,k2, v1=" + FunctionSet.HLL_HASH + "(k2)"); - StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, db); StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request); scanNode.init(analyzer); @@ -417,7 +414,7 @@ public void testHllColumnsNoHllHash() throws UserException { TStreamLoadPutRequest request = getBaseRequest(); request.setFileType(TFileType.FILE_LOCAL); request.setColumns("k1,k2, v1=hll_hash1(k2)"); - StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, null); + StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request); StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request); scanNode.init(analyzer); @@ -603,7 +600,7 @@ public void testWhereBad() throws UserException, UserException { TStreamLoadPutRequest request = getBaseRequest(); request.setColumns("k1,k2,v1, v2=k2"); request.setWhere("k1 1"); - StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, null); + StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request); StreamLoadScanNode scanNode = new StreamLoadScanNode(streamLoadTask.getId(), new PlanNodeId(1), dstDesc, dstTable, streamLoadTask); From 944f9dc89e5f8f6b432f56417bb25867d8edb5a7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 2 Jun 2020 20:27:35 +0800 Subject: [PATCH 02/50] Use table lock to replace db lock in some functions --- .../java/org/apache/doris/alter/Alter.java | 127 +++++++--- .../apache/doris/alter/SchemaChangeJob.java | 13 +- .../org/apache/doris/backup/RestoreJob.java | 33 +-- .../org/apache/doris/catalog/Catalog.java | 216 ++++++++++-------- .../doris/catalog/ColocateTableIndex.java | 29 +-- .../org/apache/doris/catalog/Database.java | 40 +++- .../apache/doris/catalog/MetadataViewer.java | 29 +-- .../java/org/apache/doris/catalog/Table.java | 2 +- .../doris/clone/ColocateTableBalancer.java | 37 ++- .../load/routineload/RoutineLoadJob.java | 40 +--- .../apache/doris/master/ReportHandler.java | 13 +- .../org/apache/doris/qe/ConnectProcessor.java | 16 +- .../org/apache/doris/qe/ShowExecutor.java | 146 ++++++------ .../doris/system/SystemInfoService.java | 1 - .../doris/task/HadoopLoadPendingTask.java | 58 ++--- .../org/apache/doris/task/LoadEtlTask.java | 28 +-- .../doris/task/MiniLoadPendingTask.java | 16 +- 17 files changed, 467 insertions(+), 377 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index 94d1bb2fe031e5..f0194e19ff90f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -251,20 +251,21 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { } List alterClauses = Lists.newArrayList(); - // some operations will take long time to process, need to be done outside the database lock - boolean needProcessOutsideDatabaseLock = false; + // some operations will take long time to process, need to be done outside the table lock + boolean needProcessOutsideTableLock = false; String tableName = dbTableName.getTbl(); - db.writeLock(); - try { - Table table = db.getTable(tableName); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } + Table table = db.getTable(tableName); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } + + table.writeLock(); + try { switch (table.getType()) { case OLAP: OlapTable olapTable = (OlapTable) table; - needProcessOutsideDatabaseLock = processAlterOlapTable(stmt, olapTable, alterClauses, clusterName, db); + needProcessOutsideTableLock = processAlterOlapTable(stmt, olapTable, alterClauses, clusterName, db); break; case ODBC: case MYSQL: @@ -274,12 +275,78 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { default: throw new DdlException("Do not support alter " + table.getType().toString() + " table[" + tableName + "]"); } + + OlapTable olapTable = (OlapTable) table; + stmt.rewriteAlterClause(olapTable); + + // check conflict alter ops first + alterClauses = stmt.getOps(); + AlterOperations currentAlterOps = new AlterOperations(); + currentAlterOps.checkConflict(alterClauses); + + // check cluster capacity and db quota, only need to check once. + if (currentAlterOps.needCheckCapacity()) { + Catalog.getCurrentSystemInfo().checkClusterCapacity(clusterName); + db.checkQuota(); + } + + if (olapTable.getState() != OlapTableState.NORMAL) { + throw new DdlException( + "Table[" + table.getName() + "]'s state is not NORMAL. Do not allow doing ALTER ops"); + } + + if (currentAlterOps.hasSchemaChangeOp()) { + // if modify storage type to v2, do schema change to convert all related tablets to segment v2 format + schemaChangeHandler.process(alterClauses, clusterName, db, olapTable); + } else if (currentAlterOps.hasRollupOp()) { + materializedViewHandler.process(alterClauses, clusterName, db, olapTable); + } else if (currentAlterOps.hasPartitionOp()) { + Preconditions.checkState(alterClauses.size() == 1); + AlterClause alterClause = alterClauses.get(0); + if (alterClause instanceof DropPartitionClause) { + if (!((DropPartitionClause) alterClause).isTempPartition()) { + DynamicPartitionUtil.checkAlterAllowed((OlapTable) db.getTable(tableName)); + } + Catalog.getCurrentCatalog().dropPartition(db, olapTable, ((DropPartitionClause) alterClause)); + } else if (alterClause instanceof ReplacePartitionClause) { + Catalog.getCurrentCatalog().replaceTempPartition(db, tableName, (ReplacePartitionClause) alterClause); + } else if (alterClause instanceof ModifyPartitionClause) { + ModifyPartitionClause clause = ((ModifyPartitionClause) alterClause); + // expand the partition names if it is 'Modify Partition(*)' + if (clause.isNeedExpand()) { + List partitionNames = clause.getPartitionNames(); + partitionNames.clear(); + for (Partition partition : olapTable.getPartitions()) { + partitionNames.add(partition.getName()); + } + } + Map properties = clause.getProperties(); + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_INMEMORY)) { + needProcessOutsideTableLock = true; + } else { + List partitionNames = clause.getPartitionNames(); + modifyPartitionsProperty(db, olapTable, partitionNames, properties); + } + } else if (alterClause instanceof AddPartitionClause) { + needProcessOutsideTableLock = true; + } else { + throw new DdlException("Invalid alter operation: " + alterClause.getOpType()); + } + } else if (currentAlterOps.hasRenameOp()) { + processRename(db, olapTable, alterClauses); + } else if (currentAlterOps.hasReplaceTableOp()) { + processReplaceTable(db, olapTable, alterClauses); + } else if (currentAlterOps.contains(AlterOpType.MODIFY_TABLE_PROPERTY_SYNC)) { + needProcessOutsideTableLock = true; + } else { + throw new DdlException("Invalid alter operations: " + currentAlterOps); + } } finally { - db.writeUnlock(); + table.writeUnlock(); } - // the following ops should done outside db lock. because it contain synchronized create operation - if (needProcessOutsideDatabaseLock) { + // the following ops should done outside table lock. because it contain synchronized create operation + if (needProcessOutsideTableLock) { Preconditions.checkState(alterClauses.size() == 1); AlterClause alterClause = alterClauses.get(0); if (alterClause instanceof AddPartitionClause) { @@ -296,12 +363,12 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { ((SchemaChangeHandler) schemaChangeHandler).updatePartitionsInMemoryMeta( db, tableName, partitionNames, properties); - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(tableName); + olapTable.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(tableName); modifyPartitionsProperty(db, olapTable, partitionNames, properties); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } else if (alterClause instanceof ModifyTablePropertiesClause) { Map properties = alterClause.getProperties(); @@ -406,15 +473,7 @@ public void processAlterView(AlterViewStmt stmt, ConnectContext ctx) throws User String tableName = dbTableName.getTbl(); db.writeLock(); try { - Table table = db.getTable(tableName); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } - - if (table.getType() != TableType.VIEW) { - throw new DdlException("The specified table [" + tableName + "] is not a view"); - } - + Table table = db.getTableOrThrowException(tableName, TableType.VIEW); View view = (View) table; modifyViewDef(db, view, stmt.getInlineViewDef(), ctx.getSessionVariable().getSqlMode(), stmt.getColumns()); } finally { @@ -423,16 +482,20 @@ public void processAlterView(AlterViewStmt stmt, ConnectContext ctx) throws User } private void modifyViewDef(Database db, View view, String inlineViewDef, long sqlMode, List newFullSchema) throws DdlException { - String viewName = view.getName(); - - view.setInlineViewDefWithSqlMode(inlineViewDef, sqlMode); + view.writeLock(); try { - view.init(); - } catch (UserException e) { - throw new DdlException("failed to init view stmt", e); + view.setInlineViewDefWithSqlMode(inlineViewDef, sqlMode); + try { + view.init(); + } catch (UserException e) { + throw new DdlException("failed to init view stmt", e); + } + view.setNewFullSchema(newFullSchema); + } finally { + view.writeUnlock(); } - view.setNewFullSchema(newFullSchema); + String viewName = view.getName(); db.dropTable(viewName); db.createTable(view); @@ -513,7 +576,7 @@ public void modifyPartitionsProperty(Database db, List partitionNames, Map properties) throws DdlException, AnalysisException { - Preconditions.checkArgument(db.isWriteLockHeldByCurrentThread()); + Preconditions.checkArgument(olapTable.isWriteLockHeldByCurrentThread()); List modifyPartitionInfos = Lists.newArrayList(); if (olapTable.getState() != OlapTableState.NORMAL) { throw new DdlException("Table[" + olapTable.getName() + "]'s state is not NORMAL"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java index ef4d151d1049c0..e9b995a5b1ef21 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java @@ -584,12 +584,13 @@ public void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo, if (db == null) { throw new MetaNotFoundException("Cannot find db[" + dbId + "]"); } - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + throw new MetaNotFoundException("Cannot find table[" + tableId + "]"); + } + + olapTable.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - throw new MetaNotFoundException("Cannot find table[" + tableId + "]"); - } Preconditions.checkState(olapTable.getState() == OlapTableState.SCHEMA_CHANGE); Partition partition = olapTable.getPartition(partitionId); @@ -626,7 +627,7 @@ public void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo, replica.setPathHash(finishTabletInfo.getPathHash()); } } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } Catalog.getCurrentSystemInfo().updateBackendReportVersion(schemaChangeTask.getBackendId(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java index 8b79bf065c3759..4fb45f4afde68e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -419,20 +419,20 @@ private void checkAndPrepareMeta() { // Set all restored tbls' state to RESTORE // Table's origin state must be NORMAL and does not have unfinished load job. - db.writeLock(); - try { - for (BackupTableInfo tblInfo : jobInfo.tables.values()) { - Table tbl = db.getTable(jobInfo.getAliasByOriginNameIfSet(tblInfo.name)); - if (tbl == null) { - continue; - } + for (BackupTableInfo tblInfo : jobInfo.tables.values()) { + Table tbl = db.getTable(jobInfo.getAliasByOriginNameIfSet(tblInfo.name)); + if (tbl == null) { + continue; + } - if (tbl.getType() != TableType.OLAP) { - status = new Status(ErrCode.COMMON_ERROR, "Only support retore OLAP table: " + tbl.getName()); - return; - } + if (tbl.getType() != TableType.OLAP) { + status = new Status(ErrCode.COMMON_ERROR, "Only support retore OLAP table: " + tbl.getName()); + return; + } - OlapTable olapTbl = (OlapTable) tbl; + OlapTable olapTbl = (OlapTable) tbl; + olapTbl.writeLock(); + try { if (olapTbl.getState() != OlapTableState.NORMAL) { status = new Status(ErrCode.COMMON_ERROR, "Table " + tbl.getName() + "'s state is not NORMAL: " + olapTbl.getState().name()); @@ -443,7 +443,7 @@ private void checkAndPrepareMeta() { status = new Status(ErrCode.COMMON_ERROR, "Do not support restoring table with temp partitions"); return; } - + for (Partition partition : olapTbl.getPartitions()) { if (!catalog.getLoadInstance().checkPartitionLoadFinished(partition.getId(), null)) { status = new Status(ErrCode.COMMON_ERROR, @@ -451,11 +451,12 @@ private void checkAndPrepareMeta() { return; } } - + olapTbl.setState(OlapTableState.RESTORE); + } finally { + olapTbl.writeUnlock(); } - } finally { - db.writeUnlock(); + } // Check and prepare meta objects. diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index 4860d689b5df2a..e9823237fa3d6f 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -260,7 +260,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; - public class Catalog { private static final Logger LOG = LogManager.getLogger(Catalog.class); // 0 ~ 9999 used for qe @@ -2044,9 +2043,16 @@ public long saveDb(DataOutputStream dos, long checksum) throws IOException { if (!InfoSchemaDb.isInfoSchemaDb(dbName)) { checksum ^= entry.getKey(); db.readLock(); + List
tableList = db.getOrderedTablesById(); + for (Table table : tableList) { + table.readLock(); + } try { db.write(dos); } finally { + for (int i = tableList.size() - 1; i >= 0; i--) { + tableList.get(i).readUnlock(); + } db.readUnlock(); } } @@ -3040,18 +3046,13 @@ public void createTable(CreateTableStmt stmt) throws DdlException { } // check if table exists in db - db.readLock(); - try { - if (db.getTable(tableName) != null) { - if (stmt.isSetIfNotExists()) { - LOG.info("create table[{}] which already exists", tableName); - return; - } else { - ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); - } + if (db.getTable(tableName) != null) { + if (stmt.isSetIfNotExists()) { + LOG.info("create table[{}] which already exists", tableName); + return; + } else { + ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); } - } finally { - db.readUnlock(); } if (engineName.equals("olap")) { @@ -3121,19 +3122,20 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa String partitionName = singlePartitionDesc.getPartitionName(); // check - db.readLock(); - try { - Table table = db.getTable(tableName); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } + Table table = db.getTable(tableName); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } - if (table.getType() != TableType.OLAP) { - throw new DdlException("Table[" + tableName + "] is not OLAP table"); - } + if (table.getType() != TableType.OLAP) { + throw new DdlException("Table[" + tableName + "] is not OLAP table"); + } + + // check state + olapTable = (OlapTable) table; - // check state - olapTable = (OlapTable) table; + olapTable.readLock(); + try { if (olapTable.getState() != OlapTableState.NORMAL) { throw new DdlException("Table[" + tableName + "]'s state is not NORMAL"); } @@ -3210,7 +3212,7 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa } catch (AnalysisException e) { throw new DdlException(e.getMessage()); } finally { - db.readUnlock(); + olapTable.readUnlock(); } Preconditions.checkNotNull(distributionInfo); @@ -3241,17 +3243,17 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa ); // check again - db.writeLock(); - try { - Table table = db.getTable(tableName); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } + table = db.getTable(tableName); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } - if (table.getType() != TableType.OLAP) { - throw new DdlException("Table[" + tableName + "] is not OLAP table"); - } + if (table.getType() != TableType.OLAP) { + throw new DdlException("Table[" + tableName + "] is not OLAP table"); + } + table.writeLock(); + try { olapTable = (OlapTable) table; if (olapTable.getState() != OlapTableState.NORMAL) { throw new DdlException("Table[" + tableName + "]'s state is not NORMAL"); @@ -3318,7 +3320,7 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa LOG.info("succeed in creating partition[{}], temp: {}", partitionId, isTempPartition); } finally { - db.writeUnlock(); + table.writeUnlock(); } } catch (DdlException e) { for (Long tabletId : tabletIdSet) { @@ -3328,13 +3330,12 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa } } - public void replayAddPartition(PartitionPersistInfo info) throws DdlException { + public void replayAddPartition(PartitionPersistInfo info) throws UserException { Database db = this.getDb(info.getDbId()); - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTableOrThrowException(info.getTableId(), TableType.OLAP); + olapTable.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(info.getTableId()); Partition partition = info.getPartition(); - PartitionInfo partitionInfo = olapTable.getPartitionInfo(); if (info.isTempPartition()) { olapTable.addTempPartition(partition); @@ -3364,12 +3365,12 @@ public void replayAddPartition(PartitionPersistInfo info) throws DdlException { } } } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } public void dropPartition(Database db, OlapTable olapTable, DropPartitionClause clause) throws DdlException { - Preconditions.checkArgument(db.isWriteLockHeldByCurrentThread()); + Preconditions.checkArgument(olapTable.isWriteLockHeldByCurrentThread()); String partitionName = clause.getPartitionName(); boolean isTempPartition = clause.isTempPartition(); @@ -3879,7 +3880,6 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws DdlExcept throw e; } - return; } private void createMysqlTable(Database db, CreateTableStmt stmt) throws DdlException { @@ -4708,25 +4708,31 @@ public HashMap getPartitionIdToStorageMediumMap() { long tableId = table.getId(); OlapTable olapTable = (OlapTable) table; - PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - for (Partition partition : olapTable.getAllPartitions()) { - long partitionId = partition.getId(); - DataProperty dataProperty = partitionInfo.getDataProperty(partition.getId()); - Preconditions.checkNotNull(dataProperty, partition.getName() + ", pId:" + partitionId + ", db: " + dbId + ", tbl: " + tableId); - if (dataProperty.getStorageMedium() == TStorageMedium.SSD - && dataProperty.getCooldownTimeMs() < currentTimeMs) { - // expire. change to HDD. - // record and change when holding write lock - Multimap multimap = changedPartitionsMap.get(dbId); - if (multimap == null) { - multimap = HashMultimap.create(); - changedPartitionsMap.put(dbId, multimap); + olapTable.readLock(); + try { + PartitionInfo partitionInfo = olapTable.getPartitionInfo(); + for (Partition partition : olapTable.getAllPartitions()) { + long partitionId = partition.getId(); + DataProperty dataProperty = partitionInfo.getDataProperty(partition.getId()); + Preconditions.checkNotNull(dataProperty, partition.getName() + ", pId:" + partitionId + ", db: " + dbId + ", tbl: " + tableId); + if (dataProperty.getStorageMedium() == TStorageMedium.SSD + && dataProperty.getCooldownTimeMs() < currentTimeMs) { + // expire. change to HDD. + // record and change when holding write lock + Multimap multimap = changedPartitionsMap.get(dbId); + if (multimap == null) { + multimap = HashMultimap.create(); + changedPartitionsMap.put(dbId, multimap); + } + multimap.put(tableId, partitionId); + } else { + storageMediumMap.put(partitionId, dataProperty.getStorageMedium()); } - multimap.put(tableId, partitionId); - } else { - storageMediumMap.put(partitionId, dataProperty.getStorageMedium()); - } - } // end for partitions + } // end for partitions + } finally { + olapTable.readUnlock(); + } + } // end for tables } finally { db.readUnlock(); @@ -5564,18 +5570,13 @@ public void createView(CreateViewStmt stmt) throws DdlException { } // check if table exists in db - db.readLock(); - try { - if (db.getTable(tableName) != null) { - if (stmt.isSetIfNotExists()) { - LOG.info("create view[{}] which already exists", tableName); - return; - } else { - ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); - } + if (db.getTable(tableName) != null) { + if (stmt.isSetIfNotExists()) { + LOG.info("create view[{}] which already exists", tableName); + return; + } else { + ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); } - } finally { - db.readUnlock(); } List columns = stmt.getColumns(); @@ -6104,30 +6105,35 @@ public Set getMigrations() { } OlapTable olapTable = (OlapTable) table; - for (Partition partition : olapTable.getPartitions()) { - final short replicationNum = olapTable.getPartitionInfo() - .getReplicationNum(partition.getId()); - for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { - if (materializedIndex.getState() != IndexState.NORMAL) { - continue; - } - for (Tablet tablet : materializedIndex.getTablets()) { - int replicaNum = 0; - int quorum = replicationNum / 2 + 1; - for (Replica replica : tablet.getReplicas()) { - if (replica.getState() != ReplicaState.CLONE - && beIds.contains(replica.getBackendId())) { - replicaNum++; - } - } - if (replicaNum > quorum) { - replicaNum = quorum; + olapTable.readLock(); + try { + for (Partition partition : olapTable.getPartitions()) { + final short replicationNum = olapTable.getPartitionInfo() + .getReplicationNum(partition.getId()); + for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { + if (materializedIndex.getState() != IndexState.NORMAL) { + continue; } + for (Tablet tablet : materializedIndex.getTablets()) { + int replicaNum = 0; + int quorum = replicationNum / 2 + 1; + for (Replica replica : tablet.getReplicas()) { + if (replica.getState() != ReplicaState.CLONE + && beIds.contains(replica.getBackendId())) { + replicaNum++; + } + } + if (replicaNum > quorum) { + replicaNum = quorum; + } - tabletQuorum = tabletQuorum + replicaNum; - tabletTotal = tabletTotal + quorum; + tabletQuorum = tabletQuorum + replicaNum; + tabletTotal = tabletTotal + quorum; + } } } + } finally { + olapTable.readUnlock(); } } final BaseParam info = new BaseParam(); @@ -6317,6 +6323,10 @@ public String dumpImage() { // lock all dbs for (Database db : lockedDbMap.values()) { db.readLock(); + List
tableList = db.getOrderedTablesById(); + for (Table table : tableList) { + table.readLock(); + } } LOG.info("acquired all the dbs' read lock."); @@ -6336,9 +6346,12 @@ public String dumpImage() { // unlock all load.readUnlock(); for (Database db : lockedDbMap.values()) { + List
tableList = db.getOrderedTablesById(); + for (int i = tableList.size() - 1; i >= 0; i--) { + tableList.get(i).readUnlock(); + } db.readUnlock(); } - unlock(); } @@ -6370,17 +6383,18 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti } boolean truncateEntireTable = tblRef.getPartitionNames() == null; - db.readLock(); - try { - Table table = db.getTable(dbTbl.getTbl()); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, dbTbl.getTbl()); - } - if (table.getType() != TableType.OLAP) { - throw new DdlException("Only support truncate OLAP table"); - } + Table table = db.getTable(dbTbl.getTbl()); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, dbTbl.getTbl()); + } + if (table.getType() != TableType.OLAP) { + throw new DdlException("Only support truncate OLAP table"); + } + + table.readLock(); + try { OlapTable olapTable = (OlapTable) table; if (olapTable.getState() != OlapTableState.NORMAL) { throw new DdlException("Table' state is not NORMAL: " + olapTable.getState()); @@ -6403,7 +6417,7 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti copiedTbl = olapTable.selectiveCopy(origPartitions.keySet(), true, IndexExtState.VISIBLE); } finally { - db.readUnlock(); + table.readUnlock(); } // 2. use the copied table to create partitions diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java index 626b25f4f9961f..7c5fd98b8dc1e0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java @@ -656,30 +656,31 @@ private void convertedToNewMembers(Multimap tmpGroup2Tables, Map tableIds = tmpGroup2Tables.get(groupId.grpId); - db.readLock(); - try { - for (Long tblId : tableIds) { - OlapTable tbl = (OlapTable) db.getTable(tblId); - if (tbl == null) { - continue; - } + + for (Long tblId : tableIds) { + OlapTable tbl = (OlapTable) db.getTable(tblId); + if (tbl == null) { + continue; + } + tbl.readLock(); + try { if (tblId.equals(groupId.grpId)) { // this is a parent table, use its name as group name groupName2Id.put(groupId.dbId + "_" + tbl.getName(), groupId); - + ColocateGroupSchema groupSchema = new ColocateGroupSchema(groupId, - ((HashDistributionInfo)tbl.getDefaultDistributionInfo()).getDistributionColumns(), + ((HashDistributionInfo)tbl.getDefaultDistributionInfo()).getDistributionColumns(), tbl.getDefaultDistributionInfo().getBucketNum(), tbl.getPartitionInfo().idToReplicationNum.values().stream().findFirst().get()); group2Schema.put(groupId, groupSchema); group2BackendsPerBucketSeq.put(groupId, tmpGroup2BackendsPerBucketSeq.get(groupId.grpId)); } - - group2Tables.put(groupId, tblId); - table2Group.put(tblId, groupId); + } finally { + tbl.readUnlock(); } - } finally { - db.readUnlock(); + + group2Tables.put(groupId, tblId); + table2Group.put(tblId, groupId); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 6c2ded979e2b19..f30da1d9fdef2f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -45,6 +45,7 @@ import java.io.IOException; import java.io.UnsupportedEncodingException; import java.util.ArrayList; +import java.util.Comparator; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -53,6 +54,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; import java.util.zip.Adler32; /** @@ -76,7 +78,7 @@ public class Database extends MetaObject implements Writable { public static final long TRY_LOCK_TIMEOUT_MS = 100L; private long id; - private String fullQualifiedName; + private volatile String fullQualifiedName; private String clusterName; private ReentrantReadWriteLock rwLock; @@ -213,7 +215,12 @@ public long getUsedDataQuotaWithLock() { } OlapTable olapTable = (OlapTable) table; - usedDataQuota = usedDataQuota + olapTable.getDataSize(); + olapTable.readLock(); + try { + usedDataQuota = usedDataQuota + olapTable.getDataSize(); + } finally { + olapTable.readUnlock(); + } } return usedDataQuota; } finally { @@ -232,7 +239,12 @@ public long getReplicaQuotaLeftWithLock() { } OlapTable olapTable = (OlapTable) table; - usedReplicaQuota = usedReplicaQuota + olapTable.getReplicaCount(); + olapTable.readLock(); + try { + usedReplicaQuota = usedReplicaQuota + olapTable.getReplicaCount(); + } finally { + olapTable.readUnlock(); + } } long leftReplicaQuota = replicaQuotaSize - usedReplicaQuota; @@ -352,7 +364,14 @@ public void dropTable(String tableName) { } public List
getTables() { - return new ArrayList
(idToTable.values()); + return new ArrayList<>(idToTable.values()); + } + + // tables must get read or write table in fixed order to escape potential dead lock + public List
getOrderedTablesById() { + return idToTable.values().stream() + .sorted(Comparator.comparing(Table::getId)) + .collect(Collectors.toList()); } public List
getViews() { @@ -436,11 +455,16 @@ public int getMaxReplicationNum() { continue; } OlapTable olapTable = (OlapTable) table; - for (Partition partition : olapTable.getAllPartitions()) { - short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId()); - if (ret < replicationNum) { - ret = replicationNum; + table.readLock(); + try { + for (Partition partition : olapTable.getAllPartitions()) { + short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId()); + if (ret < replicationNum) { + ret = replicationNum; + } } + } finally { + table.readUnlock(); } } } finally { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java index cdc851de6866da..b52a3285304190 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java @@ -55,14 +55,14 @@ private static List> getTabletStatus(String dbName, String tblName, if (db == null) { throw new DdlException("Database " + dbName + " does not exist"); } - - db.readLock(); + + Table tbl = db.getTable(tblName); + if (tbl == null || tbl.getType() != TableType.OLAP) { + throw new DdlException("Table does not exist or is not OLAP table: " + tblName); + } + + tbl.readLock(); try { - Table tbl = db.getTable(tblName); - if (tbl == null || tbl.getType() != TableType.OLAP) { - throw new DdlException("Table does not exist or is not OLAP table: " + tblName); - } - OlapTable olapTable = (OlapTable) tbl; if (partitions.isEmpty()) { @@ -146,7 +146,7 @@ private static List> getTabletStatus(String dbName, String tblName, } } } finally { - db.readUnlock(); + tbl.readUnlock(); } return result; @@ -181,12 +181,13 @@ private static List> getTabletDistribution(String dbName, String tb throw new DdlException("Database " + dbName + " does not exist"); } - db.readLock(); + Table tbl = db.getTable(tblName); + if (tbl == null || tbl.getType() != TableType.OLAP) { + throw new DdlException("Table does not exist or is not OLAP table: " + tblName); + } + + tbl.readLock(); try { - Table tbl = db.getTable(tblName); - if (tbl == null || tbl.getType() != TableType.OLAP) { - throw new DdlException("Table does not exist or is not OLAP table: " + tblName); - } OlapTable olapTable = (OlapTable) tbl; @@ -242,7 +243,7 @@ private static List> getTabletDistribution(String dbName, String tb } } finally { - db.readUnlock(); + tbl.readUnlock(); } return result; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java index d3ce8bb873758a..6d1d648ece0a48 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java @@ -62,7 +62,7 @@ public enum TableType { } protected long id; - protected String name; + protected volatile String name; protected TableType type; protected long createTime; protected ReentrantReadWriteLock rwLock; diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java b/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java index 9674f411098191..78d598ca83c42e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java @@ -192,14 +192,13 @@ private void matchGroup() { } boolean isGroupStable = true; - db.readLock(); - try { - OUT: for (Long tableId : tableIds) { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null || !colocateIndex.isColocateTable(olapTable.getId())) { - continue; - } - + OUT: for (Long tableId : tableIds) { + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null || !colocateIndex.isColocateTable(olapTable.getId())) { + continue; + } + olapTable.readLock(); + try { for (Partition partition : olapTable.getPartitions()) { short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId()); long visibleVersion = partition.getVisibleVersion(); @@ -241,19 +240,19 @@ private void matchGroup() { } } } - } // end for tables - - // mark group as stable or unstable - if (isGroupStable) { - colocateIndex.markGroupStable(groupId, true); - } else { - colocateIndex.markGroupUnstable(groupId, true); + } finally { + olapTable.readUnlock(); } - } finally { - db.readUnlock(); + } // end for tables + + // mark group as stable or unstable + if (isGroupStable) { + colocateIndex.markGroupStable(groupId, true); + } else { + colocateIndex.markGroupUnstable(groupId, true); } - } // end for groups - } + } + } // end for groups /* * The balance logic is as follow: diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java index 80d4954ba0a965..cbe30e6fe2689a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java @@ -408,12 +408,7 @@ public String getDbFullName() throws MetaNotFoundException { if (database == null) { throw new MetaNotFoundException("Database " + dbId + "has been deleted"); } - database.readLock(); - try { - return database.getFullName(); - } finally { - database.readUnlock(); - } + return database.getFullName(); } public long getTableId() { @@ -425,16 +420,13 @@ public String getTableName() throws MetaNotFoundException { if (database == null) { throw new MetaNotFoundException("Database " + dbId + "has been deleted"); } - database.readLock(); - try { - Table table = database.getTable(tableId); - if (table == null) { - throw new MetaNotFoundException("Failed to find table " + tableId + " in db " + dbId); - } - return table.getName(); - } finally { - database.readUnlock(); + + Table table = database.getTable(tableId); + if (table == null) { + throw new MetaNotFoundException("Failed to find table " + tableId + " in db " + dbId); } + return table.getName(); + } public JobState getState() { @@ -789,7 +781,8 @@ public TExecPlanFragmentParams plan(TUniqueId loadId, long txnId) throws UserExc if (db == null) { throw new MetaNotFoundException("db " + dbId + " does not exist"); } - db.readLock(); + Table table = db.getTableOrThrowException(tableId, Table.TableType.OLAP); + table.readLock(); try { TExecPlanFragmentParams planParams = planner.plan(loadId); // add table indexes to transaction state @@ -801,7 +794,7 @@ public TExecPlanFragmentParams plan(TUniqueId loadId, long txnId) throws UserExc return planParams; } finally { - db.readUnlock(); + table.readUnlock(); } } @@ -1212,12 +1205,7 @@ public void update() throws UserException { // check table belong to database database.readLock(); - Table table; - try { - table = database.getTable(tableId); - } finally { - database.readUnlock(); - } + Table table = database.getTable(tableId); if (table == null) { LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id).add("db_id", dbId) .add("table_id", tableId) @@ -1272,11 +1260,7 @@ public List getShowInfo() { Table tbl = null; if (db != null) { db.readLock(); - try { - tbl = db.getTable(tableId); - } finally { - db.readUnlock(); - } + tbl = db.getTable(tableId); } readLock(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java index 56b3a9adeceeb8..4c3197e1c48c6d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java @@ -901,12 +901,13 @@ private static void handleSetTabletInMemory(long backendId, Map b if (db == null) { continue; } - db.readLock(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + continue; + } + + olapTable.readLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - continue; - } Partition partition = olapTable.getPartition(partitionId); if (partition == null) { continue; @@ -916,7 +917,7 @@ private static void handleSetTabletInMemory(long backendId, Map b tabletToInMemory.add(new ImmutableTriple<>(tabletId, tabletInfo.getSchemaHash(), feIsInMemory)); } } finally { - db.readUnlock(); + olapTable.readUnlock(); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java index 679e1841df7c96..7118c6f38b3aa7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -278,14 +278,14 @@ private void handleFieldList() throws IOException { ctx.getState().setError("Unknown database(" + ctx.getDatabase() + ")"); return; } - db.readLock(); - try { - Table table = db.getTable(tableName); - if (table == null) { - ctx.getState().setError("Unknown table(" + tableName + ")"); - return; - } + Table table = db.getTable(tableName); + if (table == null) { + ctx.getState().setError("Unknown table(" + tableName + ")"); + return; + } + table.readLock(); + try { MysqlSerializer serializer = ctx.getSerializer(); MysqlChannel channel = ctx.getMysqlChannel(); @@ -299,7 +299,7 @@ private void handleFieldList() throws IOException { } } finally { - db.readUnlock(); + table.readUnlock(); } ctx.getState().setEof(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java index 1e38769656720a..a3045af702bbe8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -570,13 +570,13 @@ private void handleShowCreateTable() throws AnalysisException { ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showStmt.getDb()); } List> rows = Lists.newArrayList(); - db.readLock(); - try { - Table table = db.getTable(showStmt.getTable()); - if (table == null) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, showStmt.getTable()); - } + Table table = db.getTable(showStmt.getTable()); + if (table == null) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, showStmt.getTable()); + } + table.readLock(); + try { List createTableStmt = Lists.newArrayList(); Catalog.getDdlStmt(table, createTableStmt, null, null, false, true /* hide password */); if (createTableStmt.isEmpty()) { @@ -599,7 +599,7 @@ private void handleShowCreateTable() throws AnalysisException { resultSet = new ShowResultSet(showStmt.getMetaData(), rows); } } finally { - db.readUnlock(); + table.readUnlock(); } } @@ -615,15 +615,15 @@ private void handleShowColumn() throws AnalysisException { List> rows = Lists.newArrayList(); Database db = ctx.getCatalog().getDb(showStmt.getDb()); if (db != null) { - db.readLock(); - try { - Table table = db.getTable(showStmt.getTable()); - if (table != null) { - PatternMatcher matcher = null; - if (showStmt.getPattern() != null) { - matcher = PatternMatcher.createMysqlPattern(showStmt.getPattern(), - CaseSensibility.COLUMN.getCaseSensibility()); - } + Table table = db.getTable(showStmt.getTable()); + if (table != null) { + PatternMatcher matcher = null; + if (showStmt.getPattern() != null) { + matcher = PatternMatcher.createMysqlPattern(showStmt.getPattern(), + CaseSensibility.COLUMN.getCaseSensibility()); + } + table.readLock(); + try { List columns = table.getBaseSchema(); for (Column col : columns) { if (matcher != null && !matcher.match(col.getName())) { @@ -639,29 +639,29 @@ private void handleShowColumn() throws AnalysisException { // Field Type Collation Null Key Default Extra // Privileges Comment rows.add(Lists.newArrayList(columnName, - columnType, - "", - isAllowNull, - isKey, - defaultValue, - aggType, - "", - col.getComment())); + columnType, + "", + isAllowNull, + isKey, + defaultValue, + aggType, + "", + col.getComment())); } else { // Field Type Null Key Default Extra rows.add(Lists.newArrayList(columnName, - columnType, - isAllowNull, - isKey, - defaultValue, - aggType)); + columnType, + isAllowNull, + isKey, + defaultValue, + aggType)); } } - } else { - ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, db.getFullName() + "." + showStmt.getTable()); + } finally { + table.readUnlock(); } - } finally { - db.readUnlock(); + } else { + ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, db.getFullName() + "." + showStmt.getTable()); } } else { ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, showStmt.getDb() + "." + showStmt.getTable()); @@ -677,22 +677,23 @@ private void handleShowIndex() throws AnalysisException { if (db == null) { ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, showStmt.getTableName().toString()); } - db.readLock(); - try { - Table table = db.getTable(showStmt.getTableName().getTbl()); - if (table != null && table instanceof OlapTable) { + + Table table = db.getTable(showStmt.getTableName().getTbl()); + if (table != null && table instanceof OlapTable) { + table.readLock(); + try { List indexes = ((OlapTable) table).getIndexes(); for (Index index : indexes) { rows.add(Lists.newArrayList(showStmt.getTableName().toString(), "", index.getIndexName(), "", index.getColumns().stream().collect(Collectors.joining(",")), "", "", "", "", "", index.getIndexType().name(), index.getComment())); } - } else { - ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, - db.getFullName() + "." + showStmt.getTableName().toString()); + } finally { + table.readUnlock(); } - } finally { - db.readUnlock(); + } else { + ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, + db.getFullName() + "." + showStmt.getTableName().toString()); } resultSet = new ShowResultSet(showStmt.getMetaData(), rows); } @@ -1151,16 +1152,15 @@ private void handleShowTablet() throws AnalysisException { break; } dbName = db.getFullName(); + Table table = db.getTable(tableId); + if (table == null || !(table instanceof OlapTable)) { + isSync = false; + break; + } - db.readLock(); + table.readLock(); try { - Table table = db.getTable(tableId); - if (table == null || !(table instanceof OlapTable)) { - isSync = false; - break; - } tableName = table.getName(); - OlapTable olapTable = (OlapTable) table; Partition partition = olapTable.getPartition(partitionId); if (partition == null) { @@ -1197,7 +1197,7 @@ private void handleShowTablet() throws AnalysisException { } } finally { - db.readUnlock(); + table.readUnlock(); } } while (false); @@ -1213,16 +1213,16 @@ private void handleShowTablet() throws AnalysisException { ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showStmt.getDbName()); } - db.readLock(); - try { - Table table = db.getTable(showStmt.getTableName()); - if (table == null) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, showStmt.getTableName()); - } - if (!(table instanceof OlapTable)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_NOT_OLAP_TABLE, showStmt.getTableName()); - } + Table table = db.getTable(showStmt.getTableName()); + if (table == null) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, showStmt.getTableName()); + } + if (!(table instanceof OlapTable)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NOT_OLAP_TABLE, showStmt.getTableName()); + } + table.readLock(); + try { OlapTable olapTable = (OlapTable) table; long sizeLimit = -1; if (showStmt.hasOffset() && showStmt.hasLimit()) { @@ -1298,7 +1298,7 @@ private void handleShowTablet() throws AnalysisException { rows.add(oneTablet); } } finally { - db.readUnlock(); + table.readUnlock(); } } @@ -1536,19 +1536,28 @@ private void handleShowDynamicPartition() { List> rows = Lists.newArrayList(); Database db = ctx.getCatalog().getDb(showDynamicPartitionStmt.getDb()); if (db != null) { + List
tableList = null; db.readLock(); try { - for (Table tbl : db.getTables()) { - if (!(tbl instanceof OlapTable)) { - continue; - } + tableList = db.getTables(); + } finally { + db.readUnlock(); + } + + for (Table tbl : tableList) { + if (!(tbl instanceof OlapTable)) { + continue; + } - DynamicPartitionScheduler dynamicPartitionScheduler = Catalog.getCurrentCatalog().getDynamicPartitionScheduler(); - OlapTable olapTable = (OlapTable) tbl; + DynamicPartitionScheduler dynamicPartitionScheduler = Catalog.getCurrentCatalog().getDynamicPartitionScheduler(); + OlapTable olapTable = (OlapTable) tbl; + olapTable.readLock(); + try { if (!olapTable.dynamicPartitionExists()) { dynamicPartitionScheduler.removeRuntimeInfo(olapTable.getName()); continue; } + // check tbl privs if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), db.getFullName(), olapTable.getName(), @@ -1574,10 +1583,11 @@ private void handleShowDynamicPartition() { dynamicPartitionScheduler.getRuntimeInfo(tableName, DynamicPartitionScheduler.DYNAMIC_PARTITION_STATE), dynamicPartitionScheduler.getRuntimeInfo(tableName, DynamicPartitionScheduler.CREATE_PARTITION_MSG), dynamicPartitionScheduler.getRuntimeInfo(tableName, DynamicPartitionScheduler.DROP_PARTITION_MSG))); + } finally { + olapTable.readUnlock(); } - } finally { - db.readUnlock(); } + resultSet = new ShowResultSet(showDynamicPartitionStmt.getMetaData(), rows); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java index f1c5b0442f4585..059ccec6121254 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java @@ -896,7 +896,6 @@ public void updateBackendReportVersion(long backendId, long newReportVersion, lo if ((atomicLong = idToReportVersionRef.get(backendId)) != null) { Database db = Catalog.getCurrentCatalog().getDb(dbId); if (db != null) { - db.readLock(); try { atomicLong.set(newReportVersion); LOG.debug("update backend {} report version: {}, db: {}", backendId, newReportVersion, dbId); diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java index a346fe85f4f6be..ebbfbce002e3fa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java @@ -70,41 +70,41 @@ public HadoopLoadPendingTask(LoadJob job) { @Override protected void createEtlRequest() throws Exception { - db.readLock(); - try { - EtlTaskConf taskConf = new EtlTaskConf(); - // output path - taskConf.setOutputPath(getOutputPath()); - // output file pattern - taskConf.setOutputFilePattern(job.getLabel() + ".%(table)s.%(view)s.%(bucket)s"); - // tables (partitions) - Map etlPartitions = createEtlPartitions(); - Preconditions.checkNotNull(etlPartitions); - taskConf.setEtlPartitions(etlPartitions); + EtlTaskConf taskConf = new EtlTaskConf(); + // output path + taskConf.setOutputPath(getOutputPath()); + // output file pattern + taskConf.setOutputFilePattern(job.getLabel() + ".%(table)s.%(view)s.%(bucket)s"); + // tables (partitions) + Map etlPartitions = createEtlPartitions(); + Preconditions.checkNotNull(etlPartitions); + taskConf.setEtlPartitions(etlPartitions); - LoadErrorHub.Param info = load.getLoadErrorHubInfo(); - // hadoop load only support mysql load error hub - if (info != null && info.getType() == HubType.MYSQL_TYPE) { - taskConf.setHubInfo(new EtlErrorHubInfo(this.job.getId(), info)); - } + LoadErrorHub.Param info = load.getLoadErrorHubInfo(); + // hadoop load only support mysql load error hub + if (info != null && info.getType() == HubType.MYSQL_TYPE) { + taskConf.setHubInfo(new EtlErrorHubInfo(this.job.getId(), info)); + } - etlTaskConf = taskConf.toDppTaskConf(); - Preconditions.checkNotNull(etlTaskConf); + etlTaskConf = taskConf.toDppTaskConf(); + Preconditions.checkNotNull(etlTaskConf); - // add table indexes to transaction state - TransactionState txnState = Catalog.getCurrentGlobalTransactionMgr().getTransactionState(job.getDbId(), job.getTransactionId()); - if (txnState == null) { - throw new LoadException("txn does not exist: " + job.getTransactionId()); + // add table indexes to transaction state + TransactionState txnState = Catalog.getCurrentGlobalTransactionMgr().getTransactionState(job.getDbId(), job.getTransactionId()); + if (txnState == null) { + throw new LoadException("txn does not exist: " + job.getTransactionId()); + } + for (long tableId : job.getIdToTableLoadInfo().keySet()) { + OlapTable table = (OlapTable) db.getTable(tableId); + if (table == null) { + throw new LoadException("table does not exist. id: " + tableId); } - for (long tableId : job.getIdToTableLoadInfo().keySet()) { - OlapTable table = (OlapTable) db.getTable(tableId); - if (table == null) { - throw new LoadException("table does not exist. id: " + tableId); - } + table.readLock(); + try { txnState.addTableIndexes(table); + } finally { + table.readUnlock(); } - } finally { - db.readUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/LoadEtlTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/LoadEtlTask.java index c754cf48f73770..7262e2eaa5452a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/LoadEtlTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/LoadEtlTask.java @@ -160,13 +160,7 @@ private void tryUpdateLoading() { try { for (Entry tableEntry : idToTableLoadInfo.entrySet()) { long tableId = tableEntry.getKey(); - OlapTable table = null; - db.readLock(); - try { - table = (OlapTable) db.getTable(tableId); - } finally { - db.readUnlock(); - } + OlapTable table = (OlapTable) db.getTable(tableId); if (table == null) { throw new MetaNotFoundException("table does not exist. id: " + tableId); } @@ -180,7 +174,7 @@ private void tryUpdateLoading() { continue; } - db.readLock(); + table.readLock(); try { Partition partition = table.getPartition(partitionId); if (partition == null) { @@ -188,7 +182,7 @@ private void tryUpdateLoading() { } // yiguolei: real time load do not need get version here } finally { - db.readUnlock(); + table.readUnlock(); } LOG.info("load job id: {}, label: {}, partition info: {}-{}-{}, partition load info: {}", @@ -239,14 +233,13 @@ protected Map getTabletLoadInfos(Map idToTableLoadInfo = job.getIdToTableLoadInfo(); for (Entry tableEntry : idToTableLoadInfo.entrySet()) { long tableId = tableEntry.getKey(); - OlapTable table = null; - db.readLock(); - try { - table = (OlapTable) db.getTable(tableId); - if (table == null) { - throw new LoadException("table does not exist. id: " + tableId); - } + OlapTable table = (OlapTable) db.getTable(tableId); + if (table == null) { + throw new LoadException("table does not exist. id: " + tableId); + } + table.readLock(); + try { TableLoadInfo tableLoadInfo = tableEntry.getValue(); for (Entry partitionEntry : tableLoadInfo.getIdToPartitionLoadInfo().entrySet()) { long partitionId = partitionEntry.getKey(); @@ -286,13 +279,12 @@ protected Map getTabletLoadInfos(Map(backendId, createRequest(taskId))); } finally { - db.readUnlock(); + destTable.readUnlock(); } } } From 08a71144348ed496cd7aee8a9c488cd2ac7d9eba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Wed, 3 Jun 2020 17:45:55 +0800 Subject: [PATCH 03/50] use table write lock to replace db write lock --- .../java/org/apache/doris/alter/Alter.java | 43 +-- .../org/apache/doris/alter/AlterHandler.java | 13 +- .../doris/alter/MaterializedViewHandler.java | 62 ++-- .../org/apache/doris/alter/RollupJob.java | 104 +++--- .../org/apache/doris/alter/RollupJobV2.java | 54 ++-- .../doris/alter/SchemaChangeHandler.java | 30 +- .../apache/doris/alter/SchemaChangeJob.java | 121 +++---- .../apache/doris/alter/SchemaChangeJobV2.java | 28 +- .../org/apache/doris/backup/RestoreJob.java | 304 +++++++++--------- .../org/apache/doris/catalog/Database.java | 2 +- .../org/apache/doris/catalog/OlapTable.java | 2 +- .../doris/clone/ColocateTableBalancer.java | 8 +- .../clone/DynamicPartitionScheduler.java | 36 +-- .../org/apache/doris/clone/TabletChecker.java | 55 ++-- .../common/proc/EsPartitionsProcDir.java | 12 +- .../doris/common/proc/EsShardProcDir.java | 4 +- .../doris/common/proc/IndexInfoProcDir.java | 8 +- .../doris/common/proc/IndicesProcDir.java | 10 +- .../doris/common/proc/PartitionsProcDir.java | 8 +- .../doris/common/proc/StatisticProcDir.java | 58 ++-- .../doris/common/proc/TablesProcDir.java | 29 +- .../doris/common/proc/TabletsProcDir.java | 16 +- .../consistency/CheckConsistencyJob.java | 15 +- .../doris/consistency/ConsistencyChecker.java | 44 +-- .../doris/http/rest/GetDdlStmtAction.java | 15 +- .../doris/http/rest/MigrationAction.java | 58 ++-- .../http/rest/StorageTypeCheckAction.java | 23 +- .../doris/http/rest/TableQueryPlanAction.java | 29 +- .../doris/http/rest/TableSchemaAction.java | 66 ++-- .../apache/doris/load/BrokerFileGroup.java | 48 +-- .../org/apache/doris/load/DeleteHandler.java | 21 +- .../java/org/apache/doris/load/DeleteJob.java | 12 +- .../java/org/apache/doris/load/ExportJob.java | 7 +- .../main/java/org/apache/doris/load/Load.java | 114 +++---- .../org/apache/doris/load/LoadChecker.java | 33 +- .../doris/load/loadv2/BrokerLoadJob.java | 77 +++-- .../apache/doris/load/loadv2/LoadManager.java | 11 +- .../load/routineload/KafkaRoutineLoadJob.java | 2 +- .../load/routineload/RoutineLoadJob.java | 13 +- .../org/apache/doris/qe/ShowExecutor.java | 2 +- .../doris/service/FrontendServiceImpl.java | 32 +- .../doris/system/SystemInfoService.java | 14 +- .../doris/transaction/TransactionState.java | 7 +- 43 files changed, 854 insertions(+), 796 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index f0194e19ff90f0..f5f3f7ada59841 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -104,19 +104,19 @@ public void processCreateMaterializedView(CreateMaterializedViewStmt stmt) // check db quota db.checkQuota(); - db.writeLock(); + Table table = db.getTable(tableName); + if (table.getType() != TableType.OLAP) { + throw new DdlException("Do not support alter non-OLAP table[" + tableName + "]"); + } + OlapTable olapTable = (OlapTable) table; + + olapTable.writeLock(); try { - Table table = db.getTable(tableName); - if (table.getType() != TableType.OLAP) { - throw new DdlException("Do not support alter non-OLAP table[" + tableName + "]"); - } - OlapTable olapTable = (OlapTable) table; olapTable.checkStableAndNormal(db.getClusterName()); - ((MaterializedViewHandler)materializedViewHandler).processCreateMaterializedView(stmt, db, olapTable); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } @@ -128,20 +128,21 @@ public void processDropMaterializedView(DropMaterializedViewStmt stmt) throws Dd ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); } - db.writeLock(); + String tableName = stmt.getTableName().getTbl(); + Table table = db.getTable(tableName); + // if table exists + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } + // check table type + if (table.getType() != TableType.OLAP) { + throw new DdlException("Do not support non-OLAP table [" + tableName + "] when drop materialized view"); + } + + OlapTable olapTable = (OlapTable) table; + olapTable.writeLock(); try { - String tableName = stmt.getTableName().getTbl(); - Table table = db.getTable(tableName); - // if table exists - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } - // check table type - if (table.getType() != TableType.OLAP) { - throw new DdlException("Do not support non-OLAP table [" + tableName + "] when drop materialized view"); - } // check table state - OlapTable olapTable = (OlapTable) table; if (olapTable.getState() != OlapTableState.NORMAL) { throw new DdlException("Table[" + table.getName() + "]'s state is not NORMAL. " + "Do not allow doing DROP ops"); @@ -150,7 +151,7 @@ public void processDropMaterializedView(DropMaterializedViewStmt stmt) throws Dd ((MaterializedViewHandler)materializedViewHandler).processDropMaterializedView(stmt, db, olapTable); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java index 65b92463e7b36b..39dcce7f8ed9f9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java @@ -427,12 +427,13 @@ public void handleFinishAlterTask(AlterReplicaTask task) throws MetaNotFoundExce throw new MetaNotFoundException("database " + task.getDbId() + " does not exist"); } - db.writeLock(); + OlapTable tbl = (OlapTable) db.getTable(task.getTableId()); + if (tbl == null) { + throw new MetaNotFoundException("tbl " + task.getTableId() + " does not exist"); + } + + tbl.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"); @@ -478,7 +479,7 @@ public void handleFinishAlterTask(AlterReplicaTask task) throws MetaNotFoundExce LOG.info("after handle alter task tablet: {}, replica: {}", task.getSignature(), replica); } finally { - db.writeUnlock(); + tbl.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java index 2eff0999091790..8dacb6d5e23dd1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java @@ -287,7 +287,6 @@ public void processBatchAddRollup(List alterClauses, Database db, O // ATTN: This order is not mandatory, because database lock will protect us, // but this order is more reasonable olapTable.setState(OlapTableState.ROLLUP); - // 2 batch submit rollup job List rollupJobV2List = new ArrayList<>(rollupNameJobMap.values()); batchAddAlterJobV2(rollupJobV2List); @@ -700,7 +699,7 @@ private long checkAndGetBaseIndex(String baseIndexName, OlapTable olapTable) thr public void processBatchDropRollup(List dropRollupClauses, Database db, OlapTable olapTable) throws DdlException, MetaNotFoundException { - db.writeLock(); + olapTable.writeLock(); try { // check drop rollup index operation for (AlterClause alterClause : dropRollupClauses) { @@ -726,13 +725,14 @@ public void processBatchDropRollup(List dropRollupClauses, Database editLog.logBatchDropRollup(new BatchDropInfo(dbId, tableId, indexIdSet)); LOG.info("finished drop rollup index[{}] in table[{}]", String.join("", rollupNameSet), olapTable.getName()); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } public void processDropMaterializedView(DropMaterializedViewStmt dropMaterializedViewStmt, Database db, OlapTable olapTable) throws DdlException, MetaNotFoundException { - Preconditions.checkState(db.isWriteLockHeldByCurrentThread()); + Preconditions.checkState(olapTable.isWriteLockHeldByCurrentThread()); + olapTable.writeLock(); try { String mvName = dropMaterializedViewStmt.getMvName(); // Step1: check drop mv index operation @@ -807,13 +807,13 @@ private long dropMaterializedView(String mvName, OlapTable olapTable) { public void replayDropRollup(DropInfo dropInfo, Catalog catalog) { Database db = catalog.getDb(dropInfo.getDbId()); - db.writeLock(); - try { - long tableId = dropInfo.getTableId(); - long rollupIndexId = dropInfo.getIndexId(); + long tableId = dropInfo.getTableId(); + long rollupIndexId = dropInfo.getIndexId(); - TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); - OlapTable olapTable = (OlapTable) db.getTable(tableId); + TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + olapTable.writeLock(); + try { for (Partition partition : olapTable.getPartitions()) { MaterializedIndex rollupIndex = partition.deleteRollupIndex(rollupIndexId); @@ -828,7 +828,7 @@ public void replayDropRollup(DropInfo dropInfo, Catalog catalog) { String rollupIndexName = olapTable.getIndexNameById(rollupIndexId); olapTable.deleteIndexInfo(rollupIndexName); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } LOG.info("replay drop rollup {}", dropInfo.getIndexId()); } @@ -863,15 +863,18 @@ private void changeTableStatus(long dbId, long tableId, OlapTableState olapTable dbId, tableId); return; } - db.writeLock(); + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + return; + } + tbl.writeLock(); try { - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null || tbl.getState() == olapTableState) { + if (tbl.getState() == olapTableState) { return; } tbl.setState(olapTableState); } finally { - db.writeUnlock(); + tbl.writeUnlock(); } } @@ -1054,13 +1057,8 @@ private void runOldAlterJob() { continue; } - db.writeLock(); - try { - OlapTable olapTable = (OlapTable) db.getTable(rollupJob.getTableId()); - rollupJob.cancel(olapTable, "cancelled"); - } finally { - db.writeUnlock(); - } + OlapTable olapTable = (OlapTable) db.getTable(rollupJob.getTableId()); + rollupJob.cancel(olapTable, "cancelled"); jobDone(rollupJob); } @@ -1174,16 +1172,16 @@ public void cancel(CancelStmt stmt) throws DdlException { AlterJob rollupJob = null; List rollupJobV2List = new ArrayList<>(); - db.writeLock(); + Table table = db.getTable(tableName); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } + if (!(table instanceof OlapTable)) { + ErrorReport.reportDdlException(ErrorCode.ERR_NOT_OLAP_TABLE, tableName); + } + OlapTable olapTable = (OlapTable) table; + olapTable.writeLock(); try { - Table table = db.getTable(tableName); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } - if (!(table instanceof OlapTable)) { - ErrorReport.reportDdlException(ErrorCode.ERR_NOT_OLAP_TABLE, tableName); - } - OlapTable olapTable = (OlapTable) table; if (olapTable.getState() != OlapTableState.ROLLUP) { throw new DdlException("Table[" + tableName + "] is not under ROLLUP. " + "Use 'ALTER TABLE DROP ROLLUP' if you want to."); @@ -1211,7 +1209,7 @@ public void cancel(CancelStmt stmt) throws DdlException { rollupJob.cancel(olapTable, "user cancelled"); } } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } // alter job v2's cancel must be called outside the database lock diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java index b5f352ed88b8d9..c92e739bbd15d3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java @@ -496,15 +496,20 @@ public synchronized void cancel(OlapTable olapTable, String msg) { // set state if (olapTable != null) { - Preconditions.checkState(olapTable.getId() == tableId); - for (Partition partition : olapTable.getPartitions()) { - if (partition.getState() == PartitionState.ROLLUP) { - partition.setState(PartitionState.NORMAL); + olapTable.writeLock(); + try { + Preconditions.checkState(olapTable.getId() == tableId); + for (Partition partition : olapTable.getPartitions()) { + if (partition.getState() == PartitionState.ROLLUP) { + partition.setState(PartitionState.NORMAL); + } } - } - if (olapTable.getState() == OlapTableState.ROLLUP) { - olapTable.setState(OlapTableState.NORMAL); + if (olapTable.getState() == OlapTableState.ROLLUP) { + olapTable.setState(OlapTableState.NORMAL); + } + } finally { + olapTable.writeUnlock(); } } @@ -632,17 +637,17 @@ public int tryFinishJob() { return -1; } - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + cancelMsg = "Table[" + tableId + "] does not exist"; + LOG.warn(cancelMsg); + return -1; + } + + olapTable.writeLock(); try { // if all previous transaction has finished, then check base and rollup replica num synchronized (this) { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - cancelMsg = "Table[" + tableId + "] does not exist"; - LOG.warn(cancelMsg); - return -1; - } - for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { long partitionId = entry.getKey(); Partition partition = olapTable.getPartition(partitionId); @@ -761,7 +766,7 @@ public int tryFinishJob() { this.transactionId = Catalog.getCurrentGlobalTransactionMgr().getTransactionIDGenerator().getNextTransactionId(); } } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } Catalog.getCurrentCatalog().getEditLog().logFinishingRollup(this); @@ -784,11 +789,15 @@ public synchronized void clear() { @Override public void replayInitJob(Database db) { - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + return; + } + olapTable.writeLock(); try { // set state TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); - OlapTable olapTable = (OlapTable) db.getTable(tableId); + for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { Partition partition = olapTable.getPartition(entry.getKey()); partition.setState(PartitionState.ROLLUP); @@ -813,16 +822,19 @@ public void replayInitJob(Database db) { // reset status to PENDING for resending the tasks in polling thread this.state = JobState.PENDING; } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } @Override public void replayFinishing(Database db) { TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + return; + } + olapTable.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { long partitionId = entry.getKey(); MaterializedIndex rollupIndex = entry.getValue(); @@ -890,7 +902,7 @@ public void replayFinishing(Database db) { olapTable.setIndexMeta(rollupIndexId, rollupIndexName, rollupSchema, 0, rollupSchemaHash, rollupShortKeyColumnCount, rollupStorageType, KeysType.fromThrift(rollupKeysType)); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } LOG.info("replay finishing the rollup job: {}", tableId); @@ -903,28 +915,26 @@ public void replayFinish(Database db) { replayFinishing(db); } - db.writeLock(); - try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - LOG.warn("table {} could not be found when replay rollup job", tableId); - return; + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable != null) { + olapTable.writeLock(); + try { + olapTable.setState(OlapTableState.NORMAL); + } finally { + olapTable.writeUnlock(); } - olapTable.setState(OlapTableState.NORMAL); - } finally { - db.writeUnlock(); } } @Override public void replayCancel(Database db) { - db.writeLock(); - try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - return; - } + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + return; + } + olapTable.writeLock(); + try{ if (!Catalog.isCheckpointThread()) { // remove from inverted index for (MaterializedIndex rollupIndex : partitionIdToRollupIndex.values()) { @@ -940,7 +950,7 @@ public void replayCancel(Database db) { } olapTable.setState(OlapTableState.NORMAL); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } @@ -953,21 +963,20 @@ public void finishJob() { return; } - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + cancelMsg = String.format("table %d does not exist", tableId); + LOG.warn(cancelMsg); + return; + } + olapTable.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - cancelMsg = String.format("table %d does not exist", tableId); - LOG.warn(cancelMsg); - return; - } olapTable.setState(OlapTableState.NORMAL); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } List list = new ArrayList<>(); - Integer[] arr = list.toArray(new Integer[0]); this.finishedTime = System.currentTimeMillis(); LOG.info("finished rollup job: {}", tableId); @@ -1140,6 +1149,7 @@ public static RollupJob read(DataInput in) throws IOException { return rollupJob; } + @Override public boolean equals(Object obj) { return true; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java index 19230a5750f6e5..430b46b252be62 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java @@ -514,10 +514,10 @@ private void cancelInternal() { 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) { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl != null) { + tbl.writeLock(); + try { for (Long partitionId : partitionIdToRollupIndex.keySet()) { MaterializedIndex rollupIndex = partitionIdToRollupIndex.get(partitionId); for (Tablet rollupTablet : rollupIndex.getTablets()) { @@ -527,9 +527,9 @@ private void cancelInternal() { partition.deleteRollupIndex(rollupIndexId); } tbl.deleteIndexInfo(rollupIndexName); + } finally { + tbl.writeUnlock(); } - } finally { - db.writeUnlock(); } } } @@ -551,16 +551,17 @@ private void replayPending(RollupJobV2 replayedJob) { return; } - db.writeLock(); + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + // table may be dropped before replaying this log. just return + return; + } + + tbl.writeLock(); try { - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - // table may be dropped before replaying this log. just return - return; - } addTabletToInvertedIndex(tbl); } finally { - db.writeUnlock(); + tbl.writeUnlock(); } // to make sure that this job will run runPendingJob() again to create the rollup replicas @@ -599,16 +600,17 @@ private void replayWaitingTxn(RollupJobV2 replayedJob) { return; } - db.writeLock(); + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + // table may be dropped before replaying this log. just return + return; + } + + tbl.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(); + tbl.writeUnlock(); } // should still be in WAITING_TXN state, so that the alter tasks will be resend again @@ -625,15 +627,15 @@ private void replayWaitingTxn(RollupJobV2 replayedJob) { 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) { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl != null) { + tbl.writeLock(); + try { Preconditions.checkState(tbl.getState() == OlapTableState.ROLLUP); onFinished(tbl); + } finally { + tbl.writeUnlock(); } - } finally { - db.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index 2102ca866a2b6c..ae67451c9ea558 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -1487,13 +1487,8 @@ private void runOldAlterJob() { continue; } - db.writeLock(); - try { - OlapTable olapTable = (OlapTable) db.getTable(alterJob.getTableId()); - alterJob.cancel(olapTable, "cancelled"); - } finally { - db.writeUnlock(); - } + OlapTable olapTable = (OlapTable) db.getTable(alterJob.getTableId()); + alterJob.cancel(olapTable, "cancelled"); jobDone(alterJob); } @@ -1890,16 +1885,17 @@ public void cancel(CancelStmt stmt) throws DdlException { AlterJob schemaChangeJob = null; AlterJobV2 schemaChangeJobV2 = null; - db.writeLock(); + + Table table = db.getTable(tableName); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } + if (!(table instanceof OlapTable)) { + ErrorReport.reportDdlException(ErrorCode.ERR_NOT_OLAP_TABLE, tableName); + } + OlapTable olapTable = (OlapTable) table; + olapTable.writeLock(); try { - Table table = db.getTable(tableName); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } - if (!(table instanceof OlapTable)) { - ErrorReport.reportDdlException(ErrorCode.ERR_NOT_OLAP_TABLE, tableName); - } - OlapTable olapTable = (OlapTable) table; if (olapTable.getState() != OlapTableState.SCHEMA_CHANGE) { throw new DdlException("Table[" + tableName + "] is not under SCHEMA_CHANGE."); } @@ -1919,7 +1915,7 @@ public void cancel(CancelStmt stmt) throws DdlException { schemaChangeJob.cancel(olapTable, "user cancelled"); } } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } // alter job v2's cancel must be called outside the database lock diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java index e9b995a5b1ef21..d41c149dc723c3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java @@ -499,41 +499,45 @@ public boolean sendTasks() { @Override public synchronized void cancel(OlapTable olapTable, String msg) { // make sure to get db write lock before calling this - if (olapTable != null) { - // 1. remove all task and set state - for (Partition partition : olapTable.getPartitions()) { - if (partition.getState() == PartitionState.NORMAL) { - continue; - } - long partitionId = partition.getId(); - for (Long indexId : this.changedIndexIdToSchema.keySet()) { - MaterializedIndex index = partition.getIndex(indexId); - if (index == null || index.getState() == IndexState.NORMAL) { + olapTable.writeLock(); + try { + // 1. remove all task and set state + for (Partition partition : olapTable.getPartitions()) { + if (partition.getState() == PartitionState.NORMAL) { continue; } - for (Tablet tablet : index.getTablets()) { - long tabletId = tablet.getId(); - for (Replica replica : tablet.getReplicas()) { - if (replica.getState() == ReplicaState.CLONE - || replica.getState() == ReplicaState.DECOMMISSION - || replica.getState() == ReplicaState.NORMAL) { - continue; - } - Preconditions.checkState(replica.getState() == ReplicaState.SCHEMA_CHANGE); - replica.setState(ReplicaState.NORMAL); - AgentTaskQueue.removeTask(replica.getBackendId(), TTaskType.SCHEMA_CHANGE, tabletId); - } // end for replicas - } // end for tablets + long partitionId = partition.getId(); + for (Long indexId : this.changedIndexIdToSchema.keySet()) { + MaterializedIndex index = partition.getIndex(indexId); + if (index == null || index.getState() == IndexState.NORMAL) { + continue; + } + for (Tablet tablet : index.getTablets()) { + long tabletId = tablet.getId(); + for (Replica replica : tablet.getReplicas()) { + if (replica.getState() == ReplicaState.CLONE + || replica.getState() == ReplicaState.DECOMMISSION + || replica.getState() == ReplicaState.NORMAL) { + continue; + } + Preconditions.checkState(replica.getState() == ReplicaState.SCHEMA_CHANGE); + replica.setState(ReplicaState.NORMAL); + AgentTaskQueue.removeTask(replica.getBackendId(), TTaskType.SCHEMA_CHANGE, tabletId); + } // end for replicas + } // end for tablets - // delete schema hash in inverted index - Catalog.getCurrentInvertedIndex().deleteNewSchemaHash(partitionId, indexId); - Preconditions.checkArgument(index.getState() == IndexState.SCHEMA_CHANGE); - index.setState(IndexState.NORMAL); - } // end for indices - partition.setState(PartitionState.NORMAL); - } // end for partitions - olapTable.setState(OlapTableState.NORMAL); + // delete schema hash in inverted index + Catalog.getCurrentInvertedIndex().deleteNewSchemaHash(partitionId, indexId); + Preconditions.checkArgument(index.getState() == IndexState.SCHEMA_CHANGE); + index.setState(IndexState.NORMAL); + } // end for indices + partition.setState(PartitionState.NORMAL); + } // end for partitions + olapTable.setState(OlapTableState.NORMAL); + } finally { + olapTable.writeUnlock(); + } } this.state = JobState.CANCELLED; @@ -880,17 +884,17 @@ public void finishJob() { return; } - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + cancelMsg = String.format("table %d does not exist", tableId); + LOG.warn(cancelMsg); + return; + } + olapTable.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - cancelMsg = String.format("table %d does not exist", tableId); - LOG.warn(cancelMsg); - return; - } olapTable.setState(OlapTableState.NORMAL); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } this.finishedTime = System.currentTimeMillis(); @@ -910,10 +914,12 @@ public synchronized void clear() { @Override public void replayInitJob(Database db) { - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + return; + } + olapTable.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - // change the state of table/partition and replica, then add object to related List and Set for (Partition partition : olapTable.getPartitions()) { for (Map.Entry entry : changedIndexIdToSchemaHash.entrySet()) { @@ -947,7 +953,7 @@ public void replayInitJob(Database db) { // reset status to PENDING for resending the tasks in polling thread this.state = JobState.PENDING; } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } @@ -1035,28 +1041,27 @@ public void replayFinish(Database db) { replayFinishing(db); } - db.writeLock(); - try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable != null) { + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable != null) { + olapTable.writeLock(); + try { olapTable.setState(OlapTableState.NORMAL); + } finally { + olapTable.writeUnlock(); } - } finally { - db.writeUnlock(); } - LOG.info("replay finish schema change job: {}", tableId); } @Override public void replayCancel(Database db) { - db.writeLock(); + // restore partition's state + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + return; + } + olapTable.writeLock(); try { - // restore partition's state - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - return; - } for (Partition partition : olapTable.getPartitions()) { long partitionId = partition.getId(); for (Long indexId : this.changedIndexIdToSchema.keySet()) { @@ -1083,13 +1088,13 @@ public void replayCancel(Database db) { } // end for indices Preconditions.checkState(partition.getState() == PartitionState.SCHEMA_CHANGE, - partition.getState()); + partition.getState()); partition.setState(PartitionState.NORMAL); } // end for partitions olapTable.setState(OlapTableState.NORMAL); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java index 8bcd16497a2840..b84e3f17a8e77e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java @@ -622,10 +622,10 @@ private void cancelInternal() { 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) { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl != null) { + tbl.writeLock(); + try { for (long partitionId : partitionIndexMap.rowKeySet()) { Partition partition = tbl.getPartition(partitionId); Preconditions.checkNotNull(partition, partitionId); @@ -643,9 +643,9 @@ private void cancelInternal() { tbl.deleteIndexInfo(shadowIndexName); } tbl.setState(OlapTableState.NORMAL); + } finally { + tbl.writeUnlock(); } - } finally { - db.writeUnlock(); } } @@ -669,14 +669,14 @@ private void replayPending(SchemaChangeJobV2 replayedJob) { return; } - db.writeLock(); + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + // table may be dropped before replaying this log. just return + return; + } + + tbl.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(); @@ -698,7 +698,7 @@ private void replayPending(SchemaChangeJobV2 replayedJob) { // set table state tbl.setState(OlapTableState.SCHEMA_CHANGE); } finally { - db.writeUnlock(); + tbl.writeUnlock(); } this.watershedTxnId = replayedJob.watershedTxnId; diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java index 4fb45f4afde68e..dd7162beb66963 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -363,17 +363,21 @@ private void checkIfNeedCancel() { status = new Status(ErrCode.NOT_FOUND, "table " + idChain.getTblId() + " has been dropped"); return; } + tbl.readLock(); + try { + Partition part = tbl.getPartition(idChain.getPartId()); + if (part == null) { + status = new Status(ErrCode.NOT_FOUND, "partition " + idChain.getPartId() + " has been dropped"); + return; + } - Partition part = tbl.getPartition(idChain.getPartId()); - if (part == null) { - status = new Status(ErrCode.NOT_FOUND, "partition " + idChain.getPartId() + " has been dropped"); - return; - } - - MaterializedIndex index = part.getIndex(idChain.getIdxId()); - if (index == null) { - status = new Status(ErrCode.NOT_FOUND, "index " + idChain.getIdxId() + " has been dropped"); - return; + MaterializedIndex index = part.getIndex(idChain.getIdxId()); + if (index == null) { + status = new Status(ErrCode.NOT_FOUND, "index " + idChain.getIdxId() + " has been dropped"); + return; + } + } finally { + tbl.readUnlock(); } } } finally { @@ -477,88 +481,92 @@ private void checkAndPrepareMeta() { OlapTable localOlapTbl = (OlapTable) localTbl; OlapTable remoteOlapTbl = (OlapTable) remoteTbl; - List intersectPartNames = Lists.newArrayList(); - Status st = localOlapTbl.getIntersectPartNamesWith(remoteOlapTbl, intersectPartNames); - if (!st.ok()) { - status = st; - return; - } - LOG.debug("get intersect part names: {}, job: {}", intersectPartNames, this); - if (localOlapTbl.getSignature(BackupHandler.SIGNATURE_VERSION, intersectPartNames) - != remoteOlapTbl.getSignature(BackupHandler.SIGNATURE_VERSION, intersectPartNames)) { - status = new Status(ErrCode.COMMON_ERROR, "Table " + jobInfo.getAliasByOriginNameIfSet(tblInfo.name) - + " already exist but with different schema"); - return; - } + localOlapTbl.readLock(); + try { + List intersectPartNames = Lists.newArrayList(); + Status st = localOlapTbl.getIntersectPartNamesWith(remoteOlapTbl, intersectPartNames); + if (!st.ok()) { + status = st; + return; + } + LOG.debug("get intersect part names: {}, job: {}", intersectPartNames, this); + if (localOlapTbl.getSignature(BackupHandler.SIGNATURE_VERSION, intersectPartNames) + != remoteOlapTbl.getSignature(BackupHandler.SIGNATURE_VERSION, intersectPartNames)) { + status = new Status(ErrCode.COMMON_ERROR, "Table " + jobInfo.getAliasByOriginNameIfSet(tblInfo.name) + + " already exist but with different schema"); + return; + } - // Table with same name and has same schema. Check partition - for (BackupPartitionInfo backupPartInfo : tblInfo.partitions.values()) { - Partition localPartition = localOlapTbl.getPartition(backupPartInfo.name); - if (localPartition != null) { - // Partition already exist. - PartitionInfo localPartInfo = localOlapTbl.getPartitionInfo(); - if (localPartInfo.getType() == PartitionType.RANGE) { - // If this is a range partition, check range - RangePartitionInfo localRangePartInfo = (RangePartitionInfo) localPartInfo; - RangePartitionInfo remoteRangePartInfo - = (RangePartitionInfo) remoteOlapTbl.getPartitionInfo(); - Range localRange = localRangePartInfo.getRange(localPartition.getId()); - Range remoteRange = remoteRangePartInfo.getRange(backupPartInfo.id); - if (localRange.equals(remoteRange)) { - // Same partition, same range - if (genFileMappingWhenBackupReplicasEqual(localPartInfo, localPartition, localTbl, backupPartInfo, tblInfo)) { + // Table with same name and has same schema. Check partition + for (BackupPartitionInfo backupPartInfo : tblInfo.partitions.values()) { + Partition localPartition = localOlapTbl.getPartition(backupPartInfo.name); + if (localPartition != null) { + // Partition already exist. + PartitionInfo localPartInfo = localOlapTbl.getPartitionInfo(); + if (localPartInfo.getType() == PartitionType.RANGE) { + // If this is a range partition, check range + RangePartitionInfo localRangePartInfo = (RangePartitionInfo) localPartInfo; + RangePartitionInfo remoteRangePartInfo + = (RangePartitionInfo) remoteOlapTbl.getPartitionInfo(); + Range localRange = localRangePartInfo.getRange(localPartition.getId()); + Range remoteRange = remoteRangePartInfo.getRange(backupPartInfo.id); + if (localRange.equals(remoteRange)) { + // Same partition, same range + if (genFileMappingWhenBackupReplicasEqual(localPartInfo, localPartition, localTbl, backupPartInfo, tblInfo)) { + return; + } + } else { + // Same partition name, different range + status = new Status(ErrCode.COMMON_ERROR, "Partition " + backupPartInfo.name + + " in table " + localTbl.getName() + + " has different range with partition in repository"); return; } } else { - // Same partition name, different range - status = new Status(ErrCode.COMMON_ERROR, "Partition " + backupPartInfo.name - + " in table " + localTbl.getName() - + " has different range with partition in repository"); - return; + // If this is a single partitioned table. + if (genFileMappingWhenBackupReplicasEqual(localPartInfo, localPartition, localTbl, backupPartInfo, tblInfo)) { + return; + } } } else { - // If this is a single partitioned table. - if (genFileMappingWhenBackupReplicasEqual(localPartInfo, localPartition, localTbl, backupPartInfo, tblInfo)) { - return; - } - } - } else { - // partitions does not exist - PartitionInfo localPartitionInfo = localOlapTbl.getPartitionInfo(); - if (localPartitionInfo.getType() == PartitionType.RANGE) { - // Check if the partition range can be added to the table - RangePartitionInfo localRangePartitionInfo = (RangePartitionInfo) localPartitionInfo; - RangePartitionInfo remoteRangePartitionInfo - = (RangePartitionInfo) remoteOlapTbl.getPartitionInfo(); - Range remoteRange = remoteRangePartitionInfo.getRange(backupPartInfo.id); - if (localRangePartitionInfo.getAnyIntersectRange(remoteRange, false) != null) { - status = new Status(ErrCode.COMMON_ERROR, "Partition " + backupPartInfo.name - + " in table " + localTbl.getName() - + " has conflict range with existing ranges"); - return; - } else { - // this partition can be added to this table, set ids - Partition restorePart = resetPartitionForRestore(localOlapTbl, remoteOlapTbl, - backupPartInfo.name, - db.getClusterName(), - restoreReplicationNum); - if (restorePart == null) { + // partitions does not exist + PartitionInfo localPartitionInfo = localOlapTbl.getPartitionInfo(); + if (localPartitionInfo.getType() == PartitionType.RANGE) { + // Check if the partition range can be added to the table + RangePartitionInfo localRangePartitionInfo = (RangePartitionInfo) localPartitionInfo; + RangePartitionInfo remoteRangePartitionInfo + = (RangePartitionInfo) remoteOlapTbl.getPartitionInfo(); + Range remoteRange = remoteRangePartitionInfo.getRange(backupPartInfo.id); + if (localRangePartitionInfo.getAnyIntersectRange(remoteRange, false) != null) { + status = new Status(ErrCode.COMMON_ERROR, "Partition " + backupPartInfo.name + + " in table " + localTbl.getName() + + " has conflict range with existing ranges"); return; + } else { + // this partition can be added to this table, set ids + Partition restorePart = resetPartitionForRestore(localOlapTbl, remoteOlapTbl, + backupPartInfo.name, + db.getClusterName(), + restoreReplicationNum); + if (restorePart == null) { + return; + } + restoredPartitions.add(Pair.create(localOlapTbl.getName(), restorePart)); } - restoredPartitions.add(Pair.create(localOlapTbl.getName(), restorePart)); + } else { + // It is impossible that a single partitioned table exist without any existing partition + status = new Status(ErrCode.COMMON_ERROR, + "No partition exist in single partitioned table " + localOlapTbl.getName()); + return; } - } else { - // It is impossible that a single partitioned table exist without any existing partition - status = new Status(ErrCode.COMMON_ERROR, - "No partition exist in single partitioned table " + localOlapTbl.getName()); - return; } } + } finally { + localOlapTbl.readUnlock(); } } else { // Table does not exist OlapTable remoteOlapTbl = (OlapTable) remoteTbl; - // Retain only expected restore partitions in this table; Set allPartNames = remoteOlapTbl.getPartitionNames(); for (String partName : allPartNames) { @@ -702,20 +710,25 @@ private void checkAndPrepareMeta() { try { for (IdChain idChain : fileMapping.getMapping().keySet()) { OlapTable tbl = (OlapTable) db.getTable(idChain.getTblId()); - Partition part = tbl.getPartition(idChain.getPartId()); - MaterializedIndex index = part.getIndex(idChain.getIdxId()); - Tablet tablet = index.getTablet(idChain.getTabletId()); - Replica replica = tablet.getReplicaById(idChain.getReplicaId()); - long signature = catalog.getNextId(); - SnapshotTask task = new SnapshotTask(null, replica.getBackendId(), signature, - jobId, db.getId(), - tbl.getId(), part.getId(), index.getId(), tablet.getId(), - part.getVisibleVersion(), part.getVisibleVersionHash(), - tbl.getSchemaHashByIndexId(index.getId()), timeoutMs, - true /* is restore task*/); - batchTask.addTask(task); - unfinishedSignatureToId.put(signature, tablet.getId()); - bePathsMap.put(replica.getBackendId(), replica.getPathHash()); + tbl.readLock(); + try { + Partition part = tbl.getPartition(idChain.getPartId()); + MaterializedIndex index = part.getIndex(idChain.getIdxId()); + Tablet tablet = index.getTablet(idChain.getTabletId()); + Replica replica = tablet.getReplicaById(idChain.getReplicaId()); + long signature = catalog.getNextId(); + SnapshotTask task = new SnapshotTask(null, replica.getBackendId(), signature, + jobId, db.getId(), + tbl.getId(), part.getId(), index.getId(), tablet.getId(), + part.getVisibleVersion(), part.getVisibleVersionHash(), + tbl.getSchemaHashByIndexId(index.getId()), timeoutMs, + true /* is restore task*/); + batchTask.addTask(task); + unfinishedSignatureToId.put(signature, tablet.getId()); + bePathsMap.put(replica.getBackendId(), replica.getPathHash()); + } finally { + tbl.readUnlock(); + } } } finally { db.readUnlock(); @@ -1037,62 +1050,66 @@ private void downloadSnapshots() { return; } OlapTable olapTbl = (OlapTable) tbl; + olapTbl.readLock(); + try { + Partition part = olapTbl.getPartition(info.getPartitionId()); + if (part == null) { + status = new Status(ErrCode.NOT_FOUND, "partition " + + info.getPartitionId() + " does not exist in restored table: " + + tbl.getName()); + return; + } - Partition part = olapTbl.getPartition(info.getPartitionId()); - if (part == null) { - status = new Status(ErrCode.NOT_FOUND, "partition " - + info.getPartitionId() + " does not exist in restored table: " - + tbl.getName()); - return; - } + MaterializedIndex idx = part.getIndex(info.getIndexId()); + if (idx == null) { + status = new Status(ErrCode.NOT_FOUND, + "index " + info.getIndexId() + " does not exist in partion " + part.getName() + + "of restored table " + tbl.getName()); + return; + } - MaterializedIndex idx = part.getIndex(info.getIndexId()); - if (idx == null) { - status = new Status(ErrCode.NOT_FOUND, - "index " + info.getIndexId() + " does not exist in partion " + part.getName() - + "of restored table " + tbl.getName()); - return; - } - - Tablet tablet = idx.getTablet(info.getTabletId()); - if (tablet == null) { - status = new Status(ErrCode.NOT_FOUND, - "tablet " + info.getTabletId() + " does not exist in restored table " - + tbl.getName()); - return; - } - - Replica replica = tablet.getReplicaByBackendId(info.getBeId()); - if (replica == null) { - status = new Status(ErrCode.NOT_FOUND, - "replica in be " + info.getBeId() + " of tablet " - + tablet.getId() + " does not exist in restored table " - + tbl.getName()); - return; - } + Tablet tablet = idx.getTablet(info.getTabletId()); + if (tablet == null) { + status = new Status(ErrCode.NOT_FOUND, + "tablet " + info.getTabletId() + " does not exist in restored table " + + tbl.getName()); + return; + } - IdChain catalogIds = new IdChain(tbl.getId(), part.getId(), idx.getId(), - info.getTabletId(), replica.getId()); - IdChain repoIds = fileMapping.get(catalogIds); - if (repoIds == null) { - status = new Status(ErrCode.NOT_FOUND, - "failed to get id mapping of catalog ids: " + catalogIds.toString()); - LOG.info("current file mapping: {}", fileMapping); - return; - } + Replica replica = tablet.getReplicaByBackendId(info.getBeId()); + if (replica == null) { + status = new Status(ErrCode.NOT_FOUND, + "replica in be " + info.getBeId() + " of tablet " + + tablet.getId() + " does not exist in restored table " + + tbl.getName()); + return; + } - String repoTabletPath = jobInfo.getFilePath(repoIds); - - // eg: - // bos://location/__palo_repository_my_repo/_ss_my_ss/_ss_content/__db_10000/ - // __tbl_10001/__part_10002/_idx_10001/__10003 - String src = repo.getRepoPath(label, repoTabletPath); - SnapshotInfo snapshotInfo = snapshotInfos.get(info.getTabletId(), info.getBeId()); - Preconditions.checkNotNull(snapshotInfo, info.getTabletId() + "-" + info.getBeId()); - // download to previous exist snapshot dir - String dest = snapshotInfo.getTabletPath(); - srcToDest.put(src, dest); - LOG.debug("create download src path: {}, dest path: {}", src, dest); + IdChain catalogIds = new IdChain(tbl.getId(), part.getId(), idx.getId(), + info.getTabletId(), replica.getId()); + IdChain repoIds = fileMapping.get(catalogIds); + if (repoIds == null) { + status = new Status(ErrCode.NOT_FOUND, + "failed to get id mapping of catalog ids: " + catalogIds.toString()); + LOG.info("current file mapping: {}", fileMapping); + return; + } + + String repoTabletPath = jobInfo.getFilePath(repoIds); + // eg: + // bos://location/__palo_repository_my_repo/_ss_my_ss/_ss_content/__db_10000/ + // __tbl_10001/__part_10002/_idx_10001/__10003 + String src = repo.getRepoPath(label, repoTabletPath); + SnapshotInfo snapshotInfo = snapshotInfos.get(info.getTabletId(), info.getBeId()); + Preconditions.checkNotNull(snapshotInfo, info.getTabletId() + "-" + info.getBeId()); + // download to previous exist snapshot dir + String dest = snapshotInfo.getTabletPath(); + srcToDest.put(src, dest); + LOG.debug("create download src path: {}, dest path: {}", src, dest); + + } finally { + olapTbl.readUnlock(); + } } long signature = catalog.getNextId(); DownloadTask task = new DownloadTask(null, beId, signature, jobId, dbId, @@ -1116,7 +1133,6 @@ private void downloadSnapshots() { // No edit log here LOG.info("finished to send download tasks to BE. num: {}. {}", batchTask.getTaskNum(), this); - return; } private void waitingAllDownloadFinished() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index f30da1d9fdef2f..3e33e150e0cdfe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -414,7 +414,7 @@ public Table getTableOrThrowException(String tableName, TableType tableType) thr throw new MetaNotFoundException("unknown table, table=" + tableName); } if (table.getType() != tableType) { - throw new UserException("table type is not " + tableType + ", type=" + table.getClass()); + throw new UserException("table type is not " + tableType + ", table=" + tableName + "type=" + table.getClass()); } return table; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index bc330e7b1112ec..d2e94dfee6d0c1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -101,7 +101,7 @@ public enum OlapTableState { WAITING_STABLE } - private OlapTableState state; + private volatile OlapTableState state; // index id -> index meta private Map indexIdToMeta = Maps.newHashMap(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java b/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java index 78d598ca83c42e..e7af2d2ab3c311 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java @@ -251,8 +251,8 @@ private void matchGroup() { } else { colocateIndex.markGroupUnstable(groupId, true); } - } - } // end for groups + } // end for groups + } /* * The balance logic is as follow: @@ -416,7 +416,7 @@ private boolean relocateAndBalance(GroupId groupId, Set unavailableBeIds, // change the backend id to backend host // return null if some of backends do not exist private List> getHostsPerBucketSeq(List> backendsPerBucketSeq, - SystemInfoService infoService) { + SystemInfoService infoService) { List> hostsPerBucketSeq = Lists.newArrayList(); for (List backendIds : backendsPerBucketSeq) { List hosts = Lists.newArrayList(); @@ -434,7 +434,7 @@ private List> getHostsPerBucketSeq(List> backendsPerBuck } private List> getSortedBackendReplicaNumPairs(List allAvailBackendIds, Set unavailBackendIds, - ClusterLoadStatistic statistic, List flatBackendsPerBucketSeq) { + ClusterLoadStatistic statistic, List flatBackendsPerBucketSeq) { // backend id -> replica num, and sorted by replica num, descending. Map backendToReplicaNum = flatBackendsPerBucketSeq.stream() .collect(Collectors.groupingBy(Function.identity(), Collectors.counting())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java index 49d65ed0df2470..070ad2e77648d0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java @@ -265,17 +265,16 @@ private void executeDynamicPartition() { String tableName; boolean skipAddPartition = false; OlapTable olapTable; - db.readLock(); + olapTable = (OlapTable) db.getTable(tableId); + // Only OlapTable has DynamicPartitionProperty + if (olapTable == null + || !olapTable.dynamicPartitionExists() + || !olapTable.getTableProperty().getDynamicPartitionProperty().getEnable()) { + iterator.remove(); + continue; + } + olapTable.readLock(); try { - olapTable = (OlapTable) db.getTable(tableId); - // Only OlapTable has DynamicPartitionProperty - if (olapTable == null - || !olapTable.dynamicPartitionExists() - || !olapTable.getTableProperty().getDynamicPartitionProperty().getEnable()) { - iterator.remove(); - continue; - } - if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { String errorMsg = "Table[" + olapTable.getName() + "]'s state is not NORMAL." + "Do not allow doing dynamic add partition. table state=" + olapTable.getState(); @@ -310,18 +309,18 @@ private void executeDynamicPartition() { dropPartitionClauses = getDropPartitionClause(db, olapTable, partitionColumn, partitionFormat); tableName = olapTable.getName(); } finally { - db.readUnlock(); + olapTable.readUnlock(); } for (DropPartitionClause dropPartitionClause : dropPartitionClauses) { - db.writeLock(); + olapTable.writeLock(); try { Catalog.getCurrentCatalog().dropPartition(db, olapTable, dropPartitionClause); clearDropPartitionFailedMsg(tableName); } catch (DdlException e) { recordDropPartitionFailedMsg(db.getFullName(), tableName, e.getMessage()); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } @@ -366,15 +365,16 @@ private void initDynamicPartitionTable() { if (db == null) { continue; } - db.readLock(); - try { - for (Table table : Catalog.getCurrentCatalog().getDb(dbId).getTables()) { + List
tableList = db.getTables(); + for (Table table : tableList) { + table.readLock(); + try { if (DynamicPartitionUtil.isDynamicPartitionTable(table)) { registerDynamicPartitionTable(db.getId(), table.getId()); } + } finally { + table.readUnlock(); } - } finally { - db.readUnlock(); } } initialize = true; diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java index e97eff8be58a90..922f308af3336f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java @@ -204,10 +204,20 @@ private void checkTablets() { continue; } + int availableBackendsNum = infoService.getClusterBackendIds(db.getClusterName(), true).size(); db.readLock(); + List
tableList = null; + List aliveBeIdsInCluster = null; try { - List aliveBeIdsInCluster = infoService.getClusterBackendIds(db.getClusterName(), true); - for (Table table : db.getTables()) { + aliveBeIdsInCluster = infoService.getClusterBackendIds(db.getClusterName(), true); + tableList = db.getTables(); + } finally { + db.readUnlock(); + } + + for (Table table : tableList) { + table.readLock(); + try { if (!table.needSchedule()) { continue; } @@ -227,12 +237,12 @@ private void checkTablets() { for (MaterializedIndex idx : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { for (Tablet tablet : idx.getTablets()) { totalTabletNum++; - + if (tabletScheduler.containsTablet(tablet.getId())) { tabletInScheduler++; continue; } - + Pair statusWithPrio = tablet.getHealthStatusWithPriority( infoService, db.getClusterName(), @@ -287,10 +297,10 @@ private void checkTablets() { olapTbl.getId(), Lists.newArrayList(partition.getId()))); } } // partitions - } // tables - } finally { - db.readUnlock(); - } + } finally { + table.readUnlock(); + } + } // tables } // end for dbs long cost = System.currentTimeMillis() - start; @@ -343,12 +353,16 @@ private void removePriosIfNecessary() { deletedPrios.add(Pair.create(dbId, tblId)); continue; } - - Set parts = tblEntry.getValue(); - parts = parts.stream().filter(p -> (tbl.getPartition(p.partId) != null && !p.isTimeout())).collect( - Collectors.toSet()); - if (parts.isEmpty()) { - deletedPrios.add(Pair.create(dbId, tblId)); + tbl.readLock(); + try { + Set parts = tblEntry.getValue(); + parts = parts.stream().filter(p -> (tbl.getPartition(p.partId) != null && !p.isTimeout())).collect( + Collectors.toSet()); + if (parts.isEmpty()) { + deletedPrios.add(Pair.create(dbId, tblId)); + } + } finally { + tbl.readUnlock(); } } @@ -423,13 +437,12 @@ public static RepairTabletInfo getRepairTabletInfo(String dbName, String tblName long dbId = db.getId(); long tblId = -1; List partIds = Lists.newArrayList(); - db.readLock(); + Table tbl = db.getTable(tblName); + if (tbl == null || tbl.getType() != TableType.OLAP) { + throw new DdlException("Table does not exist or is not OLAP table: " + tblName); + } + tbl.readLock(); try { - Table tbl = db.getTable(tblName); - if (tbl == null || tbl.getType() != TableType.OLAP) { - throw new DdlException("Table does not exist or is not OLAP table: " + tblName); - } - tblId = tbl.getId(); OlapTable olapTable = (OlapTable) tbl; @@ -445,7 +458,7 @@ public static RepairTabletInfo getRepairTabletInfo(String dbName, String tblName } } } finally { - db.readUnlock(); + tbl.readUnlock(); } Preconditions.checkState(tblId != -1); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/EsPartitionsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/EsPartitionsProcDir.java index 5b470e991ec3ff..18c3fb8e757d13 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/EsPartitionsProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/EsPartitionsProcDir.java @@ -62,7 +62,7 @@ public ProcResult fetchResult() throws AnalysisException { // get info List> partitionInfos = new ArrayList>(); - db.readLock(); + esTable.readLock(); try { RangePartitionInfo rangePartitionInfo = null; if (esTable.getPartitionInfo().getType() == PartitionType.RANGE) { @@ -97,7 +97,7 @@ public ProcResult fetchResult() throws AnalysisException { partitionInfos.add(partitionInfo); } } finally { - db.readUnlock(); + esTable.readUnlock(); } // set result @@ -121,13 +121,7 @@ public boolean register(String name, ProcNodeInterface node) { @Override public ProcNodeInterface lookup(String indexName) throws AnalysisException { - - db.readLock(); - try { - return new EsShardProcDir(db, esTable, indexName); - } finally { - db.readUnlock(); - } + return new EsShardProcDir(db, esTable, indexName); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/EsShardProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/EsShardProcDir.java index 45e34ffaea7bd0..4c748831fe39ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/EsShardProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/EsShardProcDir.java @@ -52,7 +52,7 @@ public ProcResult fetchResult() { Preconditions.checkNotNull(indexName); List> shardInfos = new ArrayList>(); - db.readLock(); + esTable.readLock(); try { // get infos EsShardPartitions esShardPartitions = esTable.getEsTablePartitions().getEsShardPartitions(indexName); @@ -75,7 +75,7 @@ public ProcResult fetchResult() { } } } finally { - db.readUnlock(); + esTable.readUnlock(); } // sort by tabletId, replicaId diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/IndexInfoProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/IndexInfoProcDir.java index cd84f6a14092f3..f9583e422a27d4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/IndexInfoProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/IndexInfoProcDir.java @@ -58,7 +58,7 @@ public ProcResult fetchResult() throws AnalysisException { BaseProcResult result = new BaseProcResult(); result.setNames(TITLE_NAMES); - db.readLock(); + table.readLock(); try { if (table.getType() == TableType.OLAP) { OlapTable olapTable = (OlapTable) table; @@ -97,7 +97,7 @@ public ProcResult fetchResult() throws AnalysisException { return result; } finally { - db.readUnlock(); + table.readUnlock(); } } @@ -118,7 +118,7 @@ public ProcNodeInterface lookup(String idxIdStr) throws AnalysisException { throw new AnalysisException("Invalid index id format: " + idxIdStr); } - db.readLock(); + table.readLock(); try { List schema = null; Set bfColumns = null; @@ -134,7 +134,7 @@ public ProcNodeInterface lookup(String idxIdStr) throws AnalysisException { } return new IndexSchemaProcNode(schema, bfColumns); } finally { - db.readUnlock(); + table.readUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/IndicesProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/IndicesProcDir.java index 63bf96d2f0c7be..9116ea66b951ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/IndicesProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/IndicesProcDir.java @@ -61,7 +61,7 @@ public ProcResult fetchResult() throws AnalysisException { BaseProcResult result = new BaseProcResult(); // get info List> indexInfos = new ArrayList>(); - db.readLock(); + olapTable.readLock(); try { result.setNames(TITLE_NAMES); for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { @@ -75,7 +75,7 @@ public ProcResult fetchResult() throws AnalysisException { } } finally { - db.readUnlock(); + olapTable.readUnlock(); } // sort by index id @@ -113,15 +113,15 @@ public ProcNodeInterface lookup(String indexIdStr) throws AnalysisException { throw new AnalysisException("Invalid index id format: " + indexIdStr); } - db.readLock(); + olapTable.readLock(); try { MaterializedIndex materializedIndex = partition.getIndex(indexId); if (materializedIndex == null) { throw new AnalysisException("Index[" + indexId + "] does not exist."); } - return new TabletsProcDir(db, materializedIndex); + return new TabletsProcDir(olapTable, materializedIndex); } finally { - db.readUnlock(); + olapTable.readUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java index be0287833ff60b..5bd62279fed1a1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java @@ -207,7 +207,7 @@ private List> getPartitionInfos() { // get info List> partitionInfos = new ArrayList>(); - db.readLock(); + olapTable.readLock(); try { List partitionIds; PartitionInfo tblPartitionInfo = olapTable.getPartitionInfo(); @@ -288,7 +288,7 @@ private List> getPartitionInfos() { partitionInfos.add(partitionInfo); } } finally { - db.readUnlock(); + olapTable.readUnlock(); } return partitionInfos; } @@ -313,7 +313,7 @@ public ProcNodeInterface lookup(String partitionIdStr) throws AnalysisException throw new AnalysisException("Invalid partition id format: " + partitionIdStr); } - db.readLock(); + olapTable.readLock(); try { Partition partition = olapTable.getPartition(partitionId); if (partition == null) { @@ -322,7 +322,7 @@ public ProcNodeInterface lookup(String partitionIdStr) throws AnalysisException return new IndicesProcDir(db, olapTable, partition); } finally { - db.readUnlock(); + olapTable.readUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java index 29f48c7dbe4f5a..001f00c00c1a9c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java @@ -122,33 +122,37 @@ public ProcResult fetchResult() throws AnalysisException { ++dbTableNum; OlapTable olapTable = (OlapTable) table; - - for (Partition partition : olapTable.getAllPartitions()) { - short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId()); - ++dbPartitionNum; - for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { - ++dbIndexNum; - for (Tablet tablet : materializedIndex.getTablets()) { - ++dbTabletNum; - dbReplicaNum += tablet.getReplicas().size(); - - Pair res = tablet.getHealthStatusWithPriority( - infoService, db.getClusterName(), - partition.getVisibleVersion(), partition.getVisibleVersionHash(), - replicationNum, aliveBeIdsInCluster); - - // here we treat REDUNDANT as HEALTHY, for user friendly. - if (res.first != TabletStatus.HEALTHY && res.first != TabletStatus.REDUNDANT - && res.first != TabletStatus.COLOCATE_REDUNDANT && res.first != TabletStatus.NEED_FURTHER_REPAIR) { - unhealthyTabletIds.put(dbId, tablet.getId()); - } - - if (!tablet.isConsistent()) { - inconsistentTabletIds.put(dbId, tablet.getId()); - } - } // end for tablets - } // end for indices - } // end for partitions + table.readLock(); + try { + for (Partition partition : olapTable.getAllPartitions()) { + short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId()); + ++dbPartitionNum; + for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { + ++dbIndexNum; + for (Tablet tablet : materializedIndex.getTablets()) { + ++dbTabletNum; + dbReplicaNum += tablet.getReplicas().size(); + + Pair res = tablet.getHealthStatusWithPriority( + infoService, db.getClusterName(), + partition.getVisibleVersion(), partition.getVisibleVersionHash(), + replicationNum, aliveBeIdsInCluster); + + // here we treat REDUNDANT as HEALTHY, for user friendly. + if (res.first != TabletStatus.HEALTHY && res.first != TabletStatus.REDUNDANT + && res.first != TabletStatus.COLOCATE_REDUNDANT && res.first != TabletStatus.NEED_FURTHER_REPAIR) { + unhealthyTabletIds.put(dbId, tablet.getId()); + } + + if (!tablet.isConsistent()) { + inconsistentTabletIds.put(dbId, tablet.getId()); + } + } // end for tablets + } // end for indices + } // end for partitions + } finally { + table.readUnlock(); + } } // end for tables List oneLine = new ArrayList(TITLE_NAMES.size()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TablesProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TablesProcDir.java index 415d102780327d..d837c17a95057c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TablesProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TablesProcDir.java @@ -72,13 +72,7 @@ public ProcNodeInterface lookup(String tableIdStr) throws AnalysisException { throw new AnalysisException("Invalid table id format: " + tableIdStr); } - Table table = null; - db.readLock(); - try { - table = db.getTable(tableId); - } finally { - db.readUnlock(); - } + Table table = db.getTable(tableId); if (table == null) { throw new AnalysisException("Table[" + tableId + "] does not exist"); } @@ -92,14 +86,21 @@ public ProcResult fetchResult() throws AnalysisException { // get info List> tableInfos = new ArrayList>(); + List
tableList = null; db.readLock(); try { - for (Table table : db.getTables()) { - List tableInfo = new ArrayList(); + tableList = db.getTables(); + } finally { + db.readUnlock(); + } - int partitionNum = 1; - long replicaCount = 0; - String partitionKey = FeConstants.null_string; + for (Table table : tableList) { + List tableInfo = new ArrayList(); + int partitionNum = 1; + long replicaCount = 0; + String partitionKey = FeConstants.null_string; + table.readLock(); + try { if (table.getType() == TableType.OLAP) { OlapTable olapTable = (OlapTable) table; if (olapTable.getPartitionInfo().getType() == PartitionType.RANGE) { @@ -137,9 +138,9 @@ public ProcResult fetchResult() throws AnalysisException { tableInfo.add(TimeUtils.longToTimeString(table.getLastCheckTime())); tableInfo.add(replicaCount); tableInfos.add(tableInfo); + } finally { + table.readUnlock(); } - } finally { - db.readUnlock(); } // sort by table id diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletsProcDir.java index cd45d3c238b74a..7075081655ad31 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletsProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletsProcDir.java @@ -18,9 +18,9 @@ package org.apache.doris.common.proc; import org.apache.doris.catalog.Catalog; -import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Tablet; import org.apache.doris.catalog.TabletInvertedIndex; import org.apache.doris.common.AnalysisException; @@ -51,21 +51,21 @@ public class TabletsProcDir implements ProcDirInterface { .add("VersionCount").add("PathHash").add("MetaUrl").add("CompactionStatus") .build(); - private Database db; + private Table table; private MaterializedIndex index; - public TabletsProcDir(Database db, MaterializedIndex index) { - this.db = db; + public TabletsProcDir(Table table, MaterializedIndex index) { + this.table = table; this.index = index; } public List> fetchComparableResult(long version, long backendId, Replica.ReplicaState state) { - Preconditions.checkNotNull(db); + Preconditions.checkNotNull(table); Preconditions.checkNotNull(index); ImmutableMap backendMap = Catalog.getCurrentSystemInfo().getIdToBackend(); List> tabletInfos = new ArrayList>(); - db.readLock(); + table.readLock(); try { // get infos for (Tablet tablet : index.getTablets()) { @@ -143,7 +143,7 @@ public List> fetchComparableResult(long version, long backendId } } } finally { - db.readUnlock(); + table.readUnlock(); } return tabletInfos; } @@ -181,7 +181,7 @@ public boolean register(String name, ProcNodeInterface node) { @Override public ProcNodeInterface lookup(String tabletIdStr) throws AnalysisException { - Preconditions.checkNotNull(db); + Preconditions.checkNotNull(table); Preconditions.checkNotNull(index); long tabletId = -1L; diff --git a/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java b/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java index b401d6ff0feeed..883cf45cf7806a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java @@ -130,13 +130,14 @@ public boolean sendTasks() { Tablet tablet = null; AgentBatchTask batchTask = new AgentBatchTask(); - db.readLock(); + Table table = db.getTable(tabletMeta.getTableId()); + if (table == null) { + LOG.debug("table[{}] does not exist", tabletMeta.getTableId()); + return false; + } + + table.readLock(); try { - Table table = db.getTable(tabletMeta.getTableId()); - if (table == null) { - LOG.debug("table[{}] does not exist", tabletMeta.getTableId()); - return false; - } OlapTable olapTable = (OlapTable) table; Partition partition = olapTable.getPartition(tabletMeta.getPartitionId()); @@ -209,7 +210,7 @@ public boolean sendTasks() { } } finally { - db.readUnlock(); + table.readUnlock(); } if (state != JobState.RUNNING) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java b/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java index 929674d6658673..660fd2be91864e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java @@ -260,21 +260,27 @@ private List chooseTablets() { try { while ((chosenOne = dbQueue.poll()) != null) { Database db = (Database) chosenOne; + List
tables = null; db.readLock(); try { - // sort tables - List
tables = db.getTables(); - Queue tableQueue = new PriorityQueue<>(Math.max(tables.size(), 1), COMPARATOR); - for (Table table : tables) { - if (table.getType() != TableType.OLAP) { - continue; - } - tableQueue.add(table); - } + tables = db.getTables(); + } finally { + db.readUnlock(); + } - while ((chosenOne = tableQueue.poll()) != null) { - OlapTable table = (OlapTable) chosenOne; + // sort tables + Queue tableQueue = new PriorityQueue<>(Math.max(tables.size(), 1), COMPARATOR); + for (Table table : tables) { + if (table.getType() != TableType.OLAP) { + continue; + } + tableQueue.add(table); + } + while ((chosenOne = tableQueue.poll()) != null) { + OlapTable table = (OlapTable) chosenOne; + table.readLock(); + try { // sort partitions Queue partitionQueue = new PriorityQueue<>(Math.max(table.getAllPartitions().size(), 1), COMPARATOR); @@ -288,7 +294,7 @@ private List chooseTablets() { // check if this partition has no data if (partition.getVisibleVersion() == Partition.PARTITION_INIT_VERSION) { LOG.debug("partition[{}]'s version is {}. ignore", partition.getId(), - Partition.PARTITION_INIT_VERSION); + Partition.PARTITION_INIT_VERSION); continue; } partitionQueue.add(partition); @@ -322,12 +328,12 @@ private List chooseTablets() { && partition.getVisibleVersionHash() == tablet.getCheckedVersionHash()) { if (tablet.isConsistent()) { LOG.debug("tablet[{}]'s version[{}-{}] has been checked. ignore", - chosenTabletId, tablet.getCheckedVersion(), - tablet.getCheckedVersionHash()); + chosenTabletId, tablet.getCheckedVersion(), + tablet.getCheckedVersionHash()); } } else { LOG.info("chose tablet[{}-{}-{}-{}-{}] to check consistency", db.getId(), - table.getId(), partition.getId(), index.getId(), chosenTabletId); + table.getId(), partition.getId(), index.getId(), chosenTabletId); chosenTablets.add(chosenTabletId); } @@ -338,10 +344,10 @@ private List chooseTablets() { return chosenTablets; } } // end while partitionQueue - } // end while tableQueue - } finally { - db.readUnlock(); - } + } finally { + table.readUnlock(); + } + } // end while tableQueue } // end while dbQueue } finally { jobsLock.readLock().unlock(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/GetDdlStmtAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/GetDdlStmtAction.java index 51982b538f29ae..61c26319ad7cab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/GetDdlStmtAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/GetDdlStmtAction.java @@ -81,17 +81,16 @@ public void executeWithoutPassword(BaseRequest request, BaseResponse response) List addPartitionStmt = Lists.newArrayList(); List createRollupStmt = Lists.newArrayList(); - db.readLock(); - try { - Table table = db.getTable(tableName); - if (table == null) { - throw new DdlException("Table[" + tableName + "] does not exist"); - } + Table table = db.getTable(tableName); + if (table == null) { + throw new DdlException("Table[" + tableName + "] does not exist"); + } + table.readLock(); + try { Catalog.getDdlStmt(table, createTableStmt, addPartitionStmt, createRollupStmt, true, false /* show password */); - } finally { - db.readUnlock(); + table.readUnlock(); } Map> results = Maps.newHashMap(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/MigrationAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/MigrationAction.java index 5ebf92f27be2b9..32abdb69928978 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/MigrationAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/MigrationAction.java @@ -83,20 +83,22 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response } List> rows = Lists.newArrayList(); - db.readLock(); - try { - if (!Strings.isNullOrEmpty(tableName)) { - Table table = db.getTable(tableName); - if (table == null) { - throw new DdlException("Table[" + tableName + "] does not exist"); - } - if (table.getType() != TableType.OLAP) { - throw new DdlException("Table[" + tableName + "] is not OlapTable"); - } - OlapTable olapTable = (OlapTable) table; + if (!Strings.isNullOrEmpty(tableName)) { + Table table = db.getTable(tableName); + if (table == null) { + throw new DdlException("Table[" + tableName + "] does not exist"); + } + + if (table.getType() != TableType.OLAP) { + throw new DdlException("Table[" + tableName + "] is not OlapTable"); + } + + OlapTable olapTable = (OlapTable) table; + olapTable.readLock(); + try { for (Partition partition : olapTable.getPartitions()) { String partitionName = partition.getName(); MaterializedIndex baseIndex = partition.getBaseIndex(); @@ -113,16 +115,28 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response rows.add(row); } } - } else { - // get all olap table - for (Table table : db.getTables()) { - if (table.getType() != TableType.OLAP) { - continue; - } + } finally { + olapTable.readUnlock(); + } + } else { + List
tableList = null; + db.readLock(); + try { + tableList = db.getTables(); + } finally { + db.readUnlock(); + } - OlapTable olapTable = (OlapTable) table; - tableName = table.getName(); + // get all olap table + for (Table table : tableList) { + if (table.getType() != TableType.OLAP) { + continue; + } + OlapTable olapTable = (OlapTable) table; + table.readLock(); + try { + tableName = table.getName(); for (Partition partition : olapTable.getPartitions()) { String partitionName = partition.getName(); MaterializedIndex baseIndex = partition.getBaseIndex(); @@ -139,13 +153,11 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response rows.add(row); } } + } finally { + table.readUnlock(); } } - - } finally { - db.readUnlock(); } - ListComparator> comparator = new ListComparator>(0, 1, 2); Collections.sort(rows, comparator); diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/StorageTypeCheckAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/StorageTypeCheckAction.java index cc0a4840c7a712..a0a3fe68e51055 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/StorageTypeCheckAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/StorageTypeCheckAction.java @@ -67,15 +67,22 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response } JSONObject root = new JSONObject(); + List
tableList = null; db.readLock(); try { - List
tbls = db.getTables(); - for (Table tbl : tbls) { - if (tbl.getType() != TableType.OLAP) { - continue; - } + tableList = db.getTables(); + } finally { + db.readUnlock(); + } - OlapTable olapTbl = (OlapTable) tbl; + for (Table tbl : tableList) { + if (tbl.getType() != TableType.OLAP) { + continue; + } + + OlapTable olapTbl = (OlapTable) tbl; + olapTbl.readLock(); + try { JSONObject indexObj = new JSONObject(); for (Map.Entry entry : olapTbl.getIndexIdToMeta().entrySet()) { MaterializedIndexMeta indexMeta = entry.getValue(); @@ -84,9 +91,9 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response } } root.put(tbl.getName(), indexObj); + } finally { + olapTbl.readUnlock(); } - } finally { - db.readUnlock(); } // to json response diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableQueryPlanAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableQueryPlanAction.java index 752d3ebb92fa10..27042d707ba60b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableQueryPlanAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableQueryPlanAction.java @@ -131,25 +131,26 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response throw new DorisHttpException(HttpResponseStatus.NOT_FOUND, "Database [" + dbName + "] " + "does not exists"); } + Table table = db.getTable(tableName); + if (table == null) { + throw new DorisHttpException(HttpResponseStatus.NOT_FOUND, + "Table [" + tableName + "] " + "does not exists"); + } + // just only support OlapTable, ignore others such as ESTable + if (table.getType() != Table.TableType.OLAP) { + // Forbidden + throw new DorisHttpException(HttpResponseStatus.FORBIDDEN, + "only support OlapTable currently, but Table [" + tableName + "] " + + "is not a OlapTable"); + } + // may be should acquire writeLock - db.readLock(); + table.readLock(); try { - Table table = db.getTable(tableName); - if (table == null) { - throw new DorisHttpException(HttpResponseStatus.NOT_FOUND, - "Table [" + tableName + "] " + "does not exists"); - } - // just only support OlapTable, ignore others such as ESTable - if (table.getType() != Table.TableType.OLAP) { - // Forbidden - throw new DorisHttpException(HttpResponseStatus.FORBIDDEN, - "only support OlapTable currently, but Table [" + tableName + "] " - + "is not a OlapTable"); - } // parse/analysis/plan the sql and acquire tablet distributions handleQuery(ConnectContext.get(), fullDbName, tableName, sql, resultMap); } finally { - db.readUnlock(); + table.readUnlock(); } } catch (DorisHttpException e) { // status code should conforms to HTTP semantic diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableSchemaAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableSchemaAction.java index 0336c602acbd80..ae8bbe623a17dd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableSchemaAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableSchemaAction.java @@ -81,43 +81,43 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response if (db == null) { throw new DorisHttpException(HttpResponseStatus.NOT_FOUND, "Database [" + dbName + "] " + "does not exists"); } - db.readLock(); + + Table table = db.getTable(tableName); + if (table == null) { + throw new DorisHttpException(HttpResponseStatus.NOT_FOUND, "Table [" + tableName + "] " + "does not exists"); + } + // just only support OlapTable, ignore others such as ESTable + if (!(table instanceof OlapTable)) { + // Forbidden + throw new DorisHttpException(HttpResponseStatus.FORBIDDEN, "Table [" + tableName + "] " + + "is not a OlapTable, only support OlapTable currently"); + } + + table.readLock(); try { - Table table = db.getTable(tableName); - if (table == null) { - throw new DorisHttpException(HttpResponseStatus.NOT_FOUND, "Table [" + tableName + "] " + "does not exists"); - } - // just only support OlapTable, ignore others such as ESTable - if (!(table instanceof OlapTable)) { - // Forbidden - throw new DorisHttpException(HttpResponseStatus.FORBIDDEN, "Table [" + tableName + "] " - + "is not a OlapTable, only support OlapTable currently"); - } - try { - List columns = table.getBaseSchema(); - List> propList = new ArrayList(columns.size()); - for (Column column : columns) { - Map baseInfo = new HashMap<>(2); - Type colType = column.getOriginType(); - PrimitiveType primitiveType = colType.getPrimitiveType(); - if (primitiveType == PrimitiveType.DECIMALV2 || primitiveType == PrimitiveType.DECIMAL) { - ScalarType scalarType = (ScalarType) colType; - baseInfo.put("precision", scalarType.getPrecision() + ""); - baseInfo.put("scale", scalarType.getScalarScale() + ""); - } - baseInfo.put("type", primitiveType.toString()); - baseInfo.put("comment", column.getComment()); - baseInfo.put("name", column.getDisplayName()); - propList.add(baseInfo); + List columns = table.getBaseSchema(); + List> propList = new ArrayList(columns.size()); + for (Column column : columns) { + Map baseInfo = new HashMap<>(2); + Type colType = column.getOriginType(); + PrimitiveType primitiveType = colType.getPrimitiveType(); + if (primitiveType == PrimitiveType.DECIMALV2 || primitiveType == PrimitiveType.DECIMAL) { + ScalarType scalarType = (ScalarType) colType; + baseInfo.put("precision", scalarType.getPrecision() + ""); + baseInfo.put("scale", scalarType.getScalarScale() + ""); } - resultMap.put("status", 200); - resultMap.put("properties", propList); - } catch (Exception e) { - // Transform the general Exception to custom DorisHttpException - throw new DorisHttpException(HttpResponseStatus.INTERNAL_SERVER_ERROR, e.getMessage() == null ? "Null Pointer Exception" : e.getMessage()); + baseInfo.put("type", primitiveType.toString()); + baseInfo.put("comment", column.getComment()); + baseInfo.put("name", column.getDisplayName()); + propList.add(baseInfo); } + resultMap.put("status", 200); + resultMap.put("properties", propList); + } catch (Exception e) { + // Transform the general Exception to custom DorisHttpException + throw new DorisHttpException(HttpResponseStatus.INTERNAL_SERVER_ERROR, e.getMessage() == null ? "Null Pointer Exception" : e.getMessage()); } finally { - db.readUnlock(); + table.readUnlock(); } } catch (DorisHttpException e) { // status code should conforms to HTTP semantic diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/BrokerFileGroup.java b/fe/fe-core/src/main/java/org/apache/doris/load/BrokerFileGroup.java index 6278ff89c424e3..6bfdec40db12d0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/BrokerFileGroup.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/BrokerFileGroup.java @@ -140,35 +140,39 @@ public void parse(Database db, DataDescription dataDescription) throws DdlExcept } OlapTable olapTable = (OlapTable) table; tableId = table.getId(); - - // partitionId - PartitionNames partitionNames = dataDescription.getPartitionNames(); - if (partitionNames != null) { - partitionIds = Lists.newArrayList(); - for (String pName : partitionNames.getPartitionNames()) { - Partition partition = olapTable.getPartition(pName, partitionNames.isTemp()); - if (partition == null) { - throw new DdlException("Unknown partition '" + pName + "' in table '" + table.getName() + "'"); + table.readLock(); + try { + // partitionId + PartitionNames partitionNames = dataDescription.getPartitionNames(); + if (partitionNames != null) { + partitionIds = Lists.newArrayList(); + for (String pName : partitionNames.getPartitionNames()) { + Partition partition = olapTable.getPartition(pName, partitionNames.isTemp()); + if (partition == null) { + throw new DdlException("Unknown partition '" + pName + "' in table '" + table.getName() + "'"); + } + partitionIds.add(partition.getId()); } - partitionIds.add(partition.getId()); } - } - if (olapTable.getState() == OlapTableState.RESTORE) { - throw new DdlException("Table [" + table.getName() + "] is under restore"); - } + if (olapTable.getState() == OlapTableState.RESTORE) { + throw new DdlException("Table [" + table.getName() + "] is under restore"); + } - if (olapTable.getKeysType() != KeysType.AGG_KEYS && dataDescription.isNegative()) { - throw new DdlException("Load for AGG_KEYS table should not specify NEGATIVE"); - } + if (olapTable.getKeysType() != KeysType.AGG_KEYS && dataDescription.isNegative()) { + throw new DdlException("Load for AGG_KEYS table should not specify NEGATIVE"); + } - // check negative for sum aggregate type - if (dataDescription.isNegative()) { - for (Column column : table.getBaseSchema()) { - if (!column.isKey() && column.getAggregationType() != AggregateType.SUM) { - throw new DdlException("Column is not SUM AggregateType. column:" + column.getName()); + // check negative for sum aggregate type + if (dataDescription.isNegative()) { + for (Column column : table.getBaseSchema()) { + if (!column.isKey() && column.getAggregationType() != AggregateType.SUM) { + throw new DdlException("Column is not SUM AggregateType. column:" + column.getName()); + } } } + } finally { + table.readUnlock(); } // column diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java index 91a8006e0b3458..08d40ed1c8e762 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java @@ -131,16 +131,17 @@ public void process(DeleteStmt stmt) throws DdlException, QueryStateException { try { MarkedCountDownLatch countDownLatch; long transactionId = -1; - db.readLock(); - try { - Table table = db.getTable(tableName); - if (table == null) { - throw new DdlException("Table does not exist. name: " + tableName); - } + Table table = db.getTable(tableName); + if (table == null) { + throw new DdlException("Table does not exist. name: " + tableName); + } - if (table.getType() != Table.TableType.OLAP) { - throw new DdlException("Not olap type table. type: " + table.getType().name()); - } + if (table.getType() != Table.TableType.OLAP) { + throw new DdlException("Not olap type table. type: " + table.getType().name()); + } + + table.readLock(); + try { OlapTable olapTable = (OlapTable) table; if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { @@ -246,7 +247,7 @@ public void process(DeleteStmt stmt) throws DdlException, QueryStateException { } throw new DdlException(t.getMessage(), t); } finally { - db.readUnlock(); + table.readUnlock(); } long timeoutMs = deleteJob.getTimeoutMs(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java index dde3e87d9c02b0..90c946c0777898 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java @@ -91,15 +91,15 @@ public void checkAndUpdateQuorum() throws MetaNotFoundException { } short replicaNum = -1; - db.readLock(); + OlapTable table = (OlapTable) db.getTable(tableId); + if (table == null) { + throw new MetaNotFoundException("can not find table "+ tableId +" when commit delete"); + } + table.readLock(); try { - OlapTable table = (OlapTable) db.getTable(tableId); - if (table == null) { - throw new MetaNotFoundException("can not find table "+ tableId +" when commit delete"); - } replicaNum = table.getPartitionInfo().getReplicationNum(partitionId); } finally { - db.readUnlock(); + table.readUnlock(); } short quorumNum = (short) (replicaNum / 2 + 1); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java index 15cbec7819d93b..ddf039660cd894 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java @@ -191,10 +191,11 @@ public void setJob(ExportStmt stmt) throws UserException { this.partitions = stmt.getPartitions(); - db.readLock(); + this.exportTable = db.getTable(stmt.getTblName().getTbl()); + + exportTable.readLock(); try { this.dbId = db.getId(); - this.exportTable = db.getTable(stmt.getTblName().getTbl()); if (exportTable == null) { throw new DdlException("Table " + stmt.getTblName().getTbl() + " does not exist"); } @@ -202,7 +203,7 @@ public void setJob(ExportStmt stmt) throws UserException { this.tableName = stmt.getTblName(); genExecFragment(); } finally { - db.readUnlock(); + exportTable.readUnlock(); } this.sql = stmt.toSql(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java index 7ec2e14fa4496a..510c286b0d75b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java @@ -400,7 +400,7 @@ private void addLoadJob(LoadJob job, Database db) throws DdlException { } // check if table is in restore process - db.readLock(); + readLock(); try { for (Long tblId : job.getIdToTableLoadInfo().keySet()) { Table tbl = db.getTable(tblId); @@ -411,7 +411,7 @@ private void addLoadJob(LoadJob job, Database db) throws DdlException { } } } finally { - db.readUnlock(); + readUnlock(); } writeLock(); @@ -523,28 +523,30 @@ private LoadJob createLoadJob(LoadStmt stmt, EtlJobType etlJobType, // mini etl tasks Map idToEtlTask = Maps.newHashMap(); long etlTaskId = 0; - db.readLock(); - try { - for (DataDescription dataDescription : dataDescriptions) { - String tableName = dataDescription.getTableName(); - OlapTable table = (OlapTable) db.getTable(tableName); - if (table == null) { - throw new DdlException("Table[" + tableName + "] does not exist"); - } + for (DataDescription dataDescription : dataDescriptions) { + String tableName = dataDescription.getTableName(); + OlapTable table = (OlapTable) db.getTable(tableName); + if (table == null) { + throw new DdlException("Table[" + tableName + "] does not exist"); + } + + table.readLock(); + try { TNetworkAddress beAddress = dataDescription.getBeAddr(); Backend backend = Catalog.getCurrentSystemInfo().getBackendWithBePort(beAddress.getHostname(), - beAddress.getPort()); + beAddress.getPort()); if (!Catalog.getCurrentSystemInfo().checkBackendAvailable(backend.getId())) { throw new DdlException("Etl backend is null or not available"); } MiniEtlTaskInfo taskInfo = new MiniEtlTaskInfo(etlTaskId++, backend.getId(), table.getId()); idToEtlTask.put(taskInfo.getId(), taskInfo); + } finally { + table.readUnlock(); } - } finally { - db.readUnlock(); } + job.setMiniEtlTasks(idToEtlTask); job.setPriority(TPriority.HIGH); @@ -631,17 +633,18 @@ public static void checkAndCreateSource(Database db, DataDescription dataDescrip // source column names and partitions String tableName = dataDescription.getTableName(); Map>> columnToFunction = null; - db.readLock(); - try { - Table table = db.getTable(tableName); - if (table == null) { - throw new DdlException("Table [" + tableName + "] does not exist"); - } - tableId = table.getId(); - if (table.getType() != TableType.OLAP) { - throw new DdlException("Table [" + tableName + "] is not olap table"); - } + Table table = db.getTable(tableName); + if (table == null) { + throw new DdlException("Table [" + tableName + "] does not exist"); + } + tableId = table.getId(); + if (table.getType() != TableType.OLAP) { + throw new DdlException("Table [" + tableName + "] is not olap table"); + } + + table.readLock(); + try { if (((OlapTable) table).getPartitionInfo().isMultiColumnPartition() && jobType == EtlJobType.HADOOP) { throw new DdlException("Load by hadoop cluster does not support table with multi partition columns." + " Table: " + table.getName() + ". Try using broker load. See 'help broker load;'"); @@ -832,7 +835,7 @@ public static void checkAndCreateSource(Database db, DataDescription dataDescrip } } } finally { - db.readUnlock(); + table.readUnlock(); } // column separator @@ -2234,25 +2237,25 @@ public List> getLoadJobUnfinishedInfo(long jobId) { return infos; } - db.readLock(); - try { - readLock(); - try { - Map tabletMap = loadJob.getIdToTabletLoadInfo(); - for (long tabletId : tabletMap.keySet()) { - TabletMeta tabletMeta = invertedIndex.getTabletMeta(tabletId); - if (tabletMeta == null) { - // tablet may be dropped during loading - continue; - } - long tableId = tabletMeta.getTableId(); + readLock(); + try { + Map tabletMap = loadJob.getIdToTabletLoadInfo(); + for (long tabletId : tabletMap.keySet()) { + TabletMeta tabletMeta = invertedIndex.getTabletMeta(tabletId); + if (tabletMeta == null) { + // tablet may be dropped during loading + continue; + } - OlapTable table = (OlapTable) db.getTable(tableId); - if (table == null) { - continue; - } + long tableId = tabletMeta.getTableId(); + OlapTable table = (OlapTable) db.getTable(tableId); + if (table == null) { + continue; + } + table.readLock(); + try { long partitionId = tabletMeta.getPartitionId(); Partition partition = table.getPartition(partitionId); if (partition == null) { @@ -2291,13 +2294,12 @@ public List> getLoadJobUnfinishedInfo(long jobId) { infos.add(info); } - } // end for tablet - - } finally { - readUnlock(); + } finally { + table.readUnlock(); + } } } finally { - db.readUnlock(); + readUnlock(); } // sort by version, backendId @@ -3526,16 +3528,18 @@ public void delete(DeleteStmt stmt) throws DdlException { long partitionId = -1; LoadJob loadDeleteJob = null; boolean addRunningPartition = false; - db.readLock(); - try { - Table table = db.getTable(tableName); - if (table == null) { - throw new DdlException("Table does not exist. name: " + tableName); - } + Table table = db.getTable(tableName); - if (table.getType() != TableType.OLAP) { - throw new DdlException("Not olap type table. type: " + table.getType().name()); - } + 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()); + } + + table.readLock(); + try { OlapTable olapTable = (OlapTable) table; if (olapTable.getState() != OlapTableState.NORMAL) { @@ -3604,7 +3608,7 @@ public void delete(DeleteStmt stmt) throws DdlException { writeUnlock(); } } - db.readUnlock(); + table.readUnlock(); } try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java index 10477e7fb74252..77b7cfe3ebc5f2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java @@ -351,13 +351,7 @@ private Set submitPushTasks(LoadJob job, Database db) { Map idToTableLoadInfo = job.getIdToTableLoadInfo(); for (Entry tableEntry : idToTableLoadInfo.entrySet()) { long tableId = tableEntry.getKey(); - OlapTable table = null; - db.readLock(); - try { - table = (OlapTable) db.getTable(tableId); - } finally { - db.readUnlock(); - } + OlapTable table = (OlapTable) db.getTable(tableId); if (table == null) { LOG.warn("table does not exist. id: {}", tableId); // if table is dropped during load, the the job is failed @@ -381,7 +375,7 @@ private Set submitPushTasks(LoadJob job, Database db) { continue; } - db.readLock(); + table.readLock(); try { Partition partition = table.getPartition(partitionId); if (partition == null) { @@ -505,7 +499,7 @@ private Set submitPushTasks(LoadJob job, Database db) { } // end for tablets } // end for indices } finally { - db.readUnlock(); + table.readUnlock(); } } // end for partitions } // end for tables @@ -553,7 +547,6 @@ private void runOneQuorumFinishedJob(LoadJob job) { // if the job is quorum finished, just set it to finished and clear related etl job if (load.updateLoadJobState(job, JobState.FINISHED)) { load.clearJob(job, JobState.QUORUM_FINISHED); - return; } } @@ -565,18 +558,14 @@ private void runOneQuorumFinishedDeleteJob(AsyncDeleteJob job) { load.removeDeleteJobAndSetState(job); return; } - db.readLock(); - try { - // if the delete job is quorum finished, just set it to finished - job.clearTasks(); - job.setState(DeleteState.FINISHED); - // log - Catalog.getCurrentCatalog().getEditLog().logFinishAsyncDelete(job); - load.removeDeleteJobAndSetState(job); - LOG.info("delete job {} finished", job.getJobId()); - } finally { - db.readUnlock(); - } + + // if the delete job is quorum finished, just set it to finished + job.clearTasks(); + job.setState(DeleteState.FINISHED); + // log + Catalog.getCurrentCatalog().getEditLog().logFinishAsyncDelete(job); + load.removeDeleteJobAndSetState(job); + LOG.info("delete job {} finished", job.getJobId()); } public static boolean checkTimeout(LoadJob job) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java index aeae05b20ee953..15735c0edcae6a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java @@ -99,9 +99,9 @@ public void beginTxn() MetricRepo.COUNTER_LOAD_ADD.increase(1L); transactionId = Catalog.getCurrentGlobalTransactionMgr() .beginTransaction(dbId, Lists.newArrayList(fileGroupAggInfo.getAllTableIds()), label, null, - new TxnCoordinator(TxnSourceType.FE, FrontendOptions.getLocalHostAddress()), - TransactionState.LoadJobSourceType.BATCH_LOAD_JOB, id, - timeoutSecond); + new TxnCoordinator(TxnSourceType.FE, FrontendOptions.getLocalHostAddress()), + TransactionState.LoadJobSourceType.BATCH_LOAD_JOB, id, + timeoutSecond); } @Override @@ -140,18 +140,18 @@ private void onPendingTaskFinished(BrokerPendingTaskAttachment attachment) { // check if job has been cancelled if (isTxnDone()) { LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) - .add("state", state) - .add("error_msg", "this task will be ignored when job is: " + state) - .build()); + .add("state", state) + .add("error_msg", "this task will be ignored when job is: " + state) + .build()); return; } if (finishedTaskIds.contains(attachment.getTaskId())) { LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) - .add("task_id", attachment.getTaskId()) - .add("error_msg", "this is a duplicated callback of pending task " - + "when broker already has loading task") - .build()); + .add("task_id", attachment.getTaskId()) + .add("error_msg", "this is a duplicated callback of pending task " + + "when broker already has loading task") + .build()); return; } @@ -166,9 +166,9 @@ private void onPendingTaskFinished(BrokerPendingTaskAttachment attachment) { createLoadingTask(db, attachment); } catch (UserException e) { LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) - .add("database_id", dbId) - .add("error_msg", "Failed to divide job into loading task.") - .build(), e); + .add("database_id", dbId) + .add("error_msg", "Failed to divide job into loading task.") + .build(), e); cancelJobWithoutCheck(new FailMsg(FailMsg.CancelType.ETL_RUN_FAIL, e.getMessage()), true, true); return; } catch (RejectedExecutionException e) { @@ -196,12 +196,12 @@ private void createLoadingTask(Database db, BrokerPendingTaskAttachment attachme OlapTable table = (OlapTable) db.getTable(tableId); if (table == null) { LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) - .add("database_id", dbId) - .add("table_id", tableId) - .add("error_msg", "Failed to divide job into loading task when table not found") - .build()); + .add("database_id", dbId) + .add("table_id", tableId) + .add("error_msg", "Failed to divide job into loading task when table not found") + .build()); throw new MetaNotFoundException("Failed to divide job into loading task when table " - + tableId + " not found"); + + tableId + " not found"); } // Generate loading task and init the plan of task @@ -243,17 +243,17 @@ private void onLoadingTaskFinished(BrokerLoadingTaskAttachment attachment) { // check if job has been cancelled if (isTxnDone()) { LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) - .add("state", state) - .add("error_msg", "this task will be ignored when job is: " + state) - .build()); + .add("state", state) + .add("error_msg", "this task will be ignored when job is: " + state) + .build()); return; } // check if task has been finished if (finishedTaskIds.contains(attachment.getTaskId())) { LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) - .add("task_id", attachment.getTaskId()) - .add("error_msg", "this is a duplicated callback of loading task").build()); + .add("task_id", attachment.getTaskId()) + .add("error_msg", "this is a duplicated callback of loading task").build()); return; } @@ -271,8 +271,8 @@ private void onLoadingTaskFinished(BrokerLoadingTaskAttachment attachment) { if (LOG.isDebugEnabled()) { LOG.debug(new LogBuilder(LogKey.LOAD_JOB, id) - .add("commit_infos", Joiner.on(",").join(commitInfos)) - .build()); + .add("commit_infos", Joiner.on(",").join(commitInfos)) + .build()); } // check data quality @@ -286,30 +286,29 @@ private void onLoadingTaskFinished(BrokerLoadingTaskAttachment attachment) { db = getDb(); } catch (MetaNotFoundException e) { LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) - .add("database_id", dbId) - .add("error_msg", "db has been deleted when job is loading") - .build(), e); + .add("database_id", dbId) + .add("error_msg", "db has been deleted when job is loading") + .build(), e); cancelJobWithoutCheck(new FailMsg(FailMsg.CancelType.LOAD_RUN_FAIL, e.getMessage()), true, true); return; } db.writeLock(); try { LOG.info(new LogBuilder(LogKey.LOAD_JOB, id) - .add("txn_id", transactionId) - .add("msg", "Load job try to commit txn") - .build()); + .add("txn_id", transactionId) + .add("msg", "Load job try to commit txn") + .build()); MetricRepo.COUNTER_LOAD_FINISHED.increase(1L); Catalog.getCurrentGlobalTransactionMgr().commitTransaction( dbId, transactionId, commitInfos, new LoadJobFinalOperation(id, loadingStatus, progress, loadStartTimestamp, - finishTimestamp, state, failMsg)); + finishTimestamp, state, failMsg)); } catch (UserException e) { LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) - .add("database_id", dbId) - .add("error_msg", "Failed to commit txn with error:" + e.getMessage()) - .build(), e); + .add("database_id", dbId) + .add("error_msg", "Failed to commit txn with error:" + e.getMessage()) + .build(), e); cancelJobWithoutCheck(new FailMsg(FailMsg.CancelType.LOAD_RUN_FAIL, e.getMessage()), true, true); - return; } finally { db.writeUnlock(); } @@ -343,11 +342,11 @@ private void writeProfile() { private void updateLoadingStatus(BrokerLoadingTaskAttachment attachment) { loadingStatus.replaceCounter(DPP_ABNORMAL_ALL, - increaseCounter(DPP_ABNORMAL_ALL, attachment.getCounter(DPP_ABNORMAL_ALL))); + increaseCounter(DPP_ABNORMAL_ALL, attachment.getCounter(DPP_ABNORMAL_ALL))); loadingStatus.replaceCounter(DPP_NORMAL_ALL, - increaseCounter(DPP_NORMAL_ALL, attachment.getCounter(DPP_NORMAL_ALL))); + increaseCounter(DPP_NORMAL_ALL, attachment.getCounter(DPP_NORMAL_ALL))); loadingStatus.replaceCounter(UNSELECTED_ROWS, - increaseCounter(UNSELECTED_ROWS, attachment.getCounter(UNSELECTED_ROWS))); + increaseCounter(UNSELECTED_ROWS, attachment.getCounter(UNSELECTED_ROWS))); if (attachment.getTrackingUrl() != null) { loadingStatus.setTrackingUrl(attachment.getTrackingUrl()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java index e345e66dc6ad31..cf1e029178af95 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java @@ -626,14 +626,9 @@ private Database checkDb(String dbName) throws DdlException { * @throws DdlException */ private void checkTable(Database database, String tableName) throws DdlException { - database.readLock(); - try { - if (database.getTable(tableName) == null) { - LOG.info("Table {} is not belongs to database {}", tableName, database.getFullName()); - throw new DdlException("Table[" + tableName + "] does not exist"); - } - } finally { - database.readUnlock(); + if (database.getTable(tableName) == null) { + LOG.info("Table {} is not belongs to database {}", tableName, database.getFullName()); + throw new DdlException("Table[" + tableName + "] does not exist"); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java index a72944aee33e80..bb464ee1520bd7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java @@ -349,7 +349,7 @@ public static KafkaRoutineLoadJob fromCreateStmt(CreateRoutineLoadStmt stmt) thr long tableId = -1L; db.readLock(); try { - unprotectedCheckMeta(db, stmt.getTableName(), stmt.getRoutineLoadDesc()); + checkMeta(db, stmt.getTableName(), stmt.getRoutineLoadDesc()); Table table = db.getTable(stmt.getTableName()); tableId = table.getId(); } finally { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java index cbe30e6fe2689a..30c2ee5182b28a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java @@ -1070,7 +1070,7 @@ private void executeTaskOnTxnStatusChanged(RoutineLoadTaskInfo routineLoadTaskIn } } - protected static void unprotectedCheckMeta(Database db, String tblName, RoutineLoadDesc routineLoadDesc) + protected static void checkMeta(Database db, String tblName, RoutineLoadDesc routineLoadDesc) throws UserException { Table table = db.getTable(tblName); if (table == null) { @@ -1092,10 +1092,15 @@ protected static void unprotectedCheckMeta(Database db, String tblName, RoutineL // check partitions OlapTable olapTable = (OlapTable) table; - for (String partName : partitionNames.getPartitionNames()) { - if (olapTable.getPartition(partName, partitionNames.isTemp()) == null) { - throw new DdlException("Partition " + partName + " does not exist"); + olapTable.readLock(); + try { + for (String partName : partitionNames.getPartitionNames()) { + if (olapTable.getPartition(partName, partitionNames.isTemp()) == null) { + throw new DdlException("Partition " + partName + " does not exist"); + } } + } finally { + olapTable.readUnlock(); } // columns will be checked when planing diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java index a3045af702bbe8..a6b52c80dfbf27 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -1263,7 +1263,7 @@ private void handleShowTablet() throws AnalysisException { if (indexId > -1 && index.getId() != indexId) { continue; } - TabletsProcDir procDir = new TabletsProcDir(db, index); + TabletsProcDir procDir = new TabletsProcDir(table, index); tabletInfos.addAll(procDir.fetchComparableResult( showStmt.getVersion(), showStmt.getBackendId(), showStmt.getReplicaState())); if (sizeLimit > -1 && tabletInfos.size() >= sizeLimit) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index e55464a99290b5..ee36fd356c4a63 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -269,19 +269,27 @@ public TListTableStatusResult listTableStatus(TGetTablesParams params) throws TE continue; } - if (matcher != null && !matcher.match(table.getName())) { - continue; + table.readLock(); + try { + if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(currentUser, params.db, + table.getName(), PrivPredicate.SHOW)) { + continue; + } + + if (matcher != null && !matcher.match(table.getName())) { + continue; + } + TTableStatus status = new TTableStatus(); + status.setName(table.getName()); + status.setType(table.getMysqlType()); + status.setEngine(table.getEngine()); + status.setComment(table.getComment()); + status.setCreateTime(table.getCreateTime()); + status.setLastCheckTime(table.getLastCheckTime()); + tablesResult.add(status); + } finally { + table.readUnlock(); } - TTableStatus status = new TTableStatus(); - status.setName(table.getName()); - status.setType(table.getMysqlType()); - status.setEngine(table.getEngine()); - status.setComment(table.getComment()); - status.setCreateTime(table.getCreateTime()); - status.setLastCheckTime(table.getLastCheckTime()); - status.setDdlSql(table.getDdlSql()); - - tablesResult.add(status); } } return result; diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java index 059ccec6121254..3ca54f32c4184d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java @@ -18,7 +18,6 @@ package org.apache.doris.system; import org.apache.doris.catalog.Catalog; -import org.apache.doris.catalog.Database; import org.apache.doris.catalog.DiskInfo; import org.apache.doris.cluster.Cluster; import org.apache.doris.common.AnalysisException; @@ -894,17 +893,8 @@ public long getBackendReportVersion(long backendId) { public void updateBackendReportVersion(long backendId, long newReportVersion, long dbId) { AtomicLong atomicLong = null; if ((atomicLong = idToReportVersionRef.get(backendId)) != null) { - Database db = Catalog.getCurrentCatalog().getDb(dbId); - if (db != null) { - try { - atomicLong.set(newReportVersion); - LOG.debug("update backend {} report version: {}, db: {}", backendId, newReportVersion, dbId); - } finally { - db.readUnlock(); - } - } else { - LOG.warn("failed to update backend report version, db {} does not exist", dbId); - } + atomicLong.set(newReportVersion); + LOG.debug("update backend {} report version: {}, db: {}", backendId, newReportVersion, dbId); } else { LOG.warn("failed to update backend report version, backend {} does not exist", backendId); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java index 0559195c692214..6ec245ca990777 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java @@ -493,12 +493,7 @@ public boolean isTimeout(long currentMillis) { return transactionStatus == TransactionStatus.PREPARE && currentMillis - prepareTime > timeoutMs; } - /* - * Add related table indexes to the transaction. - * If function should always be called before adding this transaction state to transaction manager, - * No other thread will access this state. So no need to lock - */ - public void addTableIndexes(OlapTable table) { + public synchronized void addTableIndexes(OlapTable table) { Set indexIds = loadedTblIndexes.get(table.getId()); if (indexIds == null) { indexIds = Sets.newHashSet(); From 5118571e4dd32ef905250d6e654121cc0a4ee448 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Wed, 3 Jun 2020 19:15:57 +0800 Subject: [PATCH 04/50] remove some unused temp code --- fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 3e33e150e0cdfe..325b9449914d43 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -132,7 +132,7 @@ public boolean tryReadLock(long timeout, TimeUnit unit) { return false; } } - + public void readUnlock() { this.rwLock.readLock().unlock(); } From 1354d056e8218bd394f58ad03ee05baa38b81385 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Wed, 3 Jun 2020 22:30:03 +0800 Subject: [PATCH 05/50] use table lock to replace db lock --- .../apache/doris/alter/SchemaChangeJob.java | 28 +- .../apache/doris/alter/SchemaChangeJobV2.java | 27 +- .../org/apache/doris/backup/RestoreJob.java | 207 +++++++------ .../org/apache/doris/catalog/Catalog.java | 291 ++++++++++-------- .../org/apache/doris/catalog/Database.java | 2 +- .../java/org/apache/doris/catalog/Table.java | 4 + 6 files changed, 312 insertions(+), 247 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java index d41c149dc723c3..b153e2bb6f9b85 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java @@ -666,16 +666,16 @@ public int tryFinishJob() { return -1; } - db.writeLock(); + Table table = db.getTable(tableId); + if (table == null) { + cancelMsg = String.format("table %d does not exist", tableId); + LOG.warn(cancelMsg); + return -1; + } + + table.writeLock(); try { synchronized (this) { - Table table = db.getTable(tableId); - if (table == null) { - cancelMsg = String.format("table %d does not exist", tableId); - LOG.warn(cancelMsg); - return -1; - } - boolean hasUnfinishedPartition = false; OlapTable olapTable = (OlapTable) table; for (Partition partition : olapTable.getPartitions()) { @@ -867,7 +867,7 @@ public int tryFinishJob() { this.transactionId = Catalog.getCurrentGlobalTransactionMgr().getTransactionIDGenerator().getNextTransactionId(); } } finally { - db.writeUnlock(); + table.writeUnlock(); } Catalog.getCurrentCatalog().getEditLog().logFinishingSchemaChange(this); @@ -959,10 +959,12 @@ public void replayInitJob(Database db) { @Override public void replayFinishing(Database db) { - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + return; + } + olapTable.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - // set the status to normal for (Partition partition : olapTable.getPartitions()) { long partitionId = partition.getId(); @@ -1028,7 +1030,7 @@ public void replayFinishing(Database db) { olapTable.setBloomFilterInfo(bfColumns, bfFpp); } // end for partitions } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } LOG.info("replay finishing schema change job: {}", tableId); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java index b84e3f17a8e77e..842b8a6667cfac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java @@ -717,16 +717,16 @@ private void replayWaitingTxn(SchemaChangeJobV2 replayedJob) { return; } - db.writeLock(); + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + // table may be dropped before replaying this log. just return + return; + } + tbl.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(); + tbl.writeUnlock(); } // should still be in WAITING_TXN state, so that the alter tasks will be resend again @@ -742,14 +742,15 @@ private void replayWaitingTxn(SchemaChangeJobV2 replayedJob) { 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) { + OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl != null) { + tbl.writeLock(); + try { onFinished(tbl); + } finally { + tbl.writeUnlock(); } - } finally { - db.writeUnlock(); + } } jobState = JobState.FINISHED; diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java index dd7162beb66963..fcc10c66352f22 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -652,19 +652,18 @@ private void checkAndPrepareMeta() { if (ok) { LOG.debug("finished to create all restored replcias. {}", this); - // add all restored partition and tbls to catalog - db.writeLock(); - try { - // add restored partitions. - // table should be in State RESTORE, so no other partitions can be - // added to or removed from this table during the restore process. - for (Pair entry : restoredPartitions) { - OlapTable localTbl = (OlapTable) db.getTable(entry.first); + // add restored partitions. + // table should be in State RESTORE, so no other partitions can be + // added to or removed from this table during the restore process. + for (Pair entry : restoredPartitions) { + OlapTable localTbl = (OlapTable) db.getTable(entry.first); + localTbl.writeLock(); + try { Partition restoredPart = entry.second; OlapTable remoteTbl = (OlapTable) backupMeta.getTable(entry.first); RangePartitionInfo localPartitionInfo = (RangePartitionInfo) localTbl.getPartitionInfo(); RangePartitionInfo remotePartitionInfo = (RangePartitionInfo) remoteTbl.getPartitionInfo(); - BackupPartitionInfo backupPartitionInfo + BackupPartitionInfo backupPartitionInfo = jobInfo.getTableInfo(entry.first).getPartInfo(restoredPart.getName()); long remotePartId = backupPartitionInfo.id; Range remoteRange = remotePartitionInfo.getRange(remotePartId); @@ -673,18 +672,25 @@ private void checkAndPrepareMeta() { remoteDataProperty, (short) restoreReplicationNum, remotePartitionInfo.getIsInMemory(remotePartId)); localTbl.addPartition(restoredPart); + } finally { + localTbl.writeUnlock(); } - // add restored tables - for (OlapTable tbl : restoredTbls) { + } + + // add restored tables + for (OlapTable tbl : restoredTbls) { + db.writeLock(); + try { if (!db.createTable(tbl)) { status = new Status(ErrCode.COMMON_ERROR, "Table " + tbl.getName() + " already exist in db: " + db.getFullName()); return; } + } finally { + db.writeUnlock(); } - } finally { - db.writeUnlock(); + } } else { List> unfinishedMarks = latch.getLeftMarks(); @@ -907,51 +913,62 @@ private boolean downloadAndDeserializeMetaInfo() { private void replayCheckAndPrepareMeta() { Database db = catalog.getDb(dbId); - db.writeLock(); - try { - // replay set all existing tables's state to RESTORE - for (BackupTableInfo tblInfo : jobInfo.tables.values()) { - Table tbl = db.getTable(jobInfo.getAliasByOriginNameIfSet(tblInfo.name)); - if (tbl == null) { - continue; - } - OlapTable olapTbl = (OlapTable) tbl; + + // replay set all existing tables's state to RESTORE + for (BackupTableInfo tblInfo : jobInfo.tables.values()) { + Table tbl = db.getTable(jobInfo.getAliasByOriginNameIfSet(tblInfo.name)); + if (tbl == null) { + continue; + } + OlapTable olapTbl = (OlapTable) tbl; + tbl.writeLock(); + try { olapTbl.setState(OlapTableState.RESTORE); + } finally { + tbl.writeUnlock(); } + } - // restored partitions - for (Pair entry : restoredPartitions) { - OlapTable localTbl = (OlapTable) db.getTable(entry.first); - Partition restorePart = entry.second; - OlapTable remoteTbl = (OlapTable) backupMeta.getTable(entry.first); - RangePartitionInfo localPartitionInfo = (RangePartitionInfo) localTbl.getPartitionInfo(); - RangePartitionInfo remotePartitionInfo = (RangePartitionInfo) remoteTbl.getPartitionInfo(); - BackupPartitionInfo backupPartitionInfo = jobInfo.getTableInfo(entry.first).getPartInfo(restorePart.getName()); - long remotePartId = backupPartitionInfo.id; - Range remoteRange = remotePartitionInfo.getRange(remotePartId); - DataProperty remoteDataProperty = remotePartitionInfo.getDataProperty(remotePartId); - localPartitionInfo.addPartition(restorePart.getId(), false, remoteRange, - remoteDataProperty, (short) restoreReplicationNum, - remotePartitionInfo.getIsInMemory(remotePartId)); - localTbl.addPartition(restorePart); - - // modify tablet inverted index - 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); - for (Tablet restoreTablet : restoreIdx.getTablets()) { - Catalog.getCurrentInvertedIndex().addTablet(restoreTablet.getId(), tabletMeta); - for (Replica restoreReplica : restoreTablet.getReplicas()) { - Catalog.getCurrentInvertedIndex().addReplica(restoreTablet.getId(), restoreReplica); - } + // restored partitions + for (Pair entry : restoredPartitions) { + OlapTable localTbl = (OlapTable) db.getTable(entry.first); + Partition restorePart = entry.second; + OlapTable remoteTbl = (OlapTable) backupMeta.getTable(entry.first); + RangePartitionInfo localPartitionInfo = (RangePartitionInfo) localTbl.getPartitionInfo(); + RangePartitionInfo remotePartitionInfo = (RangePartitionInfo) remoteTbl.getPartitionInfo(); + BackupPartitionInfo backupPartitionInfo = jobInfo.getTableInfo(entry.first).getPartInfo(restorePart.getName()); + long remotePartId = backupPartitionInfo.id; + Range remoteRange = remotePartitionInfo.getRange(remotePartId); + DataProperty remoteDataProperty = remotePartitionInfo.getDataProperty(remotePartId); + localPartitionInfo.addPartition(restorePart.getId(), false, remoteRange, + remoteDataProperty, (short) restoreReplicationNum, + remotePartitionInfo.getIsInMemory(remotePartId)); + localTbl.addPartition(restorePart); + + // modify tablet inverted index + 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); + for (Tablet restoreTablet : restoreIdx.getTablets()) { + Catalog.getCurrentInvertedIndex().addTablet(restoreTablet.getId(), tabletMeta); + for (Replica restoreReplica : restoreTablet.getReplicas()) { + Catalog.getCurrentInvertedIndex().addReplica(restoreTablet.getId(), restoreReplica); } } } + } - // restored tables - for (OlapTable restoreTbl : restoredTbls) { + // restored tables + for (OlapTable restoreTbl : restoredTbls) { + db.writeLock(); + try { db.createTable(restoreTbl); + } finally { + db.writeUnlock(); + } + restoreTbl.writeLock(); + try { // modify tablet inverted index for (Partition restorePart : restoreTbl.getPartitions()) { for (MaterializedIndex restoreIdx : restorePart.getMaterializedIndices(IndexExtState.VISIBLE)) { @@ -966,11 +983,10 @@ private void replayCheckAndPrepareMeta() { } } } + } finally { + restoreTbl.writeUnlock(); } - } finally { - db.writeUnlock(); } - LOG.info("replay check and prepare meta. {}", this); } @@ -1200,17 +1216,16 @@ private Status allTabletCommitted(boolean isReplay) { } // set all restored partition version and version hash - db.writeLock(); - try { - // set all tables' state to NORMAL - setTableStateToNormal(db); - - for (long tblId : restoredVersionInfo.rowKeySet()) { - Table tbl = db.getTable(tblId); - if (tbl == null) { - continue; - } - OlapTable olapTbl = (OlapTable) tbl; + // set all tables' state to NORMAL + setTableStateToNormal(db); + for (long tblId : restoredVersionInfo.rowKeySet()) { + Table tbl = db.getTable(tblId); + if (tbl == null) { + continue; + } + OlapTable olapTbl = (OlapTable) tbl; + tbl.writeLock(); + try { Map> map = restoredVersionInfo.rowMap().get(tblId); for (Map.Entry> entry : map.entrySet()) { long partId = entry.getKey(); @@ -1229,18 +1244,18 @@ private Status allTabletCommitted(boolean isReplay) { if (!replica.checkVersionCatchUp(part.getVisibleVersion(), part.getVisibleVersionHash(), false)) { replica.updateVersionInfo(part.getVisibleVersion(), part.getVisibleVersionHash(), - replica.getDataSize(), replica.getRowCount()); + replica.getDataSize(), replica.getRowCount()); } } } } LOG.debug("restore set partition {} version in table {}, version: {}, version hash: {}", - partId, tblId, entry.getValue().first, entry.getValue().second); + partId, tblId, entry.getValue().first, entry.getValue().second); } + } finally { + tbl.writeUnlock(); } - } finally { - db.writeUnlock(); } if (!isReplay) { @@ -1364,14 +1379,14 @@ public void cancelInternal(boolean isReplay) { // clean restored objs Database db = catalog.getDb(dbId); if (db != null) { - db.writeLock(); - try { - // rollback table's state to NORMAL - setTableStateToNormal(db); + // rollback table's state to NORMAL + setTableStateToNormal(db); - // remove restored tbls - for (OlapTable restoreTbl : restoredTbls) { - LOG.info("remove restored table when cancelled: {}", restoreTbl.getName()); + // remove restored tbls + for (OlapTable restoreTbl : restoredTbls) { + LOG.info("remove restored table when cancelled: {}", restoreTbl.getName()); + restoreTbl.writeLock(); + try { for (Partition part : restoreTbl.getPartitions()) { for (MaterializedIndex idx : part.getMaterializedIndices(IndexExtState.VISIBLE)) { for (Tablet tablet : idx.getTablets()) { @@ -1379,17 +1394,27 @@ public void cancelInternal(boolean isReplay) { } } } + } finally { + restoreTbl.writeUnlock(); + } + db.writeLock(); + try { db.dropTable(restoreTbl.getName()); + } finally { + db.writeUnlock(); } + } - // remove restored partitions - for (Pair entry : restoredPartitions) { - OlapTable restoreTbl = (OlapTable) db.getTable(entry.first); - if (restoreTbl == null) { - continue; - } - LOG.info("remove restored partition in table {} when cancelled: {}", - restoreTbl.getName(), entry.second.getName()); + // remove restored partitions + for (Pair entry : restoredPartitions) { + OlapTable restoreTbl = (OlapTable) db.getTable(entry.first); + if (restoreTbl == null) { + continue; + } + LOG.info("remove restored partition in table {} when cancelled: {}", + restoreTbl.getName(), entry.second.getName()); + restoreTbl.writeLock(); + try { for (MaterializedIndex idx : entry.second.getMaterializedIndices(IndexExtState.VISIBLE)) { for (Tablet tablet : idx.getTablets()) { Catalog.getCurrentInvertedIndex().deleteTablet(tablet.getId()); @@ -1397,9 +1422,10 @@ public void cancelInternal(boolean isReplay) { } restoreTbl.dropPartition(dbId, entry.second.getName(), true /* is restore */); + } finally { + restoreTbl.writeUnlock(); } - } finally { - db.writeUnlock(); + } } @@ -1436,9 +1462,14 @@ private void setTableStateToNormal(Database db) { } OlapTable olapTbl = (OlapTable) tbl; - if (olapTbl.getState() == OlapTableState.RESTORE - || olapTbl.getState() == OlapTableState.RESTORE_WITH_LOAD) { - olapTbl.setState(OlapTableState.NORMAL); + tbl.writeLock(); + try { + if (olapTbl.getState() == OlapTableState.RESTORE + || olapTbl.getState() == OlapTableState.RESTORE_WITH_LOAD) { + olapTbl.setState(OlapTableState.NORMAL); + } + } finally { + tbl.writeUnlock(); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index e9823237fa3d6f..a46e505452b9d9 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -2764,7 +2764,17 @@ public void dropDb(DropDbStmt stmt) throws DdlException { public void unprotectDropDb(Database db, boolean isForeDrop) { for (Table table : db.getTables()) { +<<<<<<< HEAD:fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java unprotectDropTable(db, table.getId(), isForeDrop); +======= + table.writeLock(); + try { + unprotectDropTable(db, table); + } finally { + table.writeUnlock(); + } + +>>>>>>> use table lock to replace db lock:fe/src/main/java/org/apache/doris/catalog/Catalog.java } } @@ -2851,18 +2861,17 @@ public void recoverTable(RecoverTableStmt recoverStmt) throws DdlException { } String tableName = recoverStmt.getTableName(); - db.writeLock(); + Table table = db.getTable(tableName); + if (table != null) { + ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); + } + table.writeLock(); try { - Table table = db.getTable(tableName); - if (table != null) { - ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); - } - if (!Catalog.getCurrentRecycleBin().recoverTable(db, tableName)) { ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); } } finally { - db.writeUnlock(); + table.writeUnlock(); } } @@ -2875,18 +2884,17 @@ public void recoverPartition(RecoverPartitionStmt recoverStmt) throws DdlExcepti } String tableName = recoverStmt.getTableName(); - db.writeLock(); - try { - Table table = db.getTable(tableName); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } - - if (table.getType() != TableType.OLAP) { - throw new DdlException("table[" + tableName + "] is not OLAP table"); - } - OlapTable olapTable = (OlapTable) table; + Table table = db.getTable(tableName); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } + if (table.getType() != TableType.OLAP) { + throw new DdlException("table[" + tableName + "] is not OLAP table"); + } + OlapTable olapTable = (OlapTable) table; + olapTable.writeLock(); + try { String partitionName = recoverStmt.getPartitionName(); if (olapTable.getPartition(partitionName) != null) { throw new DdlException("partition[" + partitionName + "] already exist in table[" + tableName + "]"); @@ -2894,7 +2902,7 @@ public void recoverPartition(RecoverPartitionStmt recoverStmt) throws DdlExcepti Catalog.getCurrentRecycleBin().recoverPartition(db.getId(), olapTable, partitionName); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } @@ -3419,16 +3427,19 @@ public void dropPartition(Database db, OlapTable olapTable, DropPartitionClause public void replayDropPartition(DropPartitionInfo info) { Database db = this.getDb(info.getDbId()); - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(info.getTableId()); + if (olapTable == null) { + return; + } + olapTable.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(info.getTableId()); if (info.isTempPartition()) { olapTable.dropTempPartition(info.getPartitionName(), true); } else { olapTable.dropPartition(info.getDbId(), info.getPartitionName(), info.isForceDrop()); } } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } @@ -4320,14 +4331,7 @@ private void createTablets(String clusterName, MaterializedIndex index, ReplicaS // if this is a colocate table, try to get backend seqs from colocation index. Database db = Catalog.getCurrentCatalog().getDb(tabletMeta.getDbId()); groupId = colocateIndex.getGroup(tabletMeta.getTableId()); - // Use db write lock here to make sure the backendsPerBucketSeq is consistent when the backendsPerBucketSeq is updating. - // This lock will release very fast. - db.writeLock(); - try { - backendsPerBucketSeq = colocateIndex.getBackendsPerBucketSeq(groupId); - } finally { - db.writeUnlock(); - } + backendsPerBucketSeq = colocateIndex.getBackendsPerBucketSeq(groupId); } // chooseBackendsArbitrary is true, means this may be the first table of colocation group, @@ -4411,19 +4415,17 @@ public void dropTable(DropTableStmt stmt) throws DdlException { ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); } - Table table = null; + Table table = db.getTable(tableName); + if (table == null) { + if (stmt.isSetIfExists()) { + LOG.info("drop table[{}] which does not exist", tableName); + return; + } else { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } + } db.writeLock(); try { - table = db.getTable(tableName); - if (table == null) { - if (stmt.isSetIfExists()) { - LOG.info("drop table[{}] which does not exist", tableName); - return; - } else { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } - } - // Check if a view if (stmt.isView()) { if (!(table instanceof View)) { @@ -4442,25 +4444,23 @@ public void dropTable(DropTableStmt stmt) throws DdlException { " please use \"DROP table FORCE\"."); } } - unprotectDropTable(db, table.getId(), stmt.isForceDrop()); DropInfo info = new DropInfo(db.getId(), table.getId(), -1L, stmt.isForceDrop()); + table.writeLock(); + try { + unprotectDropTable(db, table, stmt.isForceDrop()); + } finally { + table.writeUnlock(); + } editLog.logDropTable(info); } finally { db.writeUnlock(); } - LOG.info("finished dropping table: {} from db: {}, is force: {}", tableName, dbName, stmt.isForceDrop()); } - public boolean unprotectDropTable(Database db, long tableId, boolean isForceDrop) { - Table table = db.getTable(tableId); - // delete from db meta - if (table == null) { - return false; - } - + public boolean unprotectDropTable(Database db, Table table, boolean isForceDrop) { if (table.getType() == TableType.ELASTICSEARCH) { - esRepository.deRegisterTable(tableId); + esRepository.deRegisterTable(table.getId()); } else if (table.getType() == TableType.OLAP) { // drop all temp partitions of this table, so that there is no temp partitions in recycle bin, // which make things easier. @@ -4477,9 +4477,19 @@ public boolean unprotectDropTable(Database db, long tableId, boolean isForceDrop } public void replayDropTable(Database db, long tableId, boolean isForceDrop) { + Table table = db.getTable(tableId); + // delete from db meta + if (table == null) { + return; + } db.writeLock(); try { - unprotectDropTable(db, tableId, isForceDrop); + table.writeLock(); + try { + unprotectDropTable(db, table, isForceDrop); + } finally { + table.writeUnlock(); + } } finally { db.writeUnlock(); } @@ -4539,21 +4549,29 @@ private void unprotectUpdateReplica(ReplicaPersistInfo info) { public void replayAddReplica(ReplicaPersistInfo info) { Database db = getDb(info.getDbId()); - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(info.getTableId()); + if (olapTable == null) { + return; + } + olapTable.writeLock(); try { unprotectAddReplica(info); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } public void replayUpdateReplica(ReplicaPersistInfo info) { Database db = getDb(info.getDbId()); - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(info.getTableId()); + if (olapTable == null) { + return; + } + olapTable.writeLock(); try { unprotectUpdateReplica(info); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } @@ -4568,11 +4586,15 @@ public void unprotectDeleteReplica(ReplicaPersistInfo info) { public void replayDeleteReplica(ReplicaPersistInfo info) { Database db = getDb(info.getDbId()); - db.writeLock(); + OlapTable tbl = (OlapTable) db.getTable(info.getTableId()); + if (tbl == null) { + return; + } + tbl.writeLock(); try { unprotectDeleteReplica(info); } finally { - db.writeUnlock(); + tbl.writeUnlock(); } } @@ -4698,45 +4720,46 @@ public HashMap getPartitionIdToStorageMediumMap() { LOG.warn("db {} does not exist while doing backend report", dbId); continue; } - + List
tableList = null; db.readLock(); try { - for (Table table : db.getTables()) { - if (table.getType() != TableType.OLAP) { - continue; - } - - long tableId = table.getId(); - OlapTable olapTable = (OlapTable) table; - olapTable.readLock(); - try { - PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - for (Partition partition : olapTable.getAllPartitions()) { - long partitionId = partition.getId(); - DataProperty dataProperty = partitionInfo.getDataProperty(partition.getId()); - Preconditions.checkNotNull(dataProperty, partition.getName() + ", pId:" + partitionId + ", db: " + dbId + ", tbl: " + tableId); - if (dataProperty.getStorageMedium() == TStorageMedium.SSD - && dataProperty.getCooldownTimeMs() < currentTimeMs) { - // expire. change to HDD. - // record and change when holding write lock - Multimap multimap = changedPartitionsMap.get(dbId); - if (multimap == null) { - multimap = HashMultimap.create(); - changedPartitionsMap.put(dbId, multimap); - } - multimap.put(tableId, partitionId); - } else { - storageMediumMap.put(partitionId, dataProperty.getStorageMedium()); - } - } // end for partitions - } finally { - olapTable.readUnlock(); - } - - } // end for tables + tableList = db.getTables(); } finally { db.readUnlock(); } + + for (Table table : tableList) { + if (table.getType() != TableType.OLAP) { + continue; + } + + long tableId = table.getId(); + OlapTable olapTable = (OlapTable) table; + olapTable.readLock(); + try { + PartitionInfo partitionInfo = olapTable.getPartitionInfo(); + for (Partition partition : olapTable.getAllPartitions()) { + long partitionId = partition.getId(); + DataProperty dataProperty = partitionInfo.getDataProperty(partition.getId()); + Preconditions.checkNotNull(dataProperty, partition.getName() + ", pId:" + partitionId + ", db: " + dbId + ", tbl: " + tableId); + if (dataProperty.getStorageMedium() == TStorageMedium.SSD + && dataProperty.getCooldownTimeMs() < currentTimeMs) { + // expire. change to HDD. + // record and change when holding write lock + Multimap multimap = changedPartitionsMap.get(dbId); + if (multimap == null) { + multimap = HashMultimap.create(); + changedPartitionsMap.put(dbId, multimap); + } + multimap.put(tableId, partitionId); + } else { + storageMediumMap.put(partitionId, dataProperty.getStorageMedium()); + } + } // end for partitions + } finally { + olapTable.readUnlock(); + } + } // end for tables } // end for dbs // handle data property changed @@ -4748,20 +4771,20 @@ public HashMap getPartitionIdToStorageMediumMap() { } Multimap tableIdToPartitionIds = changedPartitionsMap.get(dbId); - // use try lock to avoid blocking a long time. - // if block too long, backend report rpc will timeout. - if (!db.tryWriteLock(Database.TRY_LOCK_TIMEOUT_MS, TimeUnit.MILLISECONDS)) { - LOG.warn("try get db {} writelock but failed when checking backend storage medium", dbId); - continue; - } - Preconditions.checkState(db.isWriteLockHeldByCurrentThread()); - try { - for (Long tableId : tableIdToPartitionIds.keySet()) { - Table table = db.getTable(tableId); - if (table == null) { - continue; - } - OlapTable olapTable = (OlapTable) table; + for (Long tableId : tableIdToPartitionIds.keySet()) { + Table table = db.getTable(tableId); + if (table == null) { + continue; + } + OlapTable olapTable = (OlapTable) table; + // use try lock to avoid blocking a long time. + // if block too long, backend report rpc will timeout. + if (!olapTable.tryWriteLock(Table.TRY_LOCK_TIMEOUT_MS, TimeUnit.MILLISECONDS)) { + LOG.warn("try get db {} writelock but failed when checking backend storage medium", dbId); + continue; + } + Preconditions.checkState(olapTable.isWriteLockHeldByCurrentThread()); + try { PartitionInfo partitionInfo = olapTable.getPartitionInfo(); Collection partitionIds = tableIdToPartitionIds.get(tableId); @@ -4789,10 +4812,10 @@ public HashMap getPartitionIdToStorageMediumMap() { editLog.logModifyPartition(info); } } // end for partitions - } // end for tables - } finally { - db.writeUnlock(); - } + } finally { + olapTable.writeUnlock(); + } + } // end for tables } // end for dbs return storageMediumMap; } @@ -6682,7 +6705,7 @@ public void replayBackendTabletsInfo(BackendTabletsInfo backendTabletsInfo) { // Convert table's distribution type from random to hash. // random distribution is no longer supported. public void convertDistributionType(Database db, OlapTable tbl) throws DdlException { - db.writeLock(); + tbl.writeLock(); try { if (!tbl.convertRandomDistributionToHashDistribution()) { throw new DdlException("Table " + tbl.getName() + " is not random distributed"); @@ -6691,19 +6714,22 @@ public void convertDistributionType(Database db, OlapTable tbl) throws DdlExcept editLog.logModifyDistributionType(tableInfo); LOG.info("finished to modify distribution type of table: " + tbl.getName()); } finally { - db.writeUnlock(); + tbl.writeUnlock(); } } public void replayConvertDistributionType(TableInfo tableInfo) { Database db = getDb(tableInfo.getDbId()); - db.writeLock(); + OlapTable tbl = (OlapTable) db.getTable(tableInfo.getTableId()); + if (tbl == null) { + return; + } + tbl.writeLock(); try { - OlapTable tbl = (OlapTable) db.getTable(tableInfo.getTableId()); tbl.convertRandomDistributionToHashDistribution(); LOG.info("replay modify distribution type of table: " + tbl.getName()); } finally { - db.writeUnlock(); + tbl.writeUnlock(); } } @@ -6715,18 +6741,18 @@ public void replaceTempPartition(Database db, String tableName, ReplacePartition List tempPartitionNames = clause.getTempPartitionNames(); boolean isStrictRange = clause.isStrictRange(); boolean useTempPartitionName = clause.useTempPartitionName(); - db.writeLock(); - try { - Table table = db.getTable(tableName); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } + Table table = db.getTable(tableName); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } - if (table.getType() != TableType.OLAP) { - throw new DdlException("Table[" + tableName + "] is not OLAP table"); - } + if (table.getType() != TableType.OLAP) { + throw new DdlException("Table[" + tableName + "] is not OLAP table"); + } - OlapTable olapTable = (OlapTable) table; + OlapTable olapTable = (OlapTable) table; + olapTable.writeLock(); + try { // check partition exist for (String partName : partitionNames) { if (!olapTable.checkPartitionNameExist(partName, false)) { @@ -6748,7 +6774,7 @@ public void replaceTempPartition(Database db, String tableName, ReplacePartition LOG.info("finished to replace partitions {} with temp partitions {} from table: {}", clause.getPartitionNames(), clause.getTempPartitionNames(), tableName); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } @@ -6757,12 +6783,13 @@ public void replayReplaceTempPartition(ReplacePartitionOperationLog replaceTempP if (db == null) { return; } - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(replaceTempPartitionLog.getTblId()); + if (olapTable == null) { + return; + } + + olapTable.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(replaceTempPartitionLog.getTblId()); - if (olapTable == null) { - return; - } olapTable.replaceTempPartitions(replaceTempPartitionLog.getPartitions(), replaceTempPartitionLog.getTempPartitions(), replaceTempPartitionLog.isStrictRange(), @@ -6770,7 +6797,7 @@ public void replayReplaceTempPartition(ReplacePartitionOperationLog replaceTempP } catch (DdlException e) { LOG.warn("should not happen. {}", e); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 325b9449914d43..3e33e150e0cdfe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -132,7 +132,7 @@ public boolean tryReadLock(long timeout, TimeUnit unit) { return false; } } - + public void readUnlock() { this.rwLock.readLock().unlock(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java index 6d1d648ece0a48..3221b0e1c9b837 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java @@ -49,6 +49,10 @@ public class Table extends MetaObject implements Writable { private static final Logger LOG = LogManager.getLogger(Table.class); + // empirical value. + // assume that the time a lock is held by thread is less then 100ms + public static final long TRY_LOCK_TIMEOUT_MS = 100L; + public enum TableType { MYSQL, ODBC, From cffcbbd2d4b384372321dd4061cf3b8395892744 Mon Sep 17 00:00:00 2001 From: caiconghui Date: Thu, 4 Jun 2020 00:59:13 +0800 Subject: [PATCH 06/50] Continue to use table lock to replace db lock --- .../org/apache/doris/catalog/Catalog.java | 72 ++++++----- .../doris/catalog/CatalogRecycleBin.java | 2 +- .../apache/doris/catalog/TabletStatMgr.java | 28 +++-- .../apache/doris/clone/TabletSchedCtx.java | 40 +++--- .../apache/doris/clone/TabletScheduler.java | 13 +- .../consistency/CheckConsistencyJob.java | 18 +-- .../doris/consistency/ConsistencyChecker.java | 6 +- .../doris/http/rest/RowCountAction.java | 23 ++-- .../doris/http/rest/TableRowCountAction.java | 26 ++-- .../main/java/org/apache/doris/load/Load.java | 51 ++++---- .../org/apache/doris/master/MasterImpl.java | 33 ++--- .../apache/doris/master/ReportHandler.java | 118 +++++++++++++++--- 12 files changed, 273 insertions(+), 157 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index a46e505452b9d9..f89973b77104e1 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -3450,12 +3450,12 @@ public void replayErasePartition(long partitionId) throws DdlException { public void replayRecoverPartition(RecoverInfo info) { long dbId = info.getDbId(); Database db = getDb(dbId); - db.writeLock(); + Table table = db.getTable(info.getTableId()); + table.writeLock(); try { - Table table = db.getTable(info.getTableId()); Catalog.getCurrentRecycleBin().replayRecoverPartition((OlapTable) table, info.getPartitionId()); } finally { - db.writeUnlock(); + table.writeUnlock(); } } @@ -5162,9 +5162,13 @@ public void replayRenameTable(TableInfo tableInfo) throws DdlException { Table table = db.getTable(tableId); String tableName = table.getName(); db.dropTable(tableName); - table.setName(newTableName); + table.writeLock(); + try { + table.setName(newTableName); + } finally { + table.writeUnlock(); + } db.createTable(table); - LOG.info("replay rename table[{}] to {}", tableName, newTableName); } finally { db.writeUnlock(); @@ -5255,16 +5259,16 @@ public void replayModifyTableColocate(TablePropertyInfo info) { Map properties = info.getPropertyMap(); Database db = getDb(info.getGroupId().dbId); - db.writeLock(); + OlapTable table = (OlapTable) db.getTable(tableId); + table.writeLock(); try { - OlapTable table = (OlapTable) db.getTable(tableId); modifyTableColocate(db, table, properties.get(PropertyAnalyzer.PROPERTIES_COLOCATE_WITH), true, info.getGroupId()); } catch (DdlException e) { // should not happen LOG.warn("failed to replay modify table colocate", e); } finally { - db.writeUnlock(); + table.writeUnlock(); } } @@ -5310,9 +5314,9 @@ public void replayRenameRollup(TableInfo tableInfo) throws DdlException { String newRollupName = tableInfo.getNewRollupName(); Database db = getDb(dbId); - db.writeLock(); + OlapTable table = (OlapTable) db.getTable(tableId); + table.writeLock(); try { - OlapTable table = (OlapTable) db.getTable(tableId); String rollupName = table.getIndexNameById(indexId); Map indexNameToIdMap = table.getIndexNameToId(); indexNameToIdMap.remove(rollupName); @@ -5320,7 +5324,7 @@ public void replayRenameRollup(TableInfo tableInfo) throws DdlException { LOG.info("replay rename rollup[{}] to {}", rollupName, newRollupName); } finally { - db.writeUnlock(); + table.writeUnlock(); } } @@ -5366,15 +5370,14 @@ public void replayRenamePartition(TableInfo tableInfo) throws DdlException { String newPartitionName = tableInfo.getNewPartitionName(); Database db = getDb(dbId); - db.writeLock(); + OlapTable table = (OlapTable) db.getTable(tableId); + table.writeLock(); try { - OlapTable table = (OlapTable) db.getTable(tableId); Partition partition = table.getPartition(partitionId); table.renamePartition(partition.getName(), newPartitionName); - LOG.info("replay rename partition[{}] to {}", partition.getName(), newPartitionName); } finally { - db.writeUnlock(); + table.writeUnlock(); } } @@ -5491,9 +5494,9 @@ public void replayModifyTableProperty(short opCode, ModifyTablePropertyOperation Map properties = info.getProperties(); Database db = getDb(dbId); - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + olapTable.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); TableProperty tableProperty = olapTable.getTableProperty(); if (tableProperty == null) { olapTable.setTableProperty(new TableProperty(properties).buildProperty(opCode)); @@ -5509,7 +5512,7 @@ public void replayModifyTableProperty(short opCode, ModifyTablePropertyOperation } } } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } @@ -6484,14 +6487,13 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti // all partitions are created successfully, try to replace the old partitions. // before replacing, we need to check again. - // Things may be changed outside the database lock. - db.writeLock(); + // Things may be changed outside the table lock. + OlapTable olapTable = (OlapTable) db.getTable(copiedTbl.getId()); + if (olapTable == null) { + throw new DdlException("Table[" + copiedTbl.getName() + "] is dropped"); + } + table.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(copiedTbl.getId()); - if (olapTable == null) { - throw new DdlException("Table[" + copiedTbl.getName() + "] is dropped"); - } - if (olapTable.getState() != OlapTableState.NORMAL) { throw new DdlException("Table' state is not NORMAL: " + olapTable.getState()); } @@ -6537,7 +6539,7 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti truncateEntireTable); editLog.logTruncateTable(info); } finally { - db.writeUnlock(); + table.writeUnlock(); } LOG.info("finished to truncate table {}, partitions: {}", @@ -6570,9 +6572,9 @@ private void truncateTableInternal(OlapTable olapTable, List newParti public void replayTruncateTable(TruncateTableInfo info) { Database db = getDb(info.getDbId()); - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(info.getTblId()); + olapTable.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(info.getTblId()); truncateTableInternal(olapTable, info.getPartitions(), info.isEntireTable()); if (!Catalog.isCheckpointThread()) { @@ -6598,7 +6600,7 @@ public void replayTruncateTable(TruncateTableInfo info) { } } } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } @@ -6875,10 +6877,16 @@ private void setReplicaStatusInternal(long tabletId, long backendId, ReplicaStat long dbId = meta.getDbId(); Database db = getDb(dbId); if (db == null) { - LOG.info("database {} of tablet {} does not exist", dbId, tabletId); + LOG.info("tablet {} in database does not exist", tabletId, dbId); return; } - db.writeLock(); + long tableId = meta.getTableId(); + Table table = db.getTable(tableId); + if (table == null) { + LOG.info("tablet {} of table {} in database {} does not exist", tabletId, tableId, dbId); + return; + } + table.writeLock(); try { Replica replica = tabletInvertedIndex.getReplica(tabletId, backendId); if (replica == null) { @@ -6896,7 +6904,7 @@ private void setReplicaStatusInternal(long tabletId, long backendId, ReplicaStat } } } finally { - db.writeUnlock(); + table.writeUnlock(); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java index 4877f5aac0f4cd..db3185ea97bbe6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java @@ -551,7 +551,7 @@ public synchronized void recoverPartition(long dbId, OlapTable table, String par LOG.info("recover partition[{}]", partitionId); } - // The caller should keep db write lock + // The caller should keep table write lock public synchronized void replayRecoverPartition(OlapTable table, long partitionId) { Iterator> iterator = idToPartition.entrySet().iterator(); while (iterator.hasNext()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java index a63784becf2b63..92543515c02d97 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java @@ -76,7 +76,7 @@ protected void runAfterCatalogReady() { } } LOG.info("finished to get tablet stat of all backends. cost: {} ms", - (System.currentTimeMillis() - start)); + (System.currentTimeMillis() - start)); // after update replica in all backends, update index row num start = System.currentTimeMillis(); @@ -86,14 +86,22 @@ protected void runAfterCatalogReady() { if (db == null) { continue; } - db.writeLock(); + List
tableList = null; + db.readLock(); try { - for (Table table : db.getTables()) { - if (table.getType() != TableType.OLAP) { - continue; - } + tableList = db.getTables(); + } finally { + db.readUnlock(); + } - OlapTable olapTable = (OlapTable) table; + for (Table table : tableList) { + if (table.getType() != TableType.OLAP) { + continue; + } + + OlapTable olapTable = (OlapTable) table; + table.writeLock(); + try { for (Partition partition : olapTable.getAllPartitions()) { long version = partition.getVisibleVersion(); long versionHash = partition.getVisibleVersionHash(); @@ -114,13 +122,13 @@ protected void runAfterCatalogReady() { } // end for partitions LOG.debug("finished to set row num for table: {} in database: {}", table.getName(), db.getFullName()); + } finally { + table.writeUnlock(); } - } finally { - db.writeUnlock(); } } LOG.info("finished to update index row num of all databases. cost: {} ms", - (System.currentTimeMillis() - start)); + (System.currentTimeMillis() - start)); } private void updateTabletStat(Long beId, TTabletStatResult result) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java index 723ef7c3c22af6..fd3668938cc3b2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java @@ -24,6 +24,7 @@ import org.apache.doris.catalog.Partition; 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.Tablet.TabletStatus; import org.apache.doris.clone.SchedException.Status; @@ -620,19 +621,22 @@ public void releaseResource(TabletScheduler tabletScheduler, boolean reserveTabl // clear all CLONE replicas Database db = Catalog.getCurrentCatalog().getDb(dbId); if (db != null) { - db.writeLock(); - try { - List cloneReplicas = Lists.newArrayList(); - tablet.getReplicas().stream().filter(r -> r.getState() == ReplicaState.CLONE).forEach(r -> { - cloneReplicas.add(r); - }); + Table table = db.getTable(tblId); + if (table != null) { + table.writeLock(); + try { + List cloneReplicas = Lists.newArrayList(); + tablet.getReplicas().stream().filter(r -> r.getState() == ReplicaState.CLONE).forEach(r -> { + cloneReplicas.add(r); + }); - for (Replica cloneReplica : cloneReplicas) { - tablet.deleteReplica(cloneReplica); - } + for (Replica cloneReplica : cloneReplicas) { + tablet.deleteReplica(cloneReplica); + } - } finally { - db.writeUnlock(); + } finally { + table.writeUnlock(); + } } } } @@ -777,13 +781,13 @@ public void finishCloneTask(CloneTask cloneTask, TFinishTaskRequest request) if (db == null) { throw new SchedException(Status.UNRECOVERABLE, "db does not exist"); } - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(tblId); + if (olapTable == null) { + throw new SchedException(Status.UNRECOVERABLE, "tbl does not exist"); + } + + olapTable.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(tblId); - if (olapTable == null) { - throw new SchedException(Status.UNRECOVERABLE, "tbl does not exist"); - } - Partition partition = olapTable.getPartition(partitionId); if (partition == null) { throw new SchedException(Status.UNRECOVERABLE, "partition does not exist"); @@ -884,7 +888,7 @@ public void finishCloneTask(CloneTask cloneTask, TFinishTaskRequest request) } throw e; } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } if (request.isSetCopySize()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java index 349db1b994c77a..58453ead151a05 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java @@ -458,13 +458,12 @@ private void scheduleTablet(TabletSchedCtx tabletCtx, AgentBatchTask batchTask) } Pair statusPair; - db.writeLock(); + OlapTable tbl = (OlapTable) db.getTable(tabletCtx.getTblId()); + if (tbl == null) { + throw new SchedException(Status.UNRECOVERABLE, "tbl does not exist"); + } + tbl.writeLock(); try { - OlapTable tbl = (OlapTable) db.getTable(tabletCtx.getTblId()); - if (tbl == null) { - throw new SchedException(Status.UNRECOVERABLE, "tbl does not exist"); - } - boolean isColocateTable = colocateTableIndex.isColocateTable(tbl.getId()); OlapTableState tableState = tbl.getState(); @@ -553,7 +552,7 @@ private void scheduleTablet(TabletSchedCtx tabletCtx, AgentBatchTask batchTask) handleTabletByTypeAndStatus(statusPair.first, tabletCtx, batchTask); } finally { - db.writeUnlock(); + tbl.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java b/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java index 883cf45cf7806a..aa183b18cc327c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java @@ -215,11 +215,11 @@ public boolean sendTasks() { if (state != JobState.RUNNING) { // failed to send task. set tablet's checked version and version hash to avoid choosing it again - db.writeLock(); + table.writeLock(); try { tablet.setCheckedVersion(checkedVersion, checkedVersionHash); } finally { - db.writeUnlock(); + table.writeUnlock(); } return false; } @@ -260,13 +260,13 @@ public synchronized int tryFinishJob() { } boolean isConsistent = true; - db.writeLock(); + Table table = db.getTable(tabletMeta.getTableId()); + if (table == null) { + LOG.warn("table[{}] does not exist", tabletMeta.getTableId()); + return -1; + } + table.writeLock(); try { - Table table = db.getTable(tabletMeta.getTableId()); - if (table == null) { - LOG.warn("table[{}] does not exist", tabletMeta.getTableId()); - return -1; - } OlapTable olapTable = (OlapTable) table; Partition partition = olapTable.getPartition(tabletMeta.getPartitionId()); @@ -369,7 +369,7 @@ public synchronized int tryFinishJob() { return 1; } finally { - db.writeUnlock(); + table.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java b/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java index 660fd2be91864e..f60073b66f122d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java @@ -371,9 +371,9 @@ public void handleFinishedConsistencyCheck(CheckConsistencyTask task, long check public void replayFinishConsistencyCheck(ConsistencyCheckInfo info, Catalog catalog) { Database db = catalog.getDb(info.getDbId()); - db.writeLock(); + OlapTable table = (OlapTable) db.getTable(info.getTableId()); + table.writeLock(); try { - OlapTable table = (OlapTable) db.getTable(info.getTableId()); Partition partition = table.getPartition(info.getPartitionId()); MaterializedIndex index = partition.getIndex(info.getIndexId()); Tablet tablet = index.getTablet(info.getTabletId()); @@ -388,7 +388,7 @@ public void replayFinishConsistencyCheck(ConsistencyCheckInfo info, Catalog cata tablet.setIsConsistent(info.isConsistent()); } finally { - db.writeUnlock(); + table.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/RowCountAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/RowCountAction.java index bc7003c014c1ae..c9b9f98ce8e643 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/RowCountAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/RowCountAction.java @@ -78,17 +78,18 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response if (db == null) { throw new DdlException("Database[" + dbName + "] does not exist"); } - db.writeLock(); + + Table table = db.getTable(tableName); + if (table == null) { + throw new DdlException("Table[" + tableName + "] does not exist"); + } + + if (table.getType() != TableType.OLAP) { + throw new DdlException("Table[" + tableName + "] is not OLAP table"); + } + + table.writeLock(); try { - Table table = db.getTable(tableName); - if (table == null) { - throw new DdlException("Table[" + tableName + "] does not exist"); - } - - if (table.getType() != TableType.OLAP) { - throw new DdlException("Table[" + tableName + "] is not OLAP table"); - } - OlapTable olapTable = (OlapTable) table; for (Partition partition : olapTable.getAllPartitions()) { long version = partition.getVisibleVersion(); @@ -110,7 +111,7 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response } // end for indices } // end for partitions } finally { - db.writeUnlock(); + table.writeUnlock(); } // to json response diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableRowCountAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableRowCountAction.java index 92a713445f646b..b255625a8fddf9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableRowCountAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableRowCountAction.java @@ -79,23 +79,25 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response if (db == null) { throw new DorisHttpException(HttpResponseStatus.NOT_FOUND, "Database [" + dbName + "] " + "does not exists"); } - db.writeLock(); + + Table table = db.getTable(tableName); + if (table == null) { + throw new DorisHttpException(HttpResponseStatus.NOT_FOUND, "Table [" + tableName + "] " + "does not exists"); + } + // just only support OlapTable, ignore others such as ESTable + if (!(table instanceof OlapTable)) { + // Forbidden + throw new DorisHttpException(HttpResponseStatus.FORBIDDEN, "Table [" + tableName + "] " + + "is not a OlapTable, only support OlapTable currently"); + } + + table.writeLock(); try { - Table table = db.getTable(tableName); - if (table == null) { - throw new DorisHttpException(HttpResponseStatus.NOT_FOUND, "Table [" + tableName + "] " + "does not exists"); - } - // just only support OlapTable, ignore others such as ESTable - if (!(table instanceof OlapTable)) { - // Forbidden - throw new DorisHttpException(HttpResponseStatus.FORBIDDEN, "Table [" + tableName + "] " - + "is not a OlapTable, only support OlapTable currently"); - } OlapTable olapTable = (OlapTable) table; resultMap.put("status", 200); resultMap.put("size", olapTable.proximateRowCount()); } finally { - db.writeUnlock(); + table.writeUnlock(); } } catch (DorisHttpException e) { // status code should conforms to HTTP semantic diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java index 510c286b0d75b4..48b01ce9a52054 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java @@ -2865,6 +2865,7 @@ public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType ca long jobId = job.getId(); long dbId = job.getDbId(); Database db = Catalog.getCurrentCatalog().getDb(dbId); + Table table = db.getTable(job.getTableId()); String errMsg = msg; if (db == null) { // if db is null, update job to cancelled @@ -3284,19 +3285,19 @@ public void unprotectDelete(DeleteInfo deleteInfo, Database db) { public void replayFinishAsyncDeleteJob(AsyncDeleteJob deleteJob, Catalog catalog) { Database db = catalog.getDb(deleteJob.getDbId()); - db.writeLock(); + writeLock(); try { - writeLock(); - try { - // Update database information - Map replicaInfos = deleteJob.getReplicaPersistInfos(); - if (replicaInfos != null) { - for (ReplicaPersistInfo info : replicaInfos.values()) { - OlapTable table = (OlapTable) db.getTable(info.getTableId()); - if (table == null) { - LOG.warn("the table[{}] is missing", info.getIndexId()); - continue; - } + // Update database information + Map replicaInfos = deleteJob.getReplicaPersistInfos(); + if (replicaInfos != null) { + for (ReplicaPersistInfo info : replicaInfos.values()) { + OlapTable table = (OlapTable) db.getTable(info.getTableId()); + if (table == null) { + LOG.warn("the table[{}] is missing", info.getIndexId()); + continue; + } + table.writeLock(); + try { Partition partition = table.getPartition(info.getPartitionId()); if (partition == null) { LOG.warn("the partition[{}] is missing", info.getIndexId()); @@ -3319,32 +3320,36 @@ public void replayFinishAsyncDeleteJob(AsyncDeleteJob deleteJob, Catalog catalog continue; } replica.updateVersionInfo(info.getVersion(), info.getVersionHash(), - info.getDataSize(), info.getRowCount()); + info.getDataSize(), info.getRowCount()); + + } finally { + table.writeUnlock(); } } - } finally { - writeUnlock(); } } finally { - db.writeUnlock(); + writeUnlock(); } - removeDeleteJobAndSetState(deleteJob); LOG.info("unprotected finish asyncDeleteJob: {}", deleteJob.getJobId()); } public void replayDelete(DeleteInfo deleteInfo, Catalog catalog) { Database db = catalog.getDb(deleteInfo.getDbId()); - db.writeLock(); + OlapTable table = (OlapTable) db.getTable(deleteInfo.getTableId()); + if (table == null) { + return; + } + writeLock(); try { - writeLock(); + table.writeLock(); try { unprotectDelete(deleteInfo, db); } finally { - writeUnlock(); + table.writeUnlock(); } } finally { - db.writeUnlock(); + writeUnlock(); } } @@ -3616,7 +3621,7 @@ public void delete(DeleteStmt stmt) throws DdlException { long startDeleteTime = System.currentTimeMillis(); long timeout = loadDeleteJob.getDeleteJobTimeout(); while (true) { - db.writeLock(); + table.writeLock(); try { if (loadDeleteJob.getState() == JobState.FINISHED || loadDeleteJob.getState() == JobState.CANCELLED) { @@ -3633,7 +3638,7 @@ public void delete(DeleteStmt stmt) throws DdlException { } } } finally { - db.writeUnlock(); + table.writeUnlock(); } Thread.sleep(1000); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java b/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java index 789fe33fc37d2d..c28581600d6efa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java @@ -334,14 +334,15 @@ private void finishRealtimePush(AgentTask task, TFinishTaskRequest request) { return; } LOG.debug("push report state: {}", pushState.name()); - - db.writeLock(); - try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - throw new MetaNotFoundException("cannot find table[" + tableId + "] when push finished"); - } + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + AgentTaskQueue.removeTask(backendId, TTaskType.REALTIME_PUSH, signature); + LOG.warn("finish push replica error, cannot find table[" + tableId + "] when push finished"); + return; + } + olapTable.writeLock(); + try { Partition partition = olapTable.getPartition(partitionId); if (partition == null) { throw new MetaNotFoundException("cannot find partition[" + partitionId + "] when push finished"); @@ -421,7 +422,7 @@ private void finishRealtimePush(AgentTask task, TFinishTaskRequest request) { AgentTaskQueue.removeTask(backendId, TTaskType.REALTIME_PUSH, signature); LOG.warn("finish push replica error", e); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } @@ -548,13 +549,15 @@ private void finishPush(AgentTask task, TFinishTaskRequest request) { LOG.debug("push report state: {}", pushState.name()); - db.writeLock(); - try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - throw new MetaNotFoundException("cannot find table[" + tableId + "] when push finished"); - } + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + AgentTaskQueue.removeTask(backendId, TTaskType.REALTIME_PUSH, signature); + LOG.warn("finish push replica error, cannot find table[" + tableId + "] when push finished"); + return; + } + olapTable.writeLock(); + try { Partition partition = olapTable.getPartition(partitionId); if (partition == null) { throw new MetaNotFoundException("cannot find partition[" + partitionId + "] when push finished"); @@ -626,7 +629,7 @@ private void finishPush(AgentTask task, TFinishTaskRequest request) { pushTask.getPushType(), pushTask.getTaskType()); LOG.warn("finish push replica error", e); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java index 4c3197e1c48c6d..d8deda26f449a6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java @@ -381,6 +381,7 @@ private static void sync(Map backendTablets, ListMultimap backendTablets, ListMultimap>>>>>> Continue to use table lock to replace db lock:fe/src/main/java/org/apache/doris/master/ReportHandler.java + int syncCounter = 0; + List tabletIds = tabletSyncMap.get(dbId); + LOG.info("before sync tablets in db[{}]. report num: {}. backend[{}]", + dbId, tabletIds.size(), backendId); + List tabletMetaList = invertedIndex.getTabletMetaList(tabletIds); + for (int i = 0; i < tabletMetaList.size(); i++) { + TabletMeta tabletMeta = tabletMetaList.get(i); + if (tabletMeta == TabletInvertedIndex.NOT_EXIST_TABLET_META) { + continue; + } + long tabletId = tabletIds.get(i); + long tableId = tabletMeta.getTableId(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + continue; + } + olapTable.writeLock(); + try { long partitionId = tabletMeta.getPartitionId(); Partition partition = olapTable.getPartition(partitionId); if (partition == null) { @@ -494,11 +515,11 @@ private static void sync(Map backendTablets, ListMultimap tabletDeleteFromMeta if (db == null) { continue; } +<<<<<<< HEAD:fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java db.writeLock(); try { int deleteCounter = 0; @@ -527,7 +549,25 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta if (olapTable == null) { continue; } +======= +>>>>>>> Continue to use table lock to replace db lock:fe/src/main/java/org/apache/doris/master/ReportHandler.java + int deleteCounter = 0; + List tabletIds = tabletDeleteFromMeta.get(dbId); + List tabletMetaList = invertedIndex.getTabletMetaList(tabletIds); + for (int i = 0; i < tabletMetaList.size(); i++) { + TabletMeta tabletMeta = tabletMetaList.get(i); + if (tabletMeta == TabletInvertedIndex.NOT_EXIST_TABLET_META) { + continue; + } + long tabletId = tabletIds.get(i); + long tableId = tabletMeta.getTableId(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + continue; + } + olapTable.writeLock(); + try { long partitionId = tabletMeta.getPartitionId(); Partition partition = olapTable.getPartition(partitionId); if (partition == null) { @@ -580,7 +620,11 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta if (Config.recover_with_empty_tablet) { // only create this task if force recovery is true LOG.warn("tablet {} has only one replica {} on backend {}" +<<<<<<< HEAD:fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java + " and it is lost. create an empty replica to recover it", +======= + + "and it is lost. create an empty replica to recover it", +>>>>>>> Continue to use table lock to replace db lock:fe/src/main/java/org/apache/doris/master/ReportHandler.java tabletId, replica.getId(), backendId); MaterializedIndexMeta indexMeta = olapTable.getIndexMetaByIndexId(indexId); Set bfColumns = olapTable.getCopiedBfColumns(); @@ -600,7 +644,11 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta // just set this replica as bad if (replica.setBad(true)) { LOG.warn("tablet {} has only one replica {} on backend {}" +<<<<<<< HEAD:fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java + " and it is lost, set it as bad", +======= + + "and it is lost, set it as bad", +>>>>>>> Continue to use table lock to replace db lock:fe/src/main/java/org/apache/doris/master/ReportHandler.java tabletId, replica.getId(), backendId); BackendTabletsInfo tabletsInfo = new BackendTabletsInfo(backendId); tabletsInfo.setBad(true); @@ -636,11 +684,11 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta LOG.error("invalid situation. tablet[{}] is empty", tabletId); } } - } // end for tabletMetas - LOG.info("delete {} replica(s) from catalog in db[{}]", deleteCounter, dbId); - } finally { - db.writeUnlock(); - } + } finally { + olapTable.writeUnlock(); + } + } // end for tabletMetas + LOG.info("delete {} replica(s) from catalog in db[{}]", deleteCounter, dbId); } // end for dbs if (Config.recover_with_empty_tablet && createReplicaBatchTask.getTaskNum() > 0) { @@ -768,6 +816,7 @@ private static void handleRecoverTablet(ListMultimap tabletRecoveryM tabletRecoveryMap.size(), backendId); TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); +<<<<<<< HEAD:fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java BackendTabletsInfo backendTabletsInfo = new BackendTabletsInfo(backendId); backendTabletsInfo.setBad(true); for (Long dbId : tabletRecoveryMap.keySet()) { @@ -796,6 +845,40 @@ private static void handleRecoverTablet(ListMultimap tabletRecoveryM if (partition == null) { continue; } +======= + if (!forceRecovery) { + LOG.warn("force recovery is disable. try reset the tablets' version" + + " or set it as bad, and waiting clone"); + + BackendTabletsInfo backendTabletsInfo = new BackendTabletsInfo(backendId); + backendTabletsInfo.setBad(true); + for (Long dbId : tabletRecoveryMap.keySet()) { + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + continue; + } + + List tabletIds = tabletRecoveryMap.get(dbId); + List tabletMetaList = invertedIndex.getTabletMetaList(tabletIds); + for (int i = 0; i < tabletMetaList.size(); i++) { + TabletMeta tabletMeta = tabletMetaList.get(i); + if (tabletMeta == TabletInvertedIndex.NOT_EXIST_TABLET_META) { + continue; + } + long tabletId = tabletIds.get(i); + long tableId = tabletMeta.getTableId(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + continue; + } + olapTable.writeLock(); + try { + long partitionId = tabletMeta.getPartitionId(); + Partition partition = olapTable.getPartition(partitionId); + if (partition == null) { + continue; + } +>>>>>>> Continue to use table lock to replace db lock:fe/src/main/java/org/apache/doris/master/ReportHandler.java long indexId = tabletMeta.getIndexId(); MaterializedIndex index = partition.getIndex(indexId); @@ -855,6 +938,8 @@ private static void handleRecoverTablet(ListMultimap tabletRecoveryM break; } } + } finally { + olapTable.writeUnlock(); } } } finally { @@ -964,13 +1049,14 @@ private static void addReplica(long tabletId, TTabletInfo backendTabletInfo, lon if (db == null) { throw new MetaNotFoundException("db[" + dbId + "] does not exist"); } - db.writeLock(); - try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - throw new MetaNotFoundException("table[" + tableId + "] does not exist"); - } + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + throw new MetaNotFoundException("table[" + tableId + "] does not exist"); + } + + olapTable.writeLock(); + try { Partition partition = olapTable.getPartition(partitionId); if (partition == null) { throw new MetaNotFoundException("partition[" + partitionId + "] does not exist"); @@ -1066,7 +1152,7 @@ private static void addReplica(long tabletId, TTabletInfo backendTabletInfo, lon "replica is enough[" + tablet.getReplicas().size() + "-" + replicationNum + "]"); } } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } From 3763e582f230a76dd6b62fe6392b4a8929f433cb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Thu, 4 Jun 2020 18:00:30 +0800 Subject: [PATCH 07/50] finish to replace db lock by table lock --- .../java/org/apache/doris/alter/Alter.java | 1 - .../org/apache/doris/catalog/Catalog.java | 13 +- .../org/apache/doris/catalog/Database.java | 18 ++- .../org/apache/doris/load/DeleteHandler.java | 12 +- .../main/java/org/apache/doris/load/Load.java | 20 +-- .../org/apache/doris/load/LoadChecker.java | 17 ++- .../doris/load/loadv2/BrokerLoadJob.java | 2 +- .../apache/doris/master/ReportHandler.java | 131 +++--------------- .../org/apache/doris/qe/StmtExecutor.java | 2 +- .../doris/service/FrontendServiceImpl.java | 4 +- .../transaction/DatabaseTransactionMgr.java | 25 +++- .../transaction/GlobalTransactionMgr.java | 16 ++- .../org/apache/doris/catalog/TableTest.java | 24 +++- 13 files changed, 132 insertions(+), 153 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index f5f3f7ada59841..435ececd138b4e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -290,7 +290,6 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { Catalog.getCurrentSystemInfo().checkClusterCapacity(clusterName); db.checkQuota(); } - if (olapTable.getState() != OlapTableState.NORMAL) { throw new DdlException( "Table[" + table.getName() + "]'s state is not NORMAL. Do not allow doing ALTER ops"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index f89973b77104e1..bcfb8638d3daf2 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -2043,7 +2043,7 @@ public long saveDb(DataOutputStream dos, long checksum) throws IOException { if (!InfoSchemaDb.isInfoSchemaDb(dbName)) { checksum ^= entry.getKey(); db.readLock(); - List
tableList = db.getOrderedTablesById(); + List
tableList = db.getTablesOnIdOrder(); for (Table table : tableList) { table.readLock(); } @@ -2764,17 +2764,12 @@ public void dropDb(DropDbStmt stmt) throws DdlException { public void unprotectDropDb(Database db, boolean isForeDrop) { for (Table table : db.getTables()) { -<<<<<<< HEAD:fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java - unprotectDropTable(db, table.getId(), isForeDrop); -======= table.writeLock(); try { - unprotectDropTable(db, table); + unprotectDropTable(db, table, isForeDrop); } finally { table.writeUnlock(); } - ->>>>>>> use table lock to replace db lock:fe/src/main/java/org/apache/doris/catalog/Catalog.java } } @@ -6349,7 +6344,7 @@ public String dumpImage() { // lock all dbs for (Database db : lockedDbMap.values()) { db.readLock(); - List
tableList = db.getOrderedTablesById(); + List
tableList = db.getTablesOnIdOrder(); for (Table table : tableList) { table.readLock(); } @@ -6372,7 +6367,7 @@ public String dumpImage() { // unlock all load.readUnlock(); for (Database db : lockedDbMap.values()) { - List
tableList = db.getOrderedTablesById(); + List
tableList = db.getTablesOnIdOrder(); for (int i = tableList.size() - 1; i >= 0; i--) { tableList.get(i).readUnlock(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 3e33e150e0cdfe..2b67192b812a91 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -368,7 +368,7 @@ public List
getTables() { } // tables must get read or write table in fixed order to escape potential dead lock - public List
getOrderedTablesById() { + public List
getTablesOnIdOrder() { return idToTable.values().stream() .sorted(Comparator.comparing(Table::getId)) .collect(Collectors.toList()); @@ -384,10 +384,24 @@ public List
getViews() { return views; } + public List
getTablesOnIdOrderOrThrowException(List tableIdList) throws MetaNotFoundException { + List
tableList = Lists.newArrayList(); + for (Long tableId : tableIdList) { + Table table = idToTable.get(tableId); + if (table == null) { + throw new MetaNotFoundException("unknown table, table=" + tableId); + } + } + if (tableList.size() > 1) { + return tableList.stream().sorted(Comparator.comparing(Table::getId)).collect(Collectors.toList()); + } + return tableList; + } + public Set getTableNamesWithLock() { readLock(); try { - return new HashSet(this.nameToTable.keySet()); + return new HashSet<>(this.nameToTable.keySet()); } finally { readUnlock(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java index 08d40ed1c8e762..12edf755953ef1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java @@ -302,14 +302,14 @@ public void process(DeleteStmt stmt) throws DdlException, QueryStateException { cancelJob(deleteJob, CancelType.UNKNOWN, e.getMessage()); throw new DdlException(e.getMessage(), e); } - commitJob(deleteJob, db, timeoutMs); + commitJob(deleteJob, db, table, timeoutMs); break; default: Preconditions.checkState(false, "wrong delete job state: " + state.name()); break; } } else { - commitJob(deleteJob, db, timeoutMs); + commitJob(deleteJob, db, table, timeoutMs); } } finally { if (!FeConstants.runningUnitTest) { @@ -318,10 +318,10 @@ public void process(DeleteStmt stmt) throws DdlException, QueryStateException { } } - private void commitJob(DeleteJob job, Database db, long timeoutMs) throws DdlException, QueryStateException { + private void commitJob(DeleteJob job, Database db, Table table, long timeoutMs) throws DdlException, QueryStateException { TransactionStatus status = null; try { - unprotectedCommitJob(job, db, timeoutMs); + unprotectedCommitJob(job, db, table, timeoutMs); status = Catalog.getCurrentGlobalTransactionMgr(). getTransactionState(db.getId(), job.getTransactionId()).getTransactionStatus(); } catch (UserException e) { @@ -363,7 +363,7 @@ private void commitJob(DeleteJob job, Database db, long timeoutMs) throws DdlExc * @return * @throws UserException */ - private boolean unprotectedCommitJob(DeleteJob job, Database db, long timeoutMs) throws UserException { + private boolean unprotectedCommitJob(DeleteJob job, Database db, Table table, long timeoutMs) throws UserException { long transactionId = job.getTransactionId(); GlobalTransactionMgr globalTransactionMgr = Catalog.getCurrentGlobalTransactionMgr(); List tabletCommitInfos = new ArrayList(); @@ -379,7 +379,7 @@ private boolean unprotectedCommitJob(DeleteJob job, Database db, long timeoutMs) tabletCommitInfos.add(new TabletCommitInfo(tabletId, replica.getBackendId())); } } - return globalTransactionMgr.commitAndPublishTransaction(db, transactionId, tabletCommitInfos, timeoutMs); + return globalTransactionMgr.commitAndPublishTransaction(db, Lists.newArrayList(table), transactionId, tabletCommitInfos, timeoutMs); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java index 48b01ce9a52054..4446f9597821ec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java @@ -2526,7 +2526,8 @@ public void unprotectQuorumLoadJob(LoadJob job, Database db) { public void replayQuorumLoadJob(LoadJob job, Catalog catalog) throws DdlException { // TODO: need to call this.writeLock()? Database db = catalog.getDb(job.getDbId()); - db.writeLock(); + Table table = db.getTable(job.getTableId()); + table.writeLock(); try { writeLock(); try { @@ -2535,7 +2536,7 @@ public void replayQuorumLoadJob(LoadJob job, Catalog catalog) throws DdlExceptio writeUnlock(); } } finally { - db.writeUnlock(); + table.writeUnlock(); } } @@ -2594,7 +2595,8 @@ public void unprotectFinishLoadJob(LoadJob job, Database db) { public void replayFinishLoadJob(LoadJob job, Catalog catalog) { // TODO: need to call this.writeLock()? Database db = catalog.getDb(job.getDbId()); - db.writeLock(); + Table table = db.getTable(job.getTableId()); + table.writeLock(); try { writeLock(); try { @@ -2603,20 +2605,20 @@ public void replayFinishLoadJob(LoadJob job, Catalog catalog) { writeUnlock(); } } finally { - db.writeUnlock(); + table.writeUnlock(); } } public void replayClearRollupInfo(ReplicaPersistInfo info, Catalog catalog) { Database db = catalog.getDb(info.getDbId()); - db.writeLock(); + OlapTable olapTable = (OlapTable) db.getTable(info.getTableId()); + olapTable.writeLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(info.getTableId()); Partition partition = olapTable.getPartition(info.getPartitionId()); MaterializedIndex index = partition.getIndex(info.getIndexId()); index.clearRollupIndexInfo(); } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } @@ -2880,7 +2882,7 @@ public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType ca writeUnlock(); } } else { - db.writeLock(); + table.writeLock(); try { writeLock(); try { @@ -2971,7 +2973,7 @@ public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType ca writeUnlock(); } } finally { - db.writeUnlock(); + table.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java index 77b7cfe3ebc5f2..f15585c8e17932 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java @@ -25,6 +25,7 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Tablet; import org.apache.doris.catalog.TabletInvertedIndex; import org.apache.doris.common.Config; @@ -240,6 +241,14 @@ private void runOneLoadingJob(LoadJob job) { load.cancelLoadJob(job, CancelType.LOAD_RUN_FAIL, "db does not exist. id: " + dbId); return; } + long tableId = job.getTableId(); + Table table = null; + try { + table = db.getTableOrThrowException(tableId, Table.TableType.OLAP); + } catch (UserException e) { + load.cancelLoadJob(job, CancelType.LOAD_RUN_FAIL, "table does not exist. dbId: " + dbId + ", tableId: " + tableId); + return; + } if (job.getTransactionId() < 0) { LOG.warn("cancel load job {} because it is an old type job, user should resubmit it", job); @@ -305,12 +314,12 @@ private void runOneLoadingJob(LoadJob job) { // if all tablets are finished or stay in quorum finished for long time, try to commit it. if (System.currentTimeMillis() - job.getQuorumFinishTimeMs() > stragglerTimeout || job.getFullTablets().containsAll(jobTotalTablets)) { - tryCommitJob(job, db); + tryCommitJob(job, table); } } } - private void tryCommitJob(LoadJob job, Database db) { + private void tryCommitJob(LoadJob job, Table table) { // check transaction state Load load = Catalog.getCurrentCatalog().getLoadInstance(); GlobalTransactionMgr globalTransactionMgr = Catalog.getCurrentGlobalTransactionMgr(); @@ -318,7 +327,7 @@ private void tryCommitJob(LoadJob job, Database db) { List tabletCommitInfos = new ArrayList(); // when be finish load task, fe will update job's finish task info, use lock here to prevent // concurrent problems - db.writeLock(); + table.writeLock(); try { TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); for (Replica replica : job.getFinishedReplicas()) { @@ -338,7 +347,7 @@ private void tryCommitJob(LoadJob job, Database db) { transactionState.getTransactionId(), job, e); load.cancelLoadJob(job, CancelType.UNKNOWN, transactionState.getReason()); } finally { - db.writeUnlock(); + table.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java index 15735c0edcae6a..90ddd9cc13d587 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java @@ -309,6 +309,7 @@ private void onLoadingTaskFinished(BrokerLoadingTaskAttachment attachment) { .add("error_msg", "Failed to commit txn with error:" + e.getMessage()) .build(), e); cancelJobWithoutCheck(new FailMsg(FailMsg.CancelType.LOAD_RUN_FAIL, e.getMessage()), true, true); + return; } finally { db.writeUnlock(); } @@ -374,4 +375,3 @@ public void afterVisible(TransactionState txnState, boolean txnOperated) { writeProfile(); } } - diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java index d8deda26f449a6..56b3a9adeceeb8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java @@ -381,7 +381,6 @@ private static void sync(Map backendTablets, ListMultimap backendTablets, ListMultimap>>>>>> Continue to use table lock to replace db lock:fe/src/main/java/org/apache/doris/master/ReportHandler.java - int syncCounter = 0; - List tabletIds = tabletSyncMap.get(dbId); - LOG.info("before sync tablets in db[{}]. report num: {}. backend[{}]", - dbId, tabletIds.size(), backendId); - List tabletMetaList = invertedIndex.getTabletMetaList(tabletIds); - for (int i = 0; i < tabletMetaList.size(); i++) { - TabletMeta tabletMeta = tabletMetaList.get(i); - if (tabletMeta == TabletInvertedIndex.NOT_EXIST_TABLET_META) { - continue; - } - long tabletId = tabletIds.get(i); - long tableId = tabletMeta.getTableId(); - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - continue; - } - olapTable.writeLock(); - try { long partitionId = tabletMeta.getPartitionId(); Partition partition = olapTable.getPartition(partitionId); if (partition == null) { @@ -515,11 +494,11 @@ private static void sync(Map backendTablets, ListMultimap tabletDeleteFromMeta if (db == null) { continue; } -<<<<<<< HEAD:fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java db.writeLock(); try { int deleteCounter = 0; @@ -549,25 +527,7 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta if (olapTable == null) { continue; } -======= ->>>>>>> Continue to use table lock to replace db lock:fe/src/main/java/org/apache/doris/master/ReportHandler.java - int deleteCounter = 0; - List tabletIds = tabletDeleteFromMeta.get(dbId); - List tabletMetaList = invertedIndex.getTabletMetaList(tabletIds); - for (int i = 0; i < tabletMetaList.size(); i++) { - TabletMeta tabletMeta = tabletMetaList.get(i); - if (tabletMeta == TabletInvertedIndex.NOT_EXIST_TABLET_META) { - continue; - } - long tabletId = tabletIds.get(i); - long tableId = tabletMeta.getTableId(); - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - continue; - } - olapTable.writeLock(); - try { long partitionId = tabletMeta.getPartitionId(); Partition partition = olapTable.getPartition(partitionId); if (partition == null) { @@ -620,11 +580,7 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta if (Config.recover_with_empty_tablet) { // only create this task if force recovery is true LOG.warn("tablet {} has only one replica {} on backend {}" -<<<<<<< HEAD:fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java + " and it is lost. create an empty replica to recover it", -======= - + "and it is lost. create an empty replica to recover it", ->>>>>>> Continue to use table lock to replace db lock:fe/src/main/java/org/apache/doris/master/ReportHandler.java tabletId, replica.getId(), backendId); MaterializedIndexMeta indexMeta = olapTable.getIndexMetaByIndexId(indexId); Set bfColumns = olapTable.getCopiedBfColumns(); @@ -644,11 +600,7 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta // just set this replica as bad if (replica.setBad(true)) { LOG.warn("tablet {} has only one replica {} on backend {}" -<<<<<<< HEAD:fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java + " and it is lost, set it as bad", -======= - + "and it is lost, set it as bad", ->>>>>>> Continue to use table lock to replace db lock:fe/src/main/java/org/apache/doris/master/ReportHandler.java tabletId, replica.getId(), backendId); BackendTabletsInfo tabletsInfo = new BackendTabletsInfo(backendId); tabletsInfo.setBad(true); @@ -684,11 +636,11 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta LOG.error("invalid situation. tablet[{}] is empty", tabletId); } } - } finally { - olapTable.writeUnlock(); - } - } // end for tabletMetas - LOG.info("delete {} replica(s) from catalog in db[{}]", deleteCounter, dbId); + } // end for tabletMetas + LOG.info("delete {} replica(s) from catalog in db[{}]", deleteCounter, dbId); + } finally { + db.writeUnlock(); + } } // end for dbs if (Config.recover_with_empty_tablet && createReplicaBatchTask.getTaskNum() > 0) { @@ -816,7 +768,6 @@ private static void handleRecoverTablet(ListMultimap tabletRecoveryM tabletRecoveryMap.size(), backendId); TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); -<<<<<<< HEAD:fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java BackendTabletsInfo backendTabletsInfo = new BackendTabletsInfo(backendId); backendTabletsInfo.setBad(true); for (Long dbId : tabletRecoveryMap.keySet()) { @@ -845,40 +796,6 @@ private static void handleRecoverTablet(ListMultimap tabletRecoveryM if (partition == null) { continue; } -======= - if (!forceRecovery) { - LOG.warn("force recovery is disable. try reset the tablets' version" - + " or set it as bad, and waiting clone"); - - BackendTabletsInfo backendTabletsInfo = new BackendTabletsInfo(backendId); - backendTabletsInfo.setBad(true); - for (Long dbId : tabletRecoveryMap.keySet()) { - Database db = Catalog.getCurrentCatalog().getDb(dbId); - if (db == null) { - continue; - } - - List tabletIds = tabletRecoveryMap.get(dbId); - List tabletMetaList = invertedIndex.getTabletMetaList(tabletIds); - for (int i = 0; i < tabletMetaList.size(); i++) { - TabletMeta tabletMeta = tabletMetaList.get(i); - if (tabletMeta == TabletInvertedIndex.NOT_EXIST_TABLET_META) { - continue; - } - long tabletId = tabletIds.get(i); - long tableId = tabletMeta.getTableId(); - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - continue; - } - olapTable.writeLock(); - try { - long partitionId = tabletMeta.getPartitionId(); - Partition partition = olapTable.getPartition(partitionId); - if (partition == null) { - continue; - } ->>>>>>> Continue to use table lock to replace db lock:fe/src/main/java/org/apache/doris/master/ReportHandler.java long indexId = tabletMeta.getIndexId(); MaterializedIndex index = partition.getIndex(indexId); @@ -938,8 +855,6 @@ private static void handleRecoverTablet(ListMultimap tabletRecoveryM break; } } - } finally { - olapTable.writeUnlock(); } } } finally { @@ -986,13 +901,12 @@ private static void handleSetTabletInMemory(long backendId, Map b if (db == null) { continue; } - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - continue; - } - - olapTable.readLock(); + db.readLock(); try { + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + continue; + } Partition partition = olapTable.getPartition(partitionId); if (partition == null) { continue; @@ -1002,7 +916,7 @@ private static void handleSetTabletInMemory(long backendId, Map b tabletToInMemory.add(new ImmutableTriple<>(tabletId, tabletInfo.getSchemaHash(), feIsInMemory)); } } finally { - olapTable.readUnlock(); + db.readUnlock(); } } } @@ -1049,14 +963,13 @@ private static void addReplica(long tabletId, TTabletInfo backendTabletInfo, lon if (db == null) { throw new MetaNotFoundException("db[" + dbId + "] does not exist"); } - - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - throw new MetaNotFoundException("table[" + tableId + "] does not exist"); - } - - olapTable.writeLock(); + db.writeLock(); try { + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + throw new MetaNotFoundException("table[" + tableId + "] does not exist"); + } + Partition partition = olapTable.getPartition(partitionId); if (partition == null) { throw new MetaNotFoundException("partition[" + partitionId + "] does not exist"); @@ -1152,7 +1065,7 @@ private static void addReplica(long tabletId, TTabletInfo backendTabletInfo, lon "replica is enough[" + tablet.getReplicas().size() + "-" + replicationNum + "]"); } } finally { - olapTable.writeUnlock(); + db.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index d2b51ce6e47b8b..bbf89b4b1b4241 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -898,7 +898,7 @@ private void handleInsertStmt() throws Exception { return; } if (Catalog.getCurrentGlobalTransactionMgr().commitAndPublishTransaction( - insertStmt.getDbObj(), insertStmt.getTransactionId(), + insertStmt.getDbObj(), Lists.newArrayList(insertStmt.getTargetTable()), insertStmt.getTransactionId(), TabletCommitInfo.fromThrift(coord.getCommitInfos()), context.getSessionVariable().getInsertVisibleTimeoutMs())) { txnStatus = TransactionStatus.VISIBLE; diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index ee36fd356c4a63..88377b9d00603c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -822,9 +822,11 @@ private boolean loadTxnCommitImpl(TLoadTxnCommitRequest request) throws UserExce } throw new UserException("unknown database, database=" + dbName); } + long timeoutMs = request.isSetThriftRpcTimeoutMs() ? request.getThriftRpcTimeoutMs() : 5000; + Table table = db.getTableOrThrowException(request.getTbl(), TableType.OLAP); boolean ret = Catalog.getCurrentGlobalTransactionMgr().commitAndPublishTransaction( - db, request.getTxnId(), + db, Lists.newArrayList(table), request.getTxnId(), TabletCommitInfo.fromThrift(request.getCommitInfos()), timeoutMs, TxnCommitAttachment.fromThrift(request.txnCommitAttachment)); if (ret) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java index f5c2b40799bb5a..ac45bbad7358b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java @@ -658,7 +658,26 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) thr writeUnlock(); } } - db.writeLock(); + List tableIdList = transactionState.getTableIdList(); + // to be compatiable with old meta version, table List may be empty + if (tableIdList.isEmpty()) { + readLock(); + try { + for (TableCommitInfo tableCommitInfo : transactionState.getIdToTableCommitInfos().values()) { + long tableId = tableCommitInfo.getTableId(); + if (!tableIdList.contains(tableId)) { + tableIdList.add(tableId); + } + } + } finally { + readUnlock(); + } + } + + List
tableList = db.getTablesOnIdOrderOrThrowException(tableIdList); + for (Table table : tableList) { + table.writeLock(); + } try { boolean hasError = false; Iterator tableCommitInfoIterator = transactionState.getIdToTableCommitInfos().values().iterator(); @@ -799,7 +818,9 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) thr } updateCatalogAfterVisible(transactionState, db); } finally { - db.writeUnlock(); + for (int i = tableList.size() - 1; i >=0; i--) { + tableList.get(i).writeUnlock(); + } } LOG.info("finish transaction {} successfully", transactionState); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java index 6f8e768b31d425..e639488c7d6610 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java @@ -19,6 +19,7 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Table; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.DuplicatedRequestException; @@ -44,7 +45,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.TimeUnit; /** * Transaction Manager @@ -176,23 +176,25 @@ public void commitTransaction(long dbId, long transactionId, List tableList, long transactionId, List tabletCommitInfos, long timeoutMillis) throws UserException { - return commitAndPublishTransaction(db, transactionId, tabletCommitInfos, timeoutMillis, null); + return commitAndPublishTransaction(db, tableList, transactionId, tabletCommitInfos, timeoutMillis, null); } - public boolean commitAndPublishTransaction(Database db, long transactionId, + public boolean commitAndPublishTransaction(Database db, List
tableList, long transactionId, List tabletCommitInfos, long timeoutMillis, TxnCommitAttachment txnCommitAttachment) throws UserException { - if (!db.tryWriteLock(timeoutMillis, TimeUnit.MILLISECONDS)) { - throw new UserException("get database write lock timeout, database=" + db.getFullName()); + for (Table table : tableList) { + table.writeLock(); } try { commitTransaction(db.getId(), transactionId, tabletCommitInfos, txnCommitAttachment); } finally { - db.writeUnlock(); + for (int i = tableList.size() - 1; i >= 0; i--) { + tableList.get(i).writeUnlock(); + } } DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(db.getId()); return dbTransactionMgr.publishTransaction(db, transactionId, timeoutMillis); diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/TableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/TableTest.java index 2066b078e460f6..718738437efd13 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/TableTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/TableTest.java @@ -36,21 +36,43 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; public class TableTest { private FakeCatalog fakeCatalog; private Catalog catalog; + private Table table; + private long tableId = 10000; + @Before public void setUp() { fakeCatalog = new FakeCatalog(); catalog = Deencapsulation.newInstance(Catalog.class); - + table = new Table(Table.TableType.OLAP); FakeCatalog.setCatalog(catalog); FakeCatalog.setMetaVersion(FeConstants.meta_version); } + @Test + public void lockTest() { + table.readLock(); + try { + Assert.assertFalse(table.tryWriteLock(0, TimeUnit.SECONDS)); + } finally { + table.readUnlock(); + } + + table.writeLock(); + try { + Assert.assertTrue(table.tryWriteLock(0, TimeUnit.SECONDS)); + } finally { + table.writeUnlock(); + } + } + + @Test public void testSerialization() throws Exception { // 1. Write objects to file From e13b952a7a3870928d268595b88d6794d78980eb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Thu, 4 Jun 2020 19:14:52 +0800 Subject: [PATCH 08/50] fix unit test --- .../src/main/java/org/apache/doris/catalog/Catalog.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index bcfb8638d3daf2..6192ca78be1e76 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -5413,7 +5413,7 @@ public void modifyTableDynamicPartition(Database db, OlapTable table, Map properties) throws DdlException { - Preconditions.checkArgument(db.isWriteLockHeldByCurrentThread()); + Preconditions.checkArgument(table.isWriteLockHeldByCurrentThread()); String defaultReplicationNumName = "default."+ PropertyAnalyzer.PROPERTIES_REPLICATION_NUM; PartitionInfo partitionInfo = table.getPartitionInfo(); if (partitionInfo.getType() == PartitionType.RANGE) { @@ -5448,7 +5448,7 @@ public void modifyTableReplicationNum(Database db, OlapTable table, Map properties) { - Preconditions.checkArgument(db.isWriteLockHeldByCurrentThread()); + Preconditions.checkArgument(table.isWriteLockHeldByCurrentThread()); TableProperty tableProperty = table.getTableProperty(); if (tableProperty == null) { tableProperty = new TableProperty(properties); From c2ff4432eaee19d879f6a0547fc37cb929f124cf Mon Sep 17 00:00:00 2001 From: caiconghui Date: Thu, 4 Jun 2020 21:58:04 +0800 Subject: [PATCH 09/50] add null check for olap_table --- .../src/main/java/org/apache/doris/alter/AlterJobV2.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java index cb8f590b9750a7..efd7346b27ede9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java @@ -188,6 +188,9 @@ public final boolean cancel(String errMsg) { */ protected boolean checkTableStable(Database db) throws AlterCancelException { OlapTable tbl = (OlapTable) db.getTable(tableId); + if (tbl == null) { + throw new AlterCancelException("Table " + tableId + " does not exist"); + } boolean isStable = tbl.isStable(Catalog.getCurrentSystemInfo(), Catalog.getCurrentCatalog().getTabletScheduler(), db.getClusterName()); From 06467838b7186d8a8bbc3231163f5e3e9ad12888 Mon Sep 17 00:00:00 2001 From: caiconghui Date: Sun, 14 Jun 2020 00:50:36 +0800 Subject: [PATCH 10/50] fix merge --- .../main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java index 90ddd9cc13d587..59aa98a936416b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java @@ -22,6 +22,7 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Table; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.DataQualityException; @@ -57,6 +58,7 @@ import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.concurrent.RejectedExecutionException; From f6481957664ef0b81e8652bd2ac646b8c2739de8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 16 Jun 2020 10:48:36 +0800 Subject: [PATCH 11/50] modify .gitignore content --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 322ddd538127bf..b5650127d09839 100644 --- a/.gitignore +++ b/.gitignore @@ -10,6 +10,7 @@ docs/.temp docs/.vuepress/dist docs/node_modules docs/build +docs/contents gensrc/build fe/fe-core/target thirdparty/src From dd9952d3c6008585558dbb02090a35629bc7dbbc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 16 Jun 2020 11:29:50 +0800 Subject: [PATCH 12/50] fix Alter Class --- .../java/org/apache/doris/alter/Alter.java | 98 ++++++------------- .../doris/alter/MaterializedViewHandler.java | 6 ++ .../org/apache/doris/catalog/Catalog.java | 6 +- .../org/apache/doris/catalog/Database.java | 8 +- .../doris/load/loadv2/BrokerLoadJob.java | 2 - 5 files changed, 44 insertions(+), 76 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index 435ececd138b4e..ecb7d0b155d0ee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -91,7 +91,7 @@ public void start() { } public void processCreateMaterializedView(CreateMaterializedViewStmt stmt) - throws DdlException, AnalysisException { + throws DdlException, AnalysisException, MetaNotFoundException { String tableName = stmt.getBaseIndexName(); // check db String dbName = stmt.getDBName(); @@ -104,11 +104,7 @@ public void processCreateMaterializedView(CreateMaterializedViewStmt stmt) // check db quota db.checkQuota(); - Table table = db.getTable(tableName); - if (table.getType() != TableType.OLAP) { - throw new DdlException("Do not support alter non-OLAP table[" + tableName + "]"); - } - OlapTable olapTable = (OlapTable) table; + OlapTable olapTable = (OlapTable) db.getTableOrThrowException(tableName, TableType.OLAP); olapTable.writeLock(); try { @@ -129,30 +125,9 @@ public void processDropMaterializedView(DropMaterializedViewStmt stmt) throws Dd } String tableName = stmt.getTableName().getTbl(); - Table table = db.getTable(tableName); - // if table exists - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } - // check table type - if (table.getType() != TableType.OLAP) { - throw new DdlException("Do not support non-OLAP table [" + tableName + "] when drop materialized view"); - } - - OlapTable olapTable = (OlapTable) table; - olapTable.writeLock(); - try { - // check table state - if (olapTable.getState() != OlapTableState.NORMAL) { - throw new DdlException("Table[" + table.getName() + "]'s state is not NORMAL. " - + "Do not allow doing DROP ops"); - } - // drop materialized view - ((MaterializedViewHandler)materializedViewHandler).processDropMaterializedView(stmt, db, olapTable); - - } finally { - olapTable.writeUnlock(); - } + OlapTable olapTable = (OlapTable) db.getTableOrThrowException(tableName, TableType.OLAP); + // drop materialized view + ((MaterializedViewHandler)materializedViewHandler).processDropMaterializedView(stmt, db, olapTable); } private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable, List alterClauses, @@ -244,21 +219,25 @@ private void processAlterExternalTable(AlterTableStmt stmt, Table externalTable, public void processAlterTable(AlterTableStmt stmt) throws UserException { TableName dbTableName = stmt.getTbl(); String dbName = dbTableName.getDb(); + String tableName = dbTableName.getTbl(); final String clusterName = stmt.getClusterName(); Database db = Catalog.getCurrentCatalog().getDb(dbName); if (db == null) { ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); } + Table table = db.getTable(tableName); List alterClauses = Lists.newArrayList(); - // some operations will take long time to process, need to be done outside the table lock boolean needProcessOutsideTableLock = false; - String tableName = dbTableName.getTbl(); - - Table table = db.getTable(tableName); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + // check conflict alter ops first + alterClauses = stmt.getOps(); + AlterOperations currentAlterOps = new AlterOperations(); + currentAlterOps.checkConflict(alterClauses); + // check cluster capacity and db quota, only need to check once. + if (currentAlterOps.needCheckCapacity()) { + Catalog.getCurrentSystemInfo().checkClusterCapacity(clusterName); + db.checkQuota(); } table.writeLock(); @@ -279,20 +258,9 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { OlapTable olapTable = (OlapTable) table; stmt.rewriteAlterClause(olapTable); - - // check conflict alter ops first - alterClauses = stmt.getOps(); - AlterOperations currentAlterOps = new AlterOperations(); - currentAlterOps.checkConflict(alterClauses); - - // check cluster capacity and db quota, only need to check once. - if (currentAlterOps.needCheckCapacity()) { - Catalog.getCurrentSystemInfo().checkClusterCapacity(clusterName); - db.checkQuota(); - } if (olapTable.getState() != OlapTableState.NORMAL) { throw new DdlException( - "Table[" + table.getName() + "]'s state is not NORMAL. Do not allow doing ALTER ops"); + "Table[" + olapTable.getName() + "]'s state is not NORMAL. Do not allow doing ALTER ops"); } if (currentAlterOps.hasSchemaChangeOp()) { @@ -362,8 +330,7 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { Preconditions.checkState(properties.containsKey(PropertyAnalyzer.PROPERTIES_INMEMORY)); ((SchemaChangeHandler) schemaChangeHandler).updatePartitionsInMemoryMeta( db, tableName, partitionNames, properties); - - OlapTable olapTable = (OlapTable) db.getTable(tableName); + OlapTable olapTable = (OlapTable) table; olapTable.writeLock(); try { modifyPartitionsProperty(db, olapTable, partitionNames, properties); @@ -471,17 +438,12 @@ public void processAlterView(AlterViewStmt stmt, ConnectContext ctx) throws User } String tableName = dbTableName.getTbl(); - db.writeLock(); - try { - Table table = db.getTableOrThrowException(tableName, TableType.VIEW); - View view = (View) table; - modifyViewDef(db, view, stmt.getInlineViewDef(), ctx.getSessionVariable().getSqlMode(), stmt.getColumns()); - } finally { - db.writeUnlock(); - } + View view = (View) db.getTableOrThrowException(tableName, TableType.VIEW); + modifyViewDef(db, view, stmt.getInlineViewDef(), ctx.getSessionVariable().getSqlMode(), stmt.getColumns()); } private void modifyViewDef(Database db, View view, String inlineViewDef, long sqlMode, List newFullSchema) throws DdlException { + db.writeLock(); view.writeLock(); try { view.setInlineViewDefWithSqlMode(inlineViewDef, sqlMode); @@ -491,17 +453,17 @@ private void modifyViewDef(Database db, View view, String inlineViewDef, long sq throw new DdlException("failed to init view stmt", e); } view.setNewFullSchema(newFullSchema); + String viewName = view.getName(); + db.dropTable(viewName); + db.createTable(view); + + AlterViewInfo alterViewInfo = new AlterViewInfo(db.getId(), view.getId(), inlineViewDef, newFullSchema, sqlMode); + Catalog.getCurrentCatalog().getEditLog().logModifyViewDef(alterViewInfo); + LOG.info("modify view[{}] definition to {}", viewName, inlineViewDef); } finally { view.writeUnlock(); + db.writeUnlock(); } - - String viewName = view.getName(); - db.dropTable(viewName); - db.createTable(view); - - AlterViewInfo alterViewInfo = new AlterViewInfo(db.getId(), view.getId(), inlineViewDef, newFullSchema, sqlMode); - Catalog.getCurrentCatalog().getEditLog().logModifyViewDef(alterViewInfo); - LOG.info("modify view[{}] definition to {}", viewName, inlineViewDef); } public void replayModifyViewDef(AlterViewInfo alterViewInfo) throws DdlException { @@ -511,9 +473,10 @@ public void replayModifyViewDef(AlterViewInfo alterViewInfo) throws DdlException List newFullSchema = alterViewInfo.getNewFullSchema(); Database db = Catalog.getCurrentCatalog().getDb(dbId); + View view = (View) db.getTable(tableId); db.writeLock(); + view.writeLock(); try { - View view = (View) db.getTable(tableId); String viewName = view.getName(); view.setInlineViewDefWithSqlMode(inlineViewDef, alterViewInfo.getSqlMode()); try { @@ -528,6 +491,7 @@ public void replayModifyViewDef(AlterViewInfo alterViewInfo) throws DdlException LOG.info("replay modify view[{}] definition to {}", viewName, inlineViewDef); } finally { + view.writeUnlock(); db.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java index 8dacb6d5e23dd1..bcd191a9035733 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java @@ -734,6 +734,12 @@ public void processDropMaterializedView(DropMaterializedViewStmt dropMaterialize Preconditions.checkState(olapTable.isWriteLockHeldByCurrentThread()); olapTable.writeLock(); try { + // check table state + if (olapTable.getState() != OlapTableState.NORMAL) { + throw new DdlException("Table[" + olapTable.getName() + "]'s state is not NORMAL. " + + "Do not allow doing DROP ops"); + } + String mvName = dropMaterializedViewStmt.getMvName(); // Step1: check drop mv index operation checkDropMaterializedView(mvName, olapTable); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index 6192ca78be1e76..aa5c6145f00484 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -5055,19 +5055,19 @@ public static short calcShortKeyColumnCount(List columns, Map alterClauses, Database db, O Map rollupNameJobMap = new LinkedHashMap<>(); // save job id for log Set logJobIdSet = new HashSet<>(); - + olapTable.writeLock(); try { + if (olapTable.existTempPartitions()) { + throw new DdlException("Can not alter table when there are temp partitions in table"); + } + // 1 check and make rollup job for (AlterClause alterClause : alterClauses) { AddRollupClause addRollupClause = (AddRollupClause) alterClause; @@ -269,6 +273,20 @@ public void processBatchAddRollup(List alterClauses, Database db, O rollupNameJobMap.put(addRollupClause.getRollupName(), alterJobV2); logJobIdSet.add(alterJobV2.getJobId()); } + + // set table' state to ROLLUP before adding rollup jobs. + // so that when the AlterHandler thread run the jobs, it will see the expected table's state. + // ATTN: This order is not mandatory, because database lock will protect us, + // but this order is more reasonable + olapTable.setState(OlapTableState.ROLLUP); + // 2 batch submit rollup job + List rollupJobV2List = new ArrayList<>(rollupNameJobMap.values()); + batchAddAlterJobV2(rollupJobV2List); + + BatchAlterJobPersistInfo batchAlterJobV2 = new BatchAlterJobPersistInfo(rollupJobV2List); + Catalog.getCurrentCatalog().getEditLog().logBatchAlterJob(batchAlterJobV2); + LOG.info("finished to create materialized view job: {}", logJobIdSet); + } catch (Exception e) { // remove tablet which has already inserted into TabletInvertedIndex TabletInvertedIndex tabletInvertedIndex = Catalog.getCurrentInvertedIndex(); @@ -280,20 +298,9 @@ public void processBatchAddRollup(List alterClauses, Database db, O } } throw e; + } finally { + olapTable.writeUnlock(); } - - // set table' state to ROLLUP before adding rollup jobs. - // so that when the AlterHandler thread run the jobs, it will see the expected table's state. - // ATTN: This order is not mandatory, because database lock will protect us, - // but this order is more reasonable - olapTable.setState(OlapTableState.ROLLUP); - // 2 batch submit rollup job - List rollupJobV2List = new ArrayList<>(rollupNameJobMap.values()); - batchAddAlterJobV2(rollupJobV2List); - - BatchAlterJobPersistInfo batchAlterJobV2 = new BatchAlterJobPersistInfo(rollupJobV2List); - Catalog.getCurrentCatalog().getEditLog().logBatchAlterJob(batchAlterJobV2); - LOG.info("finished to create materialized view job: {}", logJobIdSet); } /** @@ -701,6 +708,10 @@ public void processBatchDropRollup(List dropRollupClauses, Database throws DdlException, MetaNotFoundException { olapTable.writeLock(); try { + if (olapTable.existTempPartitions()) { + throw new DdlException("Can not alter table when there are temp partitions in table"); + } + // check drop rollup index operation for (AlterClause alterClause : dropRollupClauses) { DropRollupClause dropRollupClause = (DropRollupClause) alterClause; @@ -871,6 +882,8 @@ private void changeTableStatus(long dbId, long tableId, OlapTableState olapTable } OlapTable tbl = (OlapTable) db.getTable(tableId); if (tbl == null) { + LOG.warn("table {} has been dropped when changing table status after rollup job done", + tableId); return; } tbl.writeLock(); @@ -1145,11 +1158,6 @@ private void getOldAlterJobInfos(Database db, List> rollupJobIn @Override public void process(List alterClauses, String clusterName, Database db, OlapTable olapTable) throws DdlException, AnalysisException, MetaNotFoundException { - - if (olapTable.existTempPartitions()) { - throw new DdlException("Can not alter table when there are temp partitions in table"); - } - Optional alterClauseOptional = alterClauses.stream().findAny(); if (alterClauseOptional.isPresent()) { if (alterClauseOptional.get() instanceof AddRollupClause) { @@ -1178,14 +1186,12 @@ public void cancel(CancelStmt stmt) throws DdlException { AlterJob rollupJob = null; List rollupJobV2List = new ArrayList<>(); - Table table = db.getTable(tableName); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } - if (!(table instanceof OlapTable)) { - ErrorReport.reportDdlException(ErrorCode.ERR_NOT_OLAP_TABLE, tableName); + OlapTable olapTable = null; + try { + olapTable = (OlapTable) db.getTableOrThrowException(tableName, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + throw new DdlException(e.getMessage()); } - OlapTable olapTable = (OlapTable) table; olapTable.writeLock(); try { if (olapTable.getState() != OlapTableState.ROLLUP) { From 6ec9c842c13b96fead044b41d6e29a7c8eaf0c13 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 16 Jun 2020 13:25:18 +0800 Subject: [PATCH 14/50] Fix RollupJob --- .../org/apache/doris/alter/RollupJob.java | 62 ++++++++----------- 1 file changed, 27 insertions(+), 35 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java index c92e739bbd15d3..e5e35c5daf1886 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java @@ -29,6 +29,7 @@ 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; @@ -350,10 +351,11 @@ public int checkOrResendClearTasks() { batchClearAlterTask = new AgentBatchTask(); synchronized (this) { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - cancelMsg = "table[" + tableId + "] does not exist"; - LOG.warn(cancelMsg); + OlapTable olapTable = null; + try { + olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + LOG.warn(e.getMessage()); return -1; } olapTable.readLock(); @@ -417,10 +419,11 @@ public boolean sendTasks() { synchronized (this) { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - cancelMsg = "table[" + tableId + "] does not exist"; - LOG.warn(cancelMsg); + OlapTable olapTable = null; + try { + olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + LOG.warn(e.getMessage()); return false; } @@ -637,10 +640,11 @@ public int tryFinishJob() { return -1; } - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - cancelMsg = "Table[" + tableId + "] does not exist"; - LOG.warn(cancelMsg); + OlapTable olapTable = null; + try { + olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + LOG.warn(e.getMessage()); return -1; } @@ -790,9 +794,6 @@ public synchronized void clear() { @Override public void replayInitJob(Database db) { OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - return; - } olapTable.writeLock(); try { // set state @@ -830,9 +831,6 @@ public void replayInitJob(Database db) { public void replayFinishing(Database db) { TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - return; - } olapTable.writeLock(); try { for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { @@ -916,23 +914,17 @@ public void replayFinish(Database db) { } OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable != null) { - olapTable.writeLock(); - try { - olapTable.setState(OlapTableState.NORMAL); - } finally { - olapTable.writeUnlock(); - } + olapTable.writeLock(); + try { + olapTable.setState(OlapTableState.NORMAL); + } finally { + olapTable.writeUnlock(); } } @Override public void replayCancel(Database db) { OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - return; - } - olapTable.writeLock(); try{ if (!Catalog.isCheckpointThread()) { @@ -963,12 +955,14 @@ public void finishJob() { return; } - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - cancelMsg = String.format("table %d does not exist", tableId); - LOG.warn(cancelMsg); + OlapTable olapTable = null; + try { + olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + LOG.warn(e.getMessage()); return; } + olapTable.writeLock(); try { olapTable.setState(OlapTableState.NORMAL); @@ -976,8 +970,6 @@ public void finishJob() { olapTable.writeUnlock(); } - List list = new ArrayList<>(); - this.finishedTime = System.currentTimeMillis(); LOG.info("finished rollup job: {}", tableId); } From 721a8400d03c4ffe994227121fc9d896ba399de4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 16 Jun 2020 13:39:14 +0800 Subject: [PATCH 15/50] fix RollupJobV2 --- .../org/apache/doris/alter/RollupJobV2.java | 27 ++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java index 430b46b252be62..6548b56a5f5db5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java @@ -34,6 +34,7 @@ import org.apache.doris.catalog.Partition; 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; @@ -42,6 +43,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.MarkedCountDownLatch; +import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.io.Text; import org.apache.doris.common.util.SqlParserUtils; import org.apache.doris.common.util.TimeUtils; @@ -198,9 +200,11 @@ protected void runPendingJob() throws AlterCancelException { } } MarkedCountDownLatch countDownLatch = new MarkedCountDownLatch(totalReplicaNum); - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); + OlapTable tbl = null; + try { + tbl = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + throw new AlterCancelException(e.getMessage()); } tbl.readLock(); @@ -336,10 +340,13 @@ protected void runWaitingTxnJob() throws AlterCancelException { throw new AlterCancelException("Databasee " + dbId + " does not exist"); } - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); + OlapTable tbl = null; + try { + tbl = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + throw new AlterCancelException(e.getMessage()); } + tbl.readLock(); try { Preconditions.checkState(tbl.getState() == OlapTableState.ROLLUP); @@ -408,9 +415,11 @@ protected void runRunningJob() throws AlterCancelException { throw new AlterCancelException("Databasee " + dbId + " does not exist"); } - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); + OlapTable tbl = null; + try { + tbl = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + throw new AlterCancelException(e.getMessage()); } if (!rollupBatchTask.isFinished()) { From f76aa9b09a9aa13b0962d9ee860e029ad00733ab Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 16 Jun 2020 16:05:16 +0800 Subject: [PATCH 16/50] fix cancel --- .../java/org/apache/doris/alter/Alter.java | 3 + .../org/apache/doris/alter/AlterHandler.java | 33 +-- .../doris/alter/MaterializedViewHandler.java | 11 +- .../org/apache/doris/alter/RollupJob.java | 19 +- .../doris/alter/SchemaChangeHandler.java | 26 +- .../apache/doris/alter/SchemaChangeJob.java | 228 +++++++++--------- 6 files changed, 158 insertions(+), 162 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index ecb7d0b155d0ee..713e97602f6346 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -227,6 +227,9 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); } Table table = db.getTable(tableName); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } List alterClauses = Lists.newArrayList(); // some operations will take long time to process, need to be done outside the table lock boolean needProcessOutsideTableLock = false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java index 39dcce7f8ed9f9..4c33934365f63b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java @@ -271,35 +271,18 @@ public void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo, alterJob.handleFinishedReplica(task, finishTabletInfo, reportVersion); } - /* - * cancel alter job when drop table - * olapTable: - * table which is being dropped - */ - public void cancelWithTable(OlapTable olapTable) { - // make sure to hold to db write lock before calling this - AlterJob alterJob = getAlterJob(olapTable.getId()); - if (alterJob == null) { - return; + protected void cancelInternal(AlterJob alterJob, OlapTable olapTable, String msg) { + // cancel + if (olapTable != null) { + olapTable.writeLock(); } - alterJob.cancel(olapTable, "table is dropped"); - - // remove from alterJobs and add to finishedOrCancelledAlterJobs operation should be perform atomically - lock(); try { - alterJob = alterJobs.remove(olapTable.getId()); - if (alterJob != null) { - alterJob.clear(); - finishedOrCancelledAlterJobs.add(alterJob); - } + alterJob.cancel(olapTable, msg); } finally { - unlock(); + if (olapTable != null) { + olapTable.writeUnlock(); + } } - } - - protected void cancelInternal(AlterJob alterJob, OlapTable olapTable, String msg) { - // cancel - alterJob.cancel(olapTable, msg); jobDone(alterJob); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java index 70076bde3e6d3d..10e9d6aa48f4c3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java @@ -1077,7 +1077,16 @@ private void runOldAlterJob() { } OlapTable olapTable = (OlapTable) db.getTable(rollupJob.getTableId()); - rollupJob.cancel(olapTable, "cancelled"); + if (olapTable != null) { + olapTable.writeLock(); + } + try { + rollupJob.cancel(olapTable, "cancelled"); + } finally { + if (olapTable != null) { + olapTable.writeUnlock(); + } + } jobDone(rollupJob); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java index e5e35c5daf1886..8a7830a2e4fffb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java @@ -499,20 +499,15 @@ public synchronized void cancel(OlapTable olapTable, String msg) { // set state if (olapTable != null) { - olapTable.writeLock(); - try { - Preconditions.checkState(olapTable.getId() == tableId); - for (Partition partition : olapTable.getPartitions()) { - if (partition.getState() == PartitionState.ROLLUP) { - partition.setState(PartitionState.NORMAL); - } + Preconditions.checkState(olapTable.getId() == tableId); + for (Partition partition : olapTable.getPartitions()) { + if (partition.getState() == PartitionState.ROLLUP) { + partition.setState(PartitionState.NORMAL); } + } - if (olapTable.getState() == OlapTableState.ROLLUP) { - olapTable.setState(OlapTableState.NORMAL); - } - } finally { - olapTable.writeUnlock(); + if (olapTable.getState() == OlapTableState.ROLLUP) { + olapTable.setState(OlapTableState.NORMAL); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index ae67451c9ea558..9336e554a4a67c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -58,10 +58,9 @@ 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.MarkedCountDownLatch; +import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.DynamicPartitionUtil; @@ -1488,7 +1487,16 @@ private void runOldAlterJob() { } OlapTable olapTable = (OlapTable) db.getTable(alterJob.getTableId()); - alterJob.cancel(olapTable, "cancelled"); + if (olapTable != null) { + olapTable.writeLock(); + } + try { + alterJob.cancel(olapTable, "cancelled"); + } finally { + if (olapTable != null) { + olapTable.writeUnlock(); + } + } jobDone(alterJob); } @@ -1886,14 +1894,12 @@ public void cancel(CancelStmt stmt) throws DdlException { AlterJob schemaChangeJob = null; AlterJobV2 schemaChangeJobV2 = null; - Table table = db.getTable(tableName); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } - if (!(table instanceof OlapTable)) { - ErrorReport.reportDdlException(ErrorCode.ERR_NOT_OLAP_TABLE, tableName); + OlapTable olapTable = null; + try { + olapTable = (OlapTable) db.getTableOrThrowException(tableName, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + throw new DdlException(e.getMessage()); } - OlapTable olapTable = (OlapTable) table; olapTable.writeLock(); try { if (olapTable.getState() != OlapTableState.SCHEMA_CHANGE) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java index b153e2bb6f9b85..4e7d9dccc11fd6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java @@ -498,46 +498,41 @@ public boolean sendTasks() { @Override public synchronized void cancel(OlapTable olapTable, String msg) { - // make sure to get db write lock before calling this + // make sure to get table write lock before calling this if (olapTable != null) { - olapTable.writeLock(); - try { - // 1. remove all task and set state - for (Partition partition : olapTable.getPartitions()) { - if (partition.getState() == PartitionState.NORMAL) { + // 1. remove all task and set state + for (Partition partition : olapTable.getPartitions()) { + if (partition.getState() == PartitionState.NORMAL) { + continue; + } + long partitionId = partition.getId(); + for (Long indexId : this.changedIndexIdToSchema.keySet()) { + MaterializedIndex index = partition.getIndex(indexId); + if (index == null || index.getState() == IndexState.NORMAL) { continue; } - long partitionId = partition.getId(); - for (Long indexId : this.changedIndexIdToSchema.keySet()) { - MaterializedIndex index = partition.getIndex(indexId); - if (index == null || index.getState() == IndexState.NORMAL) { - continue; - } - for (Tablet tablet : index.getTablets()) { - long tabletId = tablet.getId(); - for (Replica replica : tablet.getReplicas()) { - if (replica.getState() == ReplicaState.CLONE - || replica.getState() == ReplicaState.DECOMMISSION - || replica.getState() == ReplicaState.NORMAL) { - continue; - } - Preconditions.checkState(replica.getState() == ReplicaState.SCHEMA_CHANGE); - replica.setState(ReplicaState.NORMAL); - AgentTaskQueue.removeTask(replica.getBackendId(), TTaskType.SCHEMA_CHANGE, tabletId); - } // end for replicas - } // end for tablets + for (Tablet tablet : index.getTablets()) { + long tabletId = tablet.getId(); + for (Replica replica : tablet.getReplicas()) { + if (replica.getState() == ReplicaState.CLONE + || replica.getState() == ReplicaState.DECOMMISSION + || replica.getState() == ReplicaState.NORMAL) { + continue; + } + Preconditions.checkState(replica.getState() == ReplicaState.SCHEMA_CHANGE); + replica.setState(ReplicaState.NORMAL); + AgentTaskQueue.removeTask(replica.getBackendId(), TTaskType.SCHEMA_CHANGE, tabletId); + } // end for replicas + } // end for tablets - // delete schema hash in inverted index - Catalog.getCurrentInvertedIndex().deleteNewSchemaHash(partitionId, indexId); - Preconditions.checkArgument(index.getState() == IndexState.SCHEMA_CHANGE); - index.setState(IndexState.NORMAL); - } // end for indices - partition.setState(PartitionState.NORMAL); - } // end for partitions - olapTable.setState(OlapTableState.NORMAL); - } finally { - olapTable.writeUnlock(); - } + // delete schema hash in inverted index + Catalog.getCurrentInvertedIndex().deleteNewSchemaHash(partitionId, indexId); + Preconditions.checkArgument(index.getState() == IndexState.SCHEMA_CHANGE); + index.setState(IndexState.NORMAL); + } // end for indices + partition.setState(PartitionState.NORMAL); + } // end for partitions + olapTable.setState(OlapTableState.NORMAL); } this.state = JobState.CANCELLED; @@ -1102,97 +1097,102 @@ public void replayCancel(Database db) { @Override public void getJobInfo(List> jobInfos, OlapTable tbl) { - if (changedIndexIdToSchemaVersion == null) { - // for compatibility - if (state == JobState.FINISHED || state == JobState.CANCELLED) { - List jobInfo = new ArrayList(); - jobInfo.add(tableId); // job id - jobInfo.add(tbl.getName()); // table name - jobInfo.add(TimeUtils.longToTimeString(createTime)); - jobInfo.add(TimeUtils.longToTimeString(finishedTime)); - jobInfo.add(FeConstants.null_string); // index name - jobInfo.add(FeConstants.null_string); // index id - jobInfo.add(FeConstants.null_string); // origin id - jobInfo.add(FeConstants.null_string); // schema version - jobInfo.add(-1); // transaction id - jobInfo.add(state.name()); // job state - jobInfo.add(cancelMsg); - jobInfo.add(FeConstants.null_string); // progress - jobInfo.add(Config.alter_table_timeout_second); // timeout - jobInfos.add(jobInfo); + tbl.readLock(); + try { + if (changedIndexIdToSchemaVersion == null) { + // for compatibility + if (state == JobState.FINISHED || state == JobState.CANCELLED) { + List jobInfo = new ArrayList(); + jobInfo.add(tableId); // job id + jobInfo.add(tbl.getName()); // table name + jobInfo.add(TimeUtils.longToTimeString(createTime)); + jobInfo.add(TimeUtils.longToTimeString(finishedTime)); + jobInfo.add(FeConstants.null_string); // index name + jobInfo.add(FeConstants.null_string); // index id + jobInfo.add(FeConstants.null_string); // origin id + jobInfo.add(FeConstants.null_string); // schema version + jobInfo.add(-1); // transaction id + jobInfo.add(state.name()); // job state + jobInfo.add(cancelMsg); + jobInfo.add(FeConstants.null_string); // progress + jobInfo.add(Config.alter_table_timeout_second); // timeout + jobInfos.add(jobInfo); + return; + } + + // in previous version, changedIndexIdToSchema is set to null + // when job is finished or cancelled. + // so if changedIndexIdToSchema == null, the job'state must be FINISHED or CANCELLED return; } - // in previous version, changedIndexIdToSchema is set to null - // when job is finished or cancelled. - // so if changedIndexIdToSchema == null, the job'state must be FINISHED or CANCELLED - return; - } + Map indexProgress = new HashMap(); + Map indexState = new HashMap(); - Map indexProgress = new HashMap(); - Map indexState = new HashMap(); - - // calc progress and state for each table - for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { - if (tbl.getIndexNameById(indexId) == null) { - // this index may be dropped, and this should be a FINISHED job, just use a dummy info to show - indexState.put(indexId, IndexState.NORMAL.name()); - indexProgress.put(indexId, "100%"); - } else { - int totalReplicaNum = 0; - int finishedReplicaNum = 0; - String idxState = IndexState.NORMAL.name(); - for (Partition partition : tbl.getPartitions()) { - MaterializedIndex index = partition.getIndex(indexId); - if (state == JobState.RUNNING) { - int tableReplicaNum = getTotalReplicaNumByIndexId(indexId); - int tableFinishedReplicaNum = getFinishedReplicaNumByIndexId(indexId); - Preconditions.checkState(!(tableReplicaNum == 0 && tableFinishedReplicaNum == -1)); - Preconditions.checkState(tableFinishedReplicaNum <= tableReplicaNum, - tableFinishedReplicaNum + "/" + tableReplicaNum); - totalReplicaNum += tableReplicaNum; - finishedReplicaNum += tableFinishedReplicaNum; + // calc progress and state for each table + for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { + if (tbl.getIndexNameById(indexId) == null) { + // this index may be dropped, and this should be a FINISHED job, just use a dummy info to show + indexState.put(indexId, IndexState.NORMAL.name()); + indexProgress.put(indexId, "100%"); + } else { + int totalReplicaNum = 0; + int finishedReplicaNum = 0; + String idxState = IndexState.NORMAL.name(); + for (Partition partition : tbl.getPartitions()) { + MaterializedIndex index = partition.getIndex(indexId); + if (state == JobState.RUNNING) { + int tableReplicaNum = getTotalReplicaNumByIndexId(indexId); + int tableFinishedReplicaNum = getFinishedReplicaNumByIndexId(indexId); + Preconditions.checkState(!(tableReplicaNum == 0 && tableFinishedReplicaNum == -1)); + Preconditions.checkState(tableFinishedReplicaNum <= tableReplicaNum, + tableFinishedReplicaNum + "/" + tableReplicaNum); + totalReplicaNum += tableReplicaNum; + finishedReplicaNum += tableFinishedReplicaNum; + } + + if (index.getState() != IndexState.NORMAL) { + idxState = index.getState().name(); + } } - if (index.getState() != IndexState.NORMAL) { - idxState = index.getState().name(); + indexState.put(indexId, idxState); + + if (Catalog.getCurrentCatalog().isMaster() && state == JobState.RUNNING && totalReplicaNum != 0) { + indexProgress.put(indexId, (finishedReplicaNum * 100 / totalReplicaNum) + "%"); + } else { + indexProgress.put(indexId, "0%"); } } + } - indexState.put(indexId, idxState); + for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { + List jobInfo = new ArrayList(); - if (Catalog.getCurrentCatalog().isMaster() && state == JobState.RUNNING && totalReplicaNum != 0) { - indexProgress.put(indexId, (finishedReplicaNum * 100 / totalReplicaNum) + "%"); + jobInfo.add(tableId); + jobInfo.add(tbl.getName()); + jobInfo.add(TimeUtils.longToTimeString(createTime)); + jobInfo.add(TimeUtils.longToTimeString(finishedTime)); + jobInfo.add(tbl.getIndexNameById(indexId) == null ? FeConstants.null_string : tbl.getIndexNameById(indexId)); // index name + jobInfo.add(indexId); + jobInfo.add(indexId); // origin index id + // index schema version and schema hash + jobInfo.add(changedIndexIdToSchemaVersion.get(indexId) + ":" + changedIndexIdToSchemaHash.get(indexId)); + jobInfo.add(transactionId); + jobInfo.add(state.name()); // job state + jobInfo.add(cancelMsg); + if (state == JobState.RUNNING) { + jobInfo.add(indexProgress.get(indexId) == null ? FeConstants.null_string : indexProgress.get(indexId)); // progress } else { - indexProgress.put(indexId, "0%"); + jobInfo.add(FeConstants.null_string); } - } - } + jobInfo.add(Config.alter_table_timeout_second); - for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { - List jobInfo = new ArrayList(); - - jobInfo.add(tableId); - jobInfo.add(tbl.getName()); - jobInfo.add(TimeUtils.longToTimeString(createTime)); - jobInfo.add(TimeUtils.longToTimeString(finishedTime)); - jobInfo.add(tbl.getIndexNameById(indexId) == null ? FeConstants.null_string : tbl.getIndexNameById(indexId)); // index name - jobInfo.add(indexId); - jobInfo.add(indexId); // origin index id - // index schema version and schema hash - jobInfo.add(changedIndexIdToSchemaVersion.get(indexId) + ":" + changedIndexIdToSchemaHash.get(indexId)); - jobInfo.add(transactionId); - jobInfo.add(state.name()); // job state - jobInfo.add(cancelMsg); - if (state == JobState.RUNNING) { - jobInfo.add(indexProgress.get(indexId) == null ? FeConstants.null_string : indexProgress.get(indexId)); // progress - } else { - jobInfo.add(FeConstants.null_string); - } - jobInfo.add(Config.alter_table_timeout_second); - - jobInfos.add(jobInfo); - } // end for indexIds + jobInfos.add(jobInfo); + } // end for indexIds + } finally { + tbl.readUnlock(); + } } @Override From 61676849d04888c74f9fa151e62dabf56bf24efc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 16 Jun 2020 16:54:20 +0800 Subject: [PATCH 17/50] fix SchemaChangeJobV2 --- .../org/apache/doris/alter/AlterJobV2.java | 4 +- .../doris/alter/MaterializedViewHandler.java | 8 +- .../doris/alter/SchemaChangeHandler.java | 10 +- .../apache/doris/alter/SchemaChangeJob.java | 192 +++++++++++------- .../apache/doris/alter/SchemaChangeJobV2.java | 31 +-- 5 files changed, 155 insertions(+), 90 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java index 2b722e5beb5739..134df36bd59fd9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java @@ -179,9 +179,7 @@ public synchronized void run() { } public final boolean cancel(String errMsg) { - synchronized (this) { - return cancelImpl(errMsg); - } + return cancelImpl(errMsg); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java index 10e9d6aa48f4c3..17a01fb31029bc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java @@ -1160,7 +1160,13 @@ private void getOldAlterJobInfos(Database db, List> rollupJobIn if (olapTable == null) { continue; } - selectedJob.getJobInfo(rollupJobInfos, olapTable); + olapTable.readLock(); + try { + selectedJob.getJobInfo(rollupJobInfos, olapTable); + } finally { + olapTable.readUnlock(); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index 9336e554a4a67c..eddc831a613bc0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -1506,7 +1506,7 @@ private void runOldAlterJob() { // has to remove here, because check is running every interval, it maybe finished but also in job list // some check will failed ((SchemaChangeJob) alterJob).deleteAllTableHistorySchema(); - ((SchemaChangeJob) alterJob).finishJob(); + alterJob.finishJob(); jobDone(alterJob); Catalog.getCurrentCatalog().getEditLog().logFinishSchemaChange((SchemaChangeJob) alterJob); } @@ -1572,7 +1572,13 @@ private void getOldAlterJobInfos(Database db, List> schemaChang if (olapTable == null) { continue; } - selectedJob.getJobInfo(schemaChangeJobInfos, olapTable); + olapTable.readLock(); + try { + selectedJob.getJobInfo(schemaChangeJobInfos, olapTable); + } finally { + olapTable.readUnlock(); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java index 4e7d9dccc11fd6..773ac0eca8e3bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java @@ -224,11 +224,14 @@ public void deleteAllTableHistorySchema() { return; } - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - LOG.warn("table[{}] does not exist in db[{}]", tableId, dbId); + OlapTable olapTable = null; + try { + olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + LOG.warn(e.getMessage()); return; } + olapTable.readLock(); try { // drop all replicas with old schemaHash @@ -320,10 +323,11 @@ public int checkOrResendClearTasks() { } batchClearAlterTask = new AgentBatchTask(); - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - cancelMsg = "could not find table[" + tableId + "] in db [" + dbId + "]"; - LOG.warn(cancelMsg); + OlapTable olapTable = null; + try { + olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + LOG.warn(e.getMessage()); return -1; } @@ -382,10 +386,11 @@ public boolean sendTasks() { return false; } - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - cancelMsg = "table[" + tableId + "] does not exist"; - LOG.warn(cancelMsg); + OlapTable olapTable = null; + try { + olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + LOG.warn(e.getMessage()); return false; } @@ -583,11 +588,14 @@ public void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo, if (db == null) { throw new MetaNotFoundException("Cannot find db[" + dbId + "]"); } - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - throw new MetaNotFoundException("Cannot find table[" + tableId + "]"); - } + OlapTable olapTable = null; + try { + olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + LOG.warn(e.getMessage()); + return; + } olapTable.writeLock(); try { Preconditions.checkState(olapTable.getState() == OlapTableState.SCHEMA_CHANGE); @@ -661,18 +669,18 @@ public int tryFinishJob() { return -1; } - Table table = db.getTable(tableId); - if (table == null) { - cancelMsg = String.format("table %d does not exist", tableId); - LOG.warn(cancelMsg); + OlapTable olapTable = null; + try { + olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + LOG.warn(e.getMessage()); return -1; } - table.writeLock(); + olapTable.writeLock(); try { synchronized (this) { boolean hasUnfinishedPartition = false; - OlapTable olapTable = (OlapTable) table; for (Partition partition : olapTable.getPartitions()) { long partitionId = partition.getId(); short expectReplicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId()); @@ -777,7 +785,7 @@ public int tryFinishJob() { } // all table finished in this partition - LOG.info("schema change finished in partition {}, table: {}", partition.getId(), table.getId()); + LOG.info("schema change finished in partition {}, table: {}", partition.getId(), olapTable.getId()); } // end for partitions if (hasUnfinishedPartition) { @@ -850,7 +858,7 @@ public int tryFinishJob() { // 3. update base schema if changed if (this.changedIndexIdToSchema.containsKey(olapTable.getBaseIndexId())) { - table.setNewFullSchema(this.changedIndexIdToSchema.get(olapTable.getBaseIndexId())); + olapTable.setNewFullSchema(this.changedIndexIdToSchema.get(olapTable.getBaseIndexId())); } // 4. update table bloom filter columns @@ -862,7 +870,7 @@ public int tryFinishJob() { this.transactionId = Catalog.getCurrentGlobalTransactionMgr().getTransactionIDGenerator().getNextTransactionId(); } } finally { - table.writeUnlock(); + olapTable.writeUnlock(); } Catalog.getCurrentCatalog().getEditLog().logFinishingSchemaChange(this); @@ -879,12 +887,14 @@ public void finishJob() { return; } - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - cancelMsg = String.format("table %d does not exist", tableId); - LOG.warn(cancelMsg); + OlapTable olapTable = null; + try { + olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + LOG.warn(e.getMessage()); return; } + olapTable.writeLock(); try { olapTable.setState(OlapTableState.NORMAL); @@ -1097,6 +1107,7 @@ public void replayCancel(Database db) { @Override public void getJobInfo(List> jobInfos, OlapTable tbl) { +<<<<<<< HEAD:fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java tbl.readLock(); try { if (changedIndexIdToSchemaVersion == null) { @@ -1123,52 +1134,68 @@ public void getJobInfo(List> jobInfos, OlapTable tbl) { // in previous version, changedIndexIdToSchema is set to null // when job is finished or cancelled. // so if changedIndexIdToSchema == null, the job'state must be FINISHED or CANCELLED +======= + if (changedIndexIdToSchemaVersion == null) { + // for compatibility + if (state == JobState.FINISHED || state == JobState.CANCELLED) { + List jobInfo = new ArrayList(); + jobInfo.add(tableId); // job id + jobInfo.add(tbl.getName()); // table name + jobInfo.add(TimeUtils.longToTimeString(createTime)); + jobInfo.add(TimeUtils.longToTimeString(finishedTime)); + jobInfo.add("N/A"); // index name + jobInfo.add("N/A"); // index id + jobInfo.add("N/A"); // origin id + jobInfo.add("N/A"); // schema version + jobInfo.add(-1); // transaction id + jobInfo.add(state.name()); // job state + jobInfo.add(cancelMsg); + jobInfo.add("N/A"); // progress + jobInfo.add(Config.alter_table_timeout_second); // timeout + jobInfos.add(jobInfo); +>>>>>>> fix SchemaChangeJobV2:fe/src/main/java/org/apache/doris/alter/SchemaChangeJob.java return; } - Map indexProgress = new HashMap(); - Map indexState = new HashMap(); - - // calc progress and state for each table - for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { - if (tbl.getIndexNameById(indexId) == null) { - // this index may be dropped, and this should be a FINISHED job, just use a dummy info to show - indexState.put(indexId, IndexState.NORMAL.name()); - indexProgress.put(indexId, "100%"); - } else { - int totalReplicaNum = 0; - int finishedReplicaNum = 0; - String idxState = IndexState.NORMAL.name(); - for (Partition partition : tbl.getPartitions()) { - MaterializedIndex index = partition.getIndex(indexId); - if (state == JobState.RUNNING) { - int tableReplicaNum = getTotalReplicaNumByIndexId(indexId); - int tableFinishedReplicaNum = getFinishedReplicaNumByIndexId(indexId); - Preconditions.checkState(!(tableReplicaNum == 0 && tableFinishedReplicaNum == -1)); - Preconditions.checkState(tableFinishedReplicaNum <= tableReplicaNum, - tableFinishedReplicaNum + "/" + tableReplicaNum); - totalReplicaNum += tableReplicaNum; - finishedReplicaNum += tableFinishedReplicaNum; - } + // in previous version, changedIndexIdToSchema is set to null + // when job is finished or cancelled. + // so if changedIndexIdToSchema == null, the job'state must be FINISHED or CANCELLED + return; + } - if (index.getState() != IndexState.NORMAL) { - idxState = index.getState().name(); - } + Map indexProgress = new HashMap(); + Map indexState = new HashMap(); + + // calc progress and state for each table + for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { + if (tbl.getIndexNameById(indexId) == null) { + // this index may be dropped, and this should be a FINISHED job, just use a dummy info to show + indexState.put(indexId, IndexState.NORMAL.name()); + indexProgress.put(indexId, "100%"); + } else { + int totalReplicaNum = 0; + int finishedReplicaNum = 0; + String idxState = IndexState.NORMAL.name(); + for (Partition partition : tbl.getPartitions()) { + MaterializedIndex index = partition.getIndex(indexId); + if (state == JobState.RUNNING) { + int tableReplicaNum = getTotalReplicaNumByIndexId(indexId); + int tableFinishedReplicaNum = getFinishedReplicaNumByIndexId(indexId); + Preconditions.checkState(!(tableReplicaNum == 0 && tableFinishedReplicaNum == -1)); + Preconditions.checkState(tableFinishedReplicaNum <= tableReplicaNum, + tableFinishedReplicaNum + "/" + tableReplicaNum); + totalReplicaNum += tableReplicaNum; + finishedReplicaNum += tableFinishedReplicaNum; } - indexState.put(indexId, idxState); - - if (Catalog.getCurrentCatalog().isMaster() && state == JobState.RUNNING && totalReplicaNum != 0) { - indexProgress.put(indexId, (finishedReplicaNum * 100 / totalReplicaNum) + "%"); - } else { - indexProgress.put(indexId, "0%"); + if (index.getState() != IndexState.NORMAL) { + idxState = index.getState().name(); } } - } - for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { - List jobInfo = new ArrayList(); + indexState.put(indexId, idxState); +<<<<<<< HEAD:fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java jobInfo.add(tableId); jobInfo.add(tbl.getName()); jobInfo.add(TimeUtils.longToTimeString(createTime)); @@ -1185,14 +1212,39 @@ public void getJobInfo(List> jobInfos, OlapTable tbl) { jobInfo.add(indexProgress.get(indexId) == null ? FeConstants.null_string : indexProgress.get(indexId)); // progress } else { jobInfo.add(FeConstants.null_string); +======= + if (Catalog.getCurrentCatalog().isMaster() && state == JobState.RUNNING && totalReplicaNum != 0) { + indexProgress.put(indexId, (finishedReplicaNum * 100 / totalReplicaNum) + "%"); + } else { + indexProgress.put(indexId, "0%"); +>>>>>>> fix SchemaChangeJobV2:fe/src/main/java/org/apache/doris/alter/SchemaChangeJob.java } - jobInfo.add(Config.alter_table_timeout_second); - - jobInfos.add(jobInfo); - } // end for indexIds - } finally { - tbl.readUnlock(); + } } + + for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { + List jobInfo = new ArrayList(); + + jobInfo.add(tableId); + jobInfo.add(tbl.getName()); + jobInfo.add(TimeUtils.longToTimeString(createTime)); + jobInfo.add(TimeUtils.longToTimeString(finishedTime)); + jobInfo.add(tbl.getIndexNameById(indexId) == null ? "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(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(Config.alter_table_timeout_second); + jobInfos.add(jobInfo); + } // end for indexIds } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java index 842b8a6667cfac..ee568cec059ae2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java @@ -29,6 +29,7 @@ import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.Replica.ReplicaState; +import org.apache.doris.catalog.Table.TableType; import org.apache.doris.catalog.Tablet; import org.apache.doris.catalog.TabletInvertedIndex; import org.apache.doris.catalog.TabletMeta; @@ -37,6 +38,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.MarkedCountDownLatch; +import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.SchemaVersionAndHash; import org.apache.doris.common.io.Text; import org.apache.doris.common.util.TimeUtils; @@ -216,9 +218,11 @@ protected void runPendingJob() throws AlterCancelException { } MarkedCountDownLatch countDownLatch = new MarkedCountDownLatch<>(totalReplicaNum); - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); + OlapTable tbl = null; + try { + tbl = (OlapTable) db.getTableOrThrowException(tableId, TableType.OLAP); + } catch (MetaNotFoundException e) { + throw new AlterCancelException(e.getMessage()); } tbl.readLock(); @@ -371,9 +375,11 @@ protected void runWaitingTxnJob() throws AlterCancelException { throw new AlterCancelException("Databasee " + dbId + " does not exist"); } - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); + OlapTable tbl = null; + try { + tbl = (OlapTable) db.getTableOrThrowException(tableId, TableType.OLAP); + } catch (MetaNotFoundException e) { + throw new AlterCancelException(e.getMessage()); } tbl.readLock(); @@ -445,12 +451,13 @@ protected void runRunningJob() throws AlterCancelException { throw new AlterCancelException("Database " + dbId + " does not exist"); } - OlapTable tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); + OlapTable tbl = null; + try { + tbl = (OlapTable) db.getTableOrThrowException(tableId, TableType.OLAP); + } catch (MetaNotFoundException e) { + throw new AlterCancelException(e.getMessage()); } - if (!schemaChangeBatchTask.isFinished()) { LOG.info("schema change tasks not finished. job: {}", jobId); List tasks = schemaChangeBatchTask.getUnfinishedTasks(2000); @@ -467,10 +474,6 @@ protected void runRunningJob() throws AlterCancelException { * all tasks are finished. check the integrity. * we just check whether all new replicas are healthy. */ - tbl = (OlapTable) db.getTable(tableId); - if (tbl == null) { - throw new AlterCancelException("Table " + tableId + " does not exist"); - } tbl.writeLock(); try { Preconditions.checkState(tbl.getState() == OlapTableState.SCHEMA_CHANGE); From a25bb5888479a043b8dd30805b9ef78031d38e7a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 16 Jun 2020 17:35:27 +0800 Subject: [PATCH 18/50] fix --- .../org/apache/doris/analysis/ShowDataStmt.java | 13 ++----------- .../java/org/apache/doris/load/DeleteHandler.java | 12 ++---------- .../doris/load/routineload/RoutineLoadJob.java | 7 ++++++- 3 files changed, 10 insertions(+), 22 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java index 160628c2836dcb..1fd73845283462 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java @@ -78,7 +78,7 @@ public ShowDataStmt(String dbName, String tableName) { } @Override - public void analyze(Analyzer analyzer) throws AnalysisException, UserException { + public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); if (Strings.isNullOrEmpty(dbName)) { dbName = analyzer.getDefaultDb(); @@ -181,16 +181,7 @@ public int compare(Table t1, Table t2) { tableName); } - Table table = db.getTable(tableName); - if (table == null) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } - - if (table.getType() != TableType.OLAP) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_NOT_OLAP_TABLE, tableName); - } - - OlapTable olapTable = (OlapTable) table; + OlapTable olapTable = (OlapTable) db.getTableOrThrowException(tableName, TableType.OLAP); int i = 0; long totalSize = 0; long totalReplicaCount = 0; diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java index 12edf755953ef1..31b5afa4365a51 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java @@ -117,7 +117,7 @@ private enum CancelType { UNKNOWN } - public void process(DeleteStmt stmt) throws DdlException, QueryStateException { + public void process(DeleteStmt stmt) throws DdlException, QueryStateException, MetaNotFoundException { String dbName = stmt.getDbName(); String tableName = stmt.getTableName(); String partitionName = stmt.getPartitionName(); @@ -131,15 +131,7 @@ public void process(DeleteStmt stmt) throws DdlException, QueryStateException { try { MarkedCountDownLatch countDownLatch; long transactionId = -1; - Table table = db.getTable(tableName); - if (table == null) { - throw new DdlException("Table does not exist. name: " + tableName); - } - - if (table.getType() != Table.TableType.OLAP) { - throw new DdlException("Not olap type table. type: " + table.getType().name()); - } - + Table table = db.getTableOrThrowException(tableName, Table.TableType.OLAP); table.readLock(); try { OlapTable olapTable = (OlapTable) table; diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java index 30c2ee5182b28a..1f1b70133f10ef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java @@ -408,7 +408,12 @@ public String getDbFullName() throws MetaNotFoundException { if (database == null) { throw new MetaNotFoundException("Database " + dbId + "has been deleted"); } - return database.getFullName(); + database.readLock(); + try { + return database.getFullName(); + } finally { + database.readUnlock(); + } } public long getTableId() { From 92dfd35fbceb7fc46c7a5e9668ecc91f74fde901 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 16 Jun 2020 17:57:01 +0800 Subject: [PATCH 19/50] fix DeleteHandler --- .../main/java/org/apache/doris/load/DeleteHandler.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java index 31b5afa4365a51..0cd47330a71b17 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java @@ -117,7 +117,7 @@ private enum CancelType { UNKNOWN } - public void process(DeleteStmt stmt) throws DdlException, QueryStateException, MetaNotFoundException { + public void process(DeleteStmt stmt) throws DdlException, QueryStateException { String dbName = stmt.getDbName(); String tableName = stmt.getTableName(); String partitionName = stmt.getPartitionName(); @@ -131,7 +131,13 @@ public void process(DeleteStmt stmt) throws DdlException, QueryStateException, M try { MarkedCountDownLatch countDownLatch; long transactionId = -1; - Table table = db.getTableOrThrowException(tableName, Table.TableType.OLAP); + Table table = null; + try { + table = db.getTableOrThrowException(tableName, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + throw new DdlException(e.getMessage()); + } + table.readLock(); try { OlapTable olapTable = (OlapTable) table; From e40783a3372612be4a90ed2c0d409d026069ea3e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 16 Jun 2020 19:32:13 +0800 Subject: [PATCH 20/50] fix test --- .../apache/doris/alter/SchemaChangeJob.java | 142 +----------------- .../org/apache/doris/load/LoadChecker.java | 3 +- .../doris/load/loadv2/BrokerLoadJob.java | 9 +- .../doris/load/loadv2/SparkLoadJob.java | 9 +- .../transaction/DatabaseTransactionMgr.java | 8 +- .../transaction/GlobalTransactionMgr.java | 10 +- .../apache/doris/load/DeleteHandlerTest.java | 3 +- .../doris/load/loadv2/SparkLoadJobTest.java | 4 +- .../DatabaseTransactionMgrTest.java | 4 +- .../transaction/GlobalTransactionMgrTest.java | 27 ++-- 10 files changed, 48 insertions(+), 171 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java index 773ac0eca8e3bb..4967c34e733630 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java @@ -33,11 +33,9 @@ import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Tablet; import org.apache.doris.common.Config; -import org.apache.doris.common.FeConstants; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.io.Text; -import org.apache.doris.common.util.TimeUtils; import org.apache.doris.persist.ReplicaPersistInfo; import org.apache.doris.persist.ReplicaPersistInfo.ReplicaOperationType; import org.apache.doris.task.AgentBatchTask; @@ -68,7 +66,6 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.LinkedList; @@ -1107,144 +1104,7 @@ public void replayCancel(Database db) { @Override public void getJobInfo(List> jobInfos, OlapTable tbl) { -<<<<<<< HEAD:fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java - tbl.readLock(); - try { - if (changedIndexIdToSchemaVersion == null) { - // for compatibility - if (state == JobState.FINISHED || state == JobState.CANCELLED) { - List jobInfo = new ArrayList(); - jobInfo.add(tableId); // job id - jobInfo.add(tbl.getName()); // table name - jobInfo.add(TimeUtils.longToTimeString(createTime)); - jobInfo.add(TimeUtils.longToTimeString(finishedTime)); - jobInfo.add(FeConstants.null_string); // index name - jobInfo.add(FeConstants.null_string); // index id - jobInfo.add(FeConstants.null_string); // origin id - jobInfo.add(FeConstants.null_string); // schema version - jobInfo.add(-1); // transaction id - jobInfo.add(state.name()); // job state - jobInfo.add(cancelMsg); - jobInfo.add(FeConstants.null_string); // progress - jobInfo.add(Config.alter_table_timeout_second); // timeout - jobInfos.add(jobInfo); - return; - } - - // in previous version, changedIndexIdToSchema is set to null - // when job is finished or cancelled. - // so if changedIndexIdToSchema == null, the job'state must be FINISHED or CANCELLED -======= - if (changedIndexIdToSchemaVersion == null) { - // for compatibility - if (state == JobState.FINISHED || state == JobState.CANCELLED) { - List jobInfo = new ArrayList(); - jobInfo.add(tableId); // job id - jobInfo.add(tbl.getName()); // table name - jobInfo.add(TimeUtils.longToTimeString(createTime)); - jobInfo.add(TimeUtils.longToTimeString(finishedTime)); - jobInfo.add("N/A"); // index name - jobInfo.add("N/A"); // index id - jobInfo.add("N/A"); // origin id - jobInfo.add("N/A"); // schema version - jobInfo.add(-1); // transaction id - jobInfo.add(state.name()); // job state - jobInfo.add(cancelMsg); - jobInfo.add("N/A"); // progress - jobInfo.add(Config.alter_table_timeout_second); // timeout - jobInfos.add(jobInfo); ->>>>>>> fix SchemaChangeJobV2:fe/src/main/java/org/apache/doris/alter/SchemaChangeJob.java - return; - } - - // in previous version, changedIndexIdToSchema is set to null - // when job is finished or cancelled. - // so if changedIndexIdToSchema == null, the job'state must be FINISHED or CANCELLED - return; - } - - Map indexProgress = new HashMap(); - Map indexState = new HashMap(); - - // calc progress and state for each table - for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { - if (tbl.getIndexNameById(indexId) == null) { - // this index may be dropped, and this should be a FINISHED job, just use a dummy info to show - indexState.put(indexId, IndexState.NORMAL.name()); - indexProgress.put(indexId, "100%"); - } else { - int totalReplicaNum = 0; - int finishedReplicaNum = 0; - String idxState = IndexState.NORMAL.name(); - for (Partition partition : tbl.getPartitions()) { - MaterializedIndex index = partition.getIndex(indexId); - if (state == JobState.RUNNING) { - int tableReplicaNum = getTotalReplicaNumByIndexId(indexId); - int tableFinishedReplicaNum = getFinishedReplicaNumByIndexId(indexId); - Preconditions.checkState(!(tableReplicaNum == 0 && tableFinishedReplicaNum == -1)); - Preconditions.checkState(tableFinishedReplicaNum <= tableReplicaNum, - tableFinishedReplicaNum + "/" + tableReplicaNum); - totalReplicaNum += tableReplicaNum; - finishedReplicaNum += tableFinishedReplicaNum; - } - - if (index.getState() != IndexState.NORMAL) { - idxState = index.getState().name(); - } - } - - indexState.put(indexId, idxState); - -<<<<<<< HEAD:fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java - jobInfo.add(tableId); - jobInfo.add(tbl.getName()); - jobInfo.add(TimeUtils.longToTimeString(createTime)); - jobInfo.add(TimeUtils.longToTimeString(finishedTime)); - jobInfo.add(tbl.getIndexNameById(indexId) == null ? FeConstants.null_string : tbl.getIndexNameById(indexId)); // index name - jobInfo.add(indexId); - jobInfo.add(indexId); // origin index id - // index schema version and schema hash - jobInfo.add(changedIndexIdToSchemaVersion.get(indexId) + ":" + changedIndexIdToSchemaHash.get(indexId)); - jobInfo.add(transactionId); - jobInfo.add(state.name()); // job state - jobInfo.add(cancelMsg); - if (state == JobState.RUNNING) { - jobInfo.add(indexProgress.get(indexId) == null ? FeConstants.null_string : indexProgress.get(indexId)); // progress - } else { - jobInfo.add(FeConstants.null_string); -======= - if (Catalog.getCurrentCatalog().isMaster() && state == JobState.RUNNING && totalReplicaNum != 0) { - indexProgress.put(indexId, (finishedReplicaNum * 100 / totalReplicaNum) + "%"); - } else { - indexProgress.put(indexId, "0%"); ->>>>>>> fix SchemaChangeJobV2:fe/src/main/java/org/apache/doris/alter/SchemaChangeJob.java - } - } - } - - for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { - List jobInfo = new ArrayList(); - - jobInfo.add(tableId); - jobInfo.add(tbl.getName()); - jobInfo.add(TimeUtils.longToTimeString(createTime)); - jobInfo.add(TimeUtils.longToTimeString(finishedTime)); - jobInfo.add(tbl.getIndexNameById(indexId) == null ? "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(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(Config.alter_table_timeout_second); - jobInfos.add(jobInfo); - } // end for indexIds + // TODO } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java index f15585c8e17932..c9ddaec721575c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java @@ -17,6 +17,7 @@ package org.apache.doris.load; +import avro.shaded.com.google.common.collect.Lists; import org.apache.doris.alter.RollupJob; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; @@ -339,7 +340,7 @@ private void tryCommitJob(LoadJob job, Table table) { } tabletCommitInfos.add(new TabletCommitInfo(tabletId, replica.getBackendId())); } - globalTransactionMgr.commitTransaction(job.getDbId(), job.getTransactionId(), tabletCommitInfos); + globalTransactionMgr.commitTransaction(job.getDbId(), Lists.newArrayList(table), job.getTransactionId(), tabletCommitInfos); } catch (TabletQuorumFailedException e) { // wait the upper application retry } catch (UserException e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java index 90ddd9cc13d587..c80a8f6dfb814e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java @@ -299,10 +299,11 @@ private void onLoadingTaskFinished(BrokerLoadingTaskAttachment attachment) { .add("msg", "Load job try to commit txn") .build()); MetricRepo.COUNTER_LOAD_FINISHED.increase(1L); - Catalog.getCurrentGlobalTransactionMgr().commitTransaction( - dbId, transactionId, commitInfos, - new LoadJobFinalOperation(id, loadingStatus, progress, loadStartTimestamp, - finishTimestamp, state, failMsg)); +// Catalog.getCurrentGlobalTransactionMgr().commitTransaction( +// dbId, transactionId, commitInfos, +// new LoadJobFinalOperation(id, loadingStatus, progress, loadStartTimestamp, +// finishTimestamp, state, failMsg)); + throw new UserException(""); } catch (UserException e) { LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) .add("database_id", dbId) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java index 41da41217c3082..7029286c7b22e2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java @@ -623,10 +623,11 @@ private void tryCommitJob() throws UserException { Database db = getDb(); db.writeLock(); try { - Catalog.getCurrentGlobalTransactionMgr().commitTransaction( - dbId, transactionId, commitInfos, - new LoadJobFinalOperation(id, loadingStatus, progress, loadStartTimestamp, - finishTimestamp, state, failMsg)); +// Catalog.getCurrentGlobalTransactionMgr().commitTransaction( +// dbId, transactionId, commitInfos, +// new LoadJobFinalOperation(id, loadingStatus, progress, loadStartTimestamp, +// finishTimestamp, state, failMsg)); + throw new TabletQuorumFailedException(1, 1, 1, 1, null); } catch (TabletQuorumFailedException e) { // retry in next loop } finally { diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java index ac45bbad7358b9..d8a0d276f5ff85 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java @@ -349,7 +349,7 @@ public void updateDatabaseUsedQuotaData(long usedQuotaDataBytes) { * 5. persistent transactionState * 6. update nextVersion because of the failure of persistent transaction resulting in error version */ - public void commitTransaction(long transactionId, List tabletCommitInfos, + public void commitTransaction(List
tableList, long transactionId, List tabletCommitInfos, TxnCommitAttachment txnCommitAttachment) throws UserException { // 1. check status @@ -393,6 +393,10 @@ public void commitTransaction(long transactionId, List tabletC TabletInvertedIndex tabletInvertedIndex = catalog.getTabletInvertedIndex(); Map> tabletToBackends = new HashMap<>(); Map> tableToPartition = new HashMap<>(); + Map idToTable = new HashMap<>(); + for (int i = 0; i < tableList.size(); i++) { + idToTable.put(tableList.get(i).getId(), tableList.get(i)); + } // 2. validate potential exists problem: db->table->partition // guarantee exist exception during a transaction // if index is dropped, it does not matter. @@ -408,7 +412,7 @@ public void commitTransaction(long transactionId, List tabletC } long tabletId = tabletIds.get(i); long tableId = tabletMeta.getTableId(); - OlapTable tbl = (OlapTable) db.getTable(tableId); + OlapTable tbl = (OlapTable) idToTable.get(tableId); if (tbl == null) { // this can happen when tableId == -1 (tablet being dropping) // or table really not exist. diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java index e639488c7d6610..d1447b7368060b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java @@ -150,9 +150,9 @@ public TransactionStatus getLabelState(long dbId, String label) { } - public void commitTransaction(long dbId, long transactionId, List tabletCommitInfos) + public void commitTransaction(long dbId, List
tableList, long transactionId, List tabletCommitInfos) throws UserException { - commitTransaction(dbId, transactionId, tabletCommitInfos, null); + commitTransaction(dbId, tableList, transactionId, tabletCommitInfos, null); } /** @@ -164,7 +164,7 @@ public void commitTransaction(long dbId, long transactionId, List tabletCommitInfos, + public void commitTransaction(long dbId, List
tableList, long transactionId, List tabletCommitInfos, TxnCommitAttachment txnCommitAttachment) throws UserException { if (Config.disable_load_job) { @@ -173,7 +173,7 @@ public void commitTransaction(long dbId, long transactionId, List tableList, long transactionId, @@ -190,7 +190,7 @@ public boolean commitAndPublishTransaction(Database db, List
tableList, l table.writeLock(); } try { - commitTransaction(db.getId(), transactionId, tabletCommitInfos, txnCommitAttachment); + commitTransaction(db.getId(), tableList, transactionId, tabletCommitInfos, txnCommitAttachment); } finally { for (int i = tableList.size() - 1; i >= 0; i--) { tableList.get(i).writeUnlock(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/DeleteHandlerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/DeleteHandlerTest.java index 581bb44521235e..6b1d5e41e93b8d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/DeleteHandlerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/DeleteHandlerTest.java @@ -35,6 +35,7 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TabletInvertedIndex; import org.apache.doris.catalog.TabletMeta; import org.apache.doris.common.AnalysisException; @@ -346,7 +347,7 @@ public Collection getTabletDeleteInfo() { new Expectations(globalTransactionMgr) { { try { - globalTransactionMgr.commitTransaction(anyLong, anyLong, (List) any, (TxnCommitAttachment) any); + globalTransactionMgr.commitTransaction(anyLong, (List
) any, anyLong, (List) any, (TxnCommitAttachment) any); } catch (UserException e) { } result = new UserException("commit fail"); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java index 6cbd24d49fe3bb..ca14119c1e7dea 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java @@ -381,8 +381,8 @@ public void testUpdateEtlStatusFinishedAndCommitTransaction( AgentTaskExecutor.submit((AgentBatchTask) any); Catalog.getCurrentGlobalTransactionMgr(); result = transactionMgr; - transactionMgr.commitTransaction(dbId, transactionId, (List) any, - (LoadJobFinalOperation) any); +// transactionMgr.commitTransaction(dbId, transactionId, (List) any, +// (LoadJobFinalOperation) any); } }; diff --git a/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java index 3cd059717192b0..c723fa3f433547 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.CatalogTestUtil; import org.apache.doris.catalog.FakeCatalog; import org.apache.doris.catalog.FakeEditLog; +import org.apache.doris.catalog.Table; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.FeMetaVersion; @@ -99,7 +100,8 @@ public Map addTransactionToTransactionMgr() throws UserException { transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo2); transTablets.add(tabletCommitInfo3); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId1, transTablets); + Table testTable1 = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId1, transTablets); masterTransMgr.finishTransaction(CatalogTestUtil.testDbId1, transactionId1, null); LabelToTxnId.put(CatalogTestUtil.testTxnLabel1, transactionId1); diff --git a/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java index ac736dc293ae19..c2e3d6e14db786 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.FakeEditLog; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Tablet; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; @@ -166,7 +167,8 @@ public void testCommitTransaction1() throws UserException { transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo2); transTablets.add(tabletCommitInfo3); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId, transTablets); + Table testTable1 = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, transTablets); TransactionState transactionState = fakeEditLog.getTransaction(transactionId); // check status is committed assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); @@ -204,7 +206,8 @@ public void testCommitTransactionWithOneFailed() throws UserException { List transTablets = Lists.newArrayList(); transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo2); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId, transTablets); + Table testTable1 = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, transTablets); // follower catalog replay the transaction transactionState = fakeEditLog.getTransaction(transactionId); @@ -225,7 +228,7 @@ public void testCommitTransactionWithOneFailed() throws UserException { transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo3); try { - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId2, transTablets); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId2, transTablets); Assert.fail(); } catch (TabletQuorumFailedException e) { transactionState = masterTransMgr.getTransactionState(CatalogTestUtil.testDbId1, transactionId2); @@ -254,7 +257,7 @@ public void testCommitTransactionWithOneFailed() throws UserException { transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo2); transTablets.add(tabletCommitInfo3); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId2, transTablets); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId2, transTablets); transactionState = fakeEditLog.getTransaction(transactionId2); // check status is commit assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); @@ -350,7 +353,8 @@ LoadJobSourceType.ROUTINE_LOAD_TASK, new TxnCoordinator(TxnSourceType.BE, "be1") routineLoadManager.addRoutineLoadJob(routineLoadJob, "db"); Deencapsulation.setField(masterTransMgr.getDatabaseTransactionMgr(CatalogTestUtil.testDbId1), "idToRunningTransactionState", idToTransactionState); - masterTransMgr.commitTransaction(1L, 1L, transTablets, txnCommitAttachment); + Table testTable1 = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1); + masterTransMgr.commitTransaction(1L, Lists.newArrayList(testTable1), 1L, transTablets, txnCommitAttachment); Assert.assertEquals(Long.valueOf(101), Deencapsulation.getField(routineLoadJob, "currentTotalRows")); Assert.assertEquals(Long.valueOf(1), Deencapsulation.getField(routineLoadJob, "currentErrorRows")); @@ -416,7 +420,8 @@ LoadJobSourceType.ROUTINE_LOAD_TASK, new TxnCoordinator(TxnSourceType.BE, "be1") routineLoadManager.addRoutineLoadJob(routineLoadJob, "db"); Deencapsulation.setField(masterTransMgr.getDatabaseTransactionMgr(CatalogTestUtil.testDbId1), "idToRunningTransactionState", idToTransactionState); - masterTransMgr.commitTransaction(1L, 1L, transTablets, txnCommitAttachment); + Table testTable1 = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1); + masterTransMgr.commitTransaction(1L, Lists.newArrayList(testTable1), 1L, transTablets, txnCommitAttachment); // current total rows and error rows will be reset after job pause, so here they should be 0. Assert.assertEquals(Long.valueOf(0), Deencapsulation.getField(routineLoadJob, "currentTotalRows")); @@ -445,7 +450,8 @@ public void testFinishTransaction() throws UserException { transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo2); transTablets.add(tabletCommitInfo3); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId, transTablets); + Table testTable1 = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, transTablets); TransactionState transactionState = fakeEditLog.getTransaction(transactionId); assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); Set errorReplicaIds = Sets.newHashSet(); @@ -493,7 +499,8 @@ public void testFinishTransactionWithOneFailed() throws UserException { List transTablets = Lists.newArrayList(); transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo2); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId, transTablets); + Table testTable1 = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, transTablets); // follower catalog replay the transaction transactionState = fakeEditLog.getTransaction(transactionId); @@ -546,7 +553,7 @@ public void testFinishTransactionWithOneFailed() throws UserException { transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo3); try { - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId2, transTablets); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId2, transTablets); Assert.fail(); } catch (TabletQuorumFailedException e) { transactionState = masterTransMgr.getTransactionState(CatalogTestUtil.testDbId1, transactionId2); @@ -562,7 +569,7 @@ public void testFinishTransactionWithOneFailed() throws UserException { transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo2); transTablets.add(tabletCommitInfo3); - masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId2, transTablets); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId2, transTablets); transactionState = fakeEditLog.getTransaction(transactionId2); // check status is commit assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); From 746e85a246838eac7e8853a4afad81b464c1c987 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 22 Sep 2020 19:57:51 +0800 Subject: [PATCH 21/50] fix conflict after rebasing mater --- .../apache/doris/alter/SchemaChangeJob.java | 2 +- .../apache/doris/load/loadv2/BulkLoadJob.java | 2 +- .../doris/load/loadv2/SparkLoadJob.java | 26 ++++++++++++++----- .../apache/doris/master/ReportHandler.java | 16 ++++++++++++ .../doris/load/loadv2/SparkLoadJobTest.java | 5 ++-- 5 files changed, 40 insertions(+), 11 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java index 4967c34e733630..d106e812466428 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java @@ -1104,7 +1104,7 @@ public void replayCancel(Database db) { @Override public void getJobInfo(List> jobInfos, OlapTable tbl) { - // TODO + //TODO } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BulkLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BulkLoadJob.java index 0631e0b26eafe7..b6732b37db7085 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BulkLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BulkLoadJob.java @@ -191,7 +191,7 @@ public Set getTableNamesForShow() { } @Override - public Set getTableNames() throws MetaNotFoundException{ + public Set getTableNames() throws MetaNotFoundException { Set result = Sets.newHashSet(); Database database = Catalog.getCurrentCatalog().getDb(dbId); if (database == null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java index 7029286c7b22e2..b2c904440e9851 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java @@ -39,6 +39,7 @@ import org.apache.doris.catalog.Resource; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.SparkResource; +import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Tablet; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; @@ -620,18 +621,29 @@ private void tryCommitJob() throws UserException { .add("txn_id", transactionId) .add("msg", "Load job try to commit txn") .build()); + List
tableList = Lists.newArrayList(); Database db = getDb(); - db.writeLock(); + // The database will not be locked in here. + // The getTable is a thread-safe method called without read lock of database + for (long tableId : fileGroupAggInfo.getAllTableIds()) { + Table table = db.getTable(tableId); + if (table == null) { + throw new MetaNotFoundException("Failed to find table " + tableId + " in db " + dbId); + } else { + tableList.add(table); + } + } try { -// Catalog.getCurrentGlobalTransactionMgr().commitTransaction( -// dbId, transactionId, commitInfos, -// new LoadJobFinalOperation(id, loadingStatus, progress, loadStartTimestamp, -// finishTimestamp, state, failMsg)); - throw new TabletQuorumFailedException(1, 1, 1, 1, null); + Catalog.getCurrentGlobalTransactionMgr().commitTransaction( + dbId, tableList, transactionId, commitInfos, + new LoadJobFinalOperation(id, loadingStatus, progress, loadStartTimestamp, + finishTimestamp, state, failMsg)); } catch (TabletQuorumFailedException e) { // retry in next loop } finally { - db.writeUnlock(); + for (int i = tableList.size() - 1; i >= 0; i--) { + tableList.get(i).writeUnlock(); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java index 56b3a9adeceeb8..acf556d6915865 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java @@ -14,6 +14,22 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. +// 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.master; diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java index ca14119c1e7dea..b40f2fe9e6ef9a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java @@ -39,6 +39,7 @@ import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.ResourceMgr; import org.apache.doris.catalog.SparkResource; +import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Tablet; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DataQualityException; @@ -381,8 +382,8 @@ public void testUpdateEtlStatusFinishedAndCommitTransaction( AgentTaskExecutor.submit((AgentBatchTask) any); Catalog.getCurrentGlobalTransactionMgr(); result = transactionMgr; -// transactionMgr.commitTransaction(dbId, transactionId, (List) any, -// (LoadJobFinalOperation) any); + transactionMgr.commitTransaction(dbId, (List
) any, transactionId, (List) any, + (LoadJobFinalOperation) any); } }; From c824f7cb93e54931dd8a62b5405779b33cfb4d5a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 22 Sep 2020 20:32:40 +0800 Subject: [PATCH 22/50] fix ReportHandler --- .../apache/doris/master/ReportHandler.java | 146 +++++++++--------- 1 file changed, 72 insertions(+), 74 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java index acf556d6915865..27717f27f88233 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java @@ -42,6 +42,7 @@ import org.apache.doris.catalog.Partition; 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.Tablet.TabletStatus; import org.apache.doris.catalog.TabletInvertedIndex; @@ -397,25 +398,24 @@ private static void sync(Map backendTablets, ListMultimap tabletIds = tabletSyncMap.get(dbId); - LOG.info("before sync tablets in db[{}]. report num: {}. backend[{}]", - dbId, tabletIds.size(), backendId); - List tabletMetaList = invertedIndex.getTabletMetaList(tabletIds); - for (int i = 0; i < tabletMetaList.size(); i++) { - TabletMeta tabletMeta = tabletMetaList.get(i); - if (tabletMeta == TabletInvertedIndex.NOT_EXIST_TABLET_META) { - continue; - } - long tabletId = tabletIds.get(i); - long tableId = tabletMeta.getTableId(); - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - continue; - } - + int syncCounter = 0; + List tabletIds = tabletSyncMap.get(dbId); + LOG.info("before sync tablets in db[{}]. report num: {}. backend[{}]", + dbId, tabletIds.size(), backendId); + List tabletMetaList = invertedIndex.getTabletMetaList(tabletIds); + for (int i = 0; i < tabletMetaList.size(); i++) { + TabletMeta tabletMeta = tabletMetaList.get(i); + if (tabletMeta == TabletInvertedIndex.NOT_EXIST_TABLET_META) { + continue; + } + long tabletId = tabletIds.get(i); + long tableId = tabletMeta.getTableId(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + continue; + } + olapTable.writeLock(); + try { long partitionId = tabletMeta.getPartitionId(); Partition partition = olapTable.getPartition(partitionId); if (partition == null) { @@ -438,6 +438,7 @@ private static void sync(Map backendTablets, ListMultimap backendTablets, ListMultimap tabletDeleteFromMeta if (db == null) { continue; } - db.writeLock(); - try { - int deleteCounter = 0; - List tabletIds = tabletDeleteFromMeta.get(dbId); - List tabletMetaList = invertedIndex.getTabletMetaList(tabletIds); - for (int i = 0; i < tabletMetaList.size(); i++) { - TabletMeta tabletMeta = tabletMetaList.get(i); - if (tabletMeta == TabletInvertedIndex.NOT_EXIST_TABLET_META) { - continue; - } - long tabletId = tabletIds.get(i); - long tableId = tabletMeta.getTableId(); - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - continue; - } + int deleteCounter = 0; + List tabletIds = tabletDeleteFromMeta.get(dbId); + List tabletMetaList = invertedIndex.getTabletMetaList(tabletIds); + for (int i = 0; i < tabletMetaList.size(); i++) { + TabletMeta tabletMeta = tabletMetaList.get(i); + if (tabletMeta == TabletInvertedIndex.NOT_EXIST_TABLET_META) { + continue; + } + long tabletId = tabletIds.get(i); + long tableId = tabletMeta.getTableId(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + continue; + } + olapTable.writeLock(); + try { long partitionId = tabletMeta.getPartitionId(); Partition partition = olapTable.getPartition(partitionId); if (partition == null) { @@ -652,11 +653,11 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta LOG.error("invalid situation. tablet[{}] is empty", tabletId); } } - } // end for tabletMetas - LOG.info("delete {} replica(s) from catalog in db[{}]", deleteCounter, dbId); - } finally { - db.writeUnlock(); + } finally { + olapTable.writeUnlock(); + } } + LOG.info("delete {} replica(s) from catalog in db[{}]", deleteCounter, dbId); } // end for dbs if (Config.recover_with_empty_tablet && createReplicaBatchTask.getTaskNum() > 0) { @@ -791,22 +792,21 @@ private static void handleRecoverTablet(ListMultimap tabletRecoveryM if (db == null) { continue; } - db.writeLock(); - try { - List tabletIds = tabletRecoveryMap.get(dbId); - List tabletMetaList = invertedIndex.getTabletMetaList(tabletIds); - for (int i = 0; i < tabletMetaList.size(); i++) { - TabletMeta tabletMeta = tabletMetaList.get(i); - if (tabletMeta == TabletInvertedIndex.NOT_EXIST_TABLET_META) { - continue; - } - long tabletId = tabletIds.get(i); - long tableId = tabletMeta.getTableId(); - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - continue; - } - + List tabletIds = tabletRecoveryMap.get(dbId); + List tabletMetaList = invertedIndex.getTabletMetaList(tabletIds); + for (int i = 0; i < tabletMetaList.size(); i++) { + TabletMeta tabletMeta = tabletMetaList.get(i); + if (tabletMeta == TabletInvertedIndex.NOT_EXIST_TABLET_META) { + continue; + } + long tabletId = tabletIds.get(i); + long tableId = tabletMeta.getTableId(); + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + continue; + } + olapTable.writeLock(); + try { long partitionId = tabletMeta.getPartitionId(); Partition partition = olapTable.getPartition(partitionId); if (partition == null) { @@ -872,9 +872,9 @@ private static void handleRecoverTablet(ListMultimap tabletRecoveryM } } } + } finally { + olapTable.writeUnlock(); } - } finally { - db.writeUnlock(); } } // end for recovery map @@ -917,12 +917,13 @@ private static void handleSetTabletInMemory(long backendId, Map b if (db == null) { continue; } - db.readLock(); + + OlapTable olapTable = (OlapTable) db.getTable(tableId); + if (olapTable == null) { + continue; + } + olapTable.readLock(); try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - continue; - } Partition partition = olapTable.getPartition(partitionId); if (partition == null) { continue; @@ -932,7 +933,7 @@ private static void handleSetTabletInMemory(long backendId, Map b tabletToInMemory.add(new ImmutableTriple<>(tabletId, tabletInfo.getSchemaHash(), feIsInMemory)); } } finally { - db.readUnlock(); + olapTable.readUnlock(); } } } @@ -979,13 +980,10 @@ private static void addReplica(long tabletId, TTabletInfo backendTabletInfo, lon if (db == null) { throw new MetaNotFoundException("db[" + dbId + "] does not exist"); } - db.writeLock(); - try { - OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - throw new MetaNotFoundException("table[" + tableId + "] does not exist"); - } + OlapTable olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP); + olapTable.writeLock(); + try { Partition partition = olapTable.getPartition(partitionId); if (partition == null) { throw new MetaNotFoundException("partition[" + partitionId + "] does not exist"); @@ -1081,7 +1079,7 @@ private static void addReplica(long tabletId, TTabletInfo backendTabletInfo, lon "replica is enough[" + tablet.getReplicas().size() + "-" + replicationNum + "]"); } } finally { - db.writeUnlock(); + olapTable.writeUnlock(); } } From 3a57f8d08fe4abda7bc8b01442fba6b9cee63140 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Thu, 24 Sep 2020 17:09:03 +0800 Subject: [PATCH 23/50] fix unit test failed for AlterTest --- .../src/main/java/org/apache/doris/alter/Alter.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index 713e97602f6346..587947c880fa0d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -233,11 +233,13 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { List alterClauses = Lists.newArrayList(); // some operations will take long time to process, need to be done outside the table lock boolean needProcessOutsideTableLock = false; + // check conflict alter ops first - alterClauses = stmt.getOps(); AlterOperations currentAlterOps = new AlterOperations(); currentAlterOps.checkConflict(alterClauses); // check cluster capacity and db quota, only need to check once. + currentAlterOps.checkConflict(stmt.getOps()); + // check cluster capacity and db quota outside table lock to escape dead lock, only need to check once. if (currentAlterOps.needCheckCapacity()) { Catalog.getCurrentSystemInfo().checkClusterCapacity(clusterName); db.checkQuota(); @@ -261,6 +263,9 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { OlapTable olapTable = (OlapTable) table; stmt.rewriteAlterClause(olapTable); + alterClauses = stmt.getOps(); + currentAlterOps = new AlterOperations(); + currentAlterOps.checkConflict(alterClauses); if (olapTable.getState() != OlapTableState.NORMAL) { throw new DdlException( "Table[" + olapTable.getName() + "]'s state is not NORMAL. Do not allow doing ALTER ops"); @@ -536,7 +541,7 @@ private void processRename(Database db, Table table, List alterClau /** * Batch update partitions' properties - * caller should hold the db lock + * caller should hold the table lock */ public void modifyPartitionsProperty(Database db, OlapTable olapTable, @@ -616,7 +621,7 @@ public ModifyPartitionInfo modifyPartitionProperty(Database db, String partitionName, Map properties) throws DdlException { - Preconditions.checkArgument(db.isWriteLockHeldByCurrentThread()); + Preconditions.checkArgument(olapTable.isWriteLockHeldByCurrentThread()); if (olapTable.getState() != OlapTableState.NORMAL) { throw new DdlException("Table[" + olapTable.getName() + "]'s state is not NORMAL"); } From 9357dc0d0dcb666f2081a7849f8a94e50e5ed8e4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 20 Oct 2020 14:58:59 +0800 Subject: [PATCH 24/50] fix --- fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index 587947c880fa0d..96c4681d5a1332 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -363,10 +363,7 @@ private void processReplaceTable(Database db, OlapTable origTable, List Date: Tue, 20 Oct 2020 21:21:18 +0800 Subject: [PATCH 25/50] Add MetaLockUtils and fix some table lock level --- .../doris/alter/SchemaChangeHandler.java | 18 +-- .../org/apache/doris/catalog/Catalog.java | 39 +++--- .../org/apache/doris/catalog/Database.java | 2 +- .../doris/common/util/MetaLockUtils.java | 125 ++++++++++++++++++ .../transaction/GlobalTransactionMgr.java | 11 +- .../apache/doris/utframe/AnotherDemoTest.java | 6 +- .../org/apache/doris/utframe/DemoTest.java | 22 +-- 7 files changed, 168 insertions(+), 55 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index eddc831a613bc0..4061d64214e14f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -1770,22 +1770,8 @@ public void updateTableInMemoryMeta(Database db, String tableName, Map partitionNames, - Map properties) throws DdlException { - OlapTable olapTable; - db.readLock(); - try { - olapTable = (OlapTable)db.getTable(tableName); - for (String partitionName : partitionNames) { - Partition partition = olapTable.getPartition(partitionName); - if (partition == null) { - throw new DdlException("Partition[" + partitionName + "] does not exist in " + - "table[" + olapTable.getName() + "]"); - } - } - } finally { - db.readUnlock(); - } - + Map properties) throws DdlException, MetaNotFoundException { + OlapTable olapTable = (OlapTable) db.getTableOrThrowException(tableName, Table.TableType.OLAP); boolean isInMemory = Boolean.parseBoolean(properties.get(PropertyAnalyzer.PROPERTIES_INMEMORY)); if (isInMemory == olapTable.isInMemory()) { return; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index aa5c6145f00484..8fe5c6b7a54939 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -118,6 +118,7 @@ import org.apache.doris.common.util.Daemon; import org.apache.doris.common.util.DynamicPartitionUtil; import org.apache.doris.common.util.MasterDaemon; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.common.util.PrintableMap; import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.common.util.QueryableReentrantLock; @@ -246,6 +247,7 @@ import java.net.URL; import java.util.ArrayList; import java.util.Collection; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -3089,19 +3091,20 @@ public void createTable(CreateTableStmt stmt) throws DdlException { public void createTableLike(CreateTableLikeStmt stmt) throws DdlException { try { Database db = Catalog.getCurrentCatalog().getDb(stmt.getExistedDbName()); + Table table = db.getTable(stmt.getExistedTableName()); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, stmt.getExistedTableName()); + } + List createTableStmt = Lists.newArrayList(); - db.readLock(); + table.readLock(); try { - Table table = db.getTable(stmt.getExistedTableName()); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, stmt.getExistedTableName()); - } Catalog.getDdlStmt(stmt.getDbName(), table, createTableStmt, null, null, false, false); if (createTableStmt.isEmpty()) { ErrorReport.reportDdlException(ErrorCode.ERROR_CREATE_TABLE_LIKE_EMPTY, "CREATE"); } } finally { - db.readUnlock(); + table.readUnlock(); } CreateTableStmt parsedCreateTableStmt = (CreateTableStmt) SqlParserUtils.parseAndAnalyzeStmt(createTableStmt.get(0), ConnectContext.get()); parsedCreateTableStmt.setTableName(stmt.getTableName()); @@ -6331,23 +6334,24 @@ public void replayUpdateClusterAndBackends(BackendIdsUpdateInfo info) { public String dumpImage() { LOG.info("begin to dump meta data"); String dumpFilePath; - Map lockedDbMap = Maps.newTreeMap(); + List databases = Lists.newArrayList(); + List> tableLists = Lists.newArrayList(); tryLock(true); try { - // sort all dbs + // sort all dbs to avoid potential dead lock for (long dbId : getDbIds()) { Database db = getDb(dbId); Preconditions.checkNotNull(db); - lockedDbMap.put(dbId, db); + databases.add(db); } + databases.sort(Comparator.comparing(Database::getId)); // lock all dbs - for (Database db : lockedDbMap.values()) { + for (Database db : databases) { db.readLock(); List
tableList = db.getTablesOnIdOrder(); - for (Table table : tableList) { - table.readLock(); - } + MetaLockUtils.readLockTables(tableList); + tableLists.add(tableList); } LOG.info("acquired all the dbs' read lock."); @@ -6366,12 +6370,9 @@ public String dumpImage() { } finally { // unlock all load.readUnlock(); - for (Database db : lockedDbMap.values()) { - List
tableList = db.getTablesOnIdOrder(); - for (int i = tableList.size() - 1; i >= 0; i--) { - tableList.get(i).readUnlock(); - } - db.readUnlock(); + for (int i = databases.size() - 1; i >= 0; i--) { + MetaLockUtils.readUnlockTables(tableLists.get(i)); + databases.get(i).readUnlock(); } unlock(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 5939e6b7fb2466..316fe60b585727 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -367,7 +367,7 @@ public List
getTables() { return new ArrayList<>(idToTable.values()); } - // tables must get read or write table in fixed order to escape potential dead lock + // tables must get read or write table in fixed order to avoid potential dead lock public List
getTablesOnIdOrder() { return idToTable.values().stream() .sorted(Comparator.comparing(Table::getId)) diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java new file mode 100644 index 00000000000000..e0278f05ad9245 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java @@ -0,0 +1,125 @@ +// 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.util; + +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Table; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +public class MetaLockUtils { + + public static void readLockDatabases(List databaseList) { + for (Database database : databaseList) { + database.readLock(); + } + } + + public static boolean tryReadLockDatabases(List databaseList, long timeout, TimeUnit unit) { + for (int i = 0; i < databaseList.size(); i++) { + if (!databaseList.get(i).tryReadLock(timeout, unit)) { + for (int j = i - 1; j >= 0; j--) { + databaseList.get(j).readUnlock(); + } + return false; + } + } + return true; + } + + public static void readUnlockDatabases(List databaseList) { + for (int i = databaseList.size() - 1; i >= 0; i--) { + databaseList.get(i).readUnlock(); + } + } + + public static void writeLockDatabases(List databaseList) { + for (Database database : databaseList) { + database.writeLock(); + } + } + + public static boolean tryWriteLockDatabases(List databaseList, long timeout, TimeUnit unit) { + for (int i = 0; i < databaseList.size(); i++) { + if (!databaseList.get(i).tryWriteLock(timeout, unit)) { + for (int j = i - 1; j >= 0; j--) { + databaseList.get(j).writeUnlock(); + } + return false; + } + } + return true; + } + + public static void writeUnlockDatabases(List databaseList) { + for (int i = databaseList.size() - 1; i >= 0; i--) { + databaseList.get(i).writeUnlock(); + } + } + + + public static void readLockTables(List
tableList) { + for (Table table : tableList) { + table.readLock(); + } + } + + public static boolean tryReadLockTables(List
tableList, long timeout, TimeUnit unit) { + for (int i = 0; i < tableList.size(); i++) { + if (!tableList.get(i).tryReadLock(timeout, unit)) { + for (int j = i - 1; j >= 0; j--) { + tableList.get(j).readUnlock(); + } + return false; + } + } + return true; + } + + public static void readUnlockTables(List
tableList) { + for (int i = tableList.size() - 1; i >= 0; i--) { + tableList.get(i).readUnlock(); + } + } + + public static void writeLockTables(List
tableList) { + for (Table table : tableList) { + table.writeLock(); + } + } + + public static boolean tryWriteLockTables(List
tableList, long timeout, TimeUnit unit) { + for (int i = 0; i < tableList.size(); i++) { + if (!tableList.get(i).tryWriteLock(timeout, unit)) { + for (int j = i - 1; j >= 0; j--) { + tableList.get(j).writeUnlock(); + } + return false; + } + } + return true; + } + + public static void writeUnlockTables(List
tableList) { + for (int i = tableList.size() - 1; i >= 0; i--) { + tableList.get(i).writeUnlock(); + } + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java index d1447b7368060b..ca2313c9a1f17f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java @@ -17,6 +17,7 @@ package org.apache.doris.transaction; +import org.apache.commons.lang3.StringUtils; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Table; @@ -27,6 +28,7 @@ import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.io.Writable; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.persist.EditLog; import org.apache.doris.thrift.TUniqueId; import org.apache.doris.transaction.TransactionState.LoadJobSourceType; @@ -45,6 +47,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; /** * Transaction Manager @@ -186,15 +189,13 @@ public boolean commitAndPublishTransaction(Database db, List
tableList, l List tabletCommitInfos, long timeoutMillis, TxnCommitAttachment txnCommitAttachment) throws UserException { - for (Table table : tableList) { - table.writeLock(); + if (!MetaLockUtils.tryWriteLockTables(tableList, timeoutMillis, TimeUnit.MILLISECONDS)) { + throw new UserException("get tableList write lock timeout, tableList=(" + StringUtils.join(tableList, ",") + ")"); } try { commitTransaction(db.getId(), tableList, transactionId, tabletCommitInfos, txnCommitAttachment); } finally { - for (int i = tableList.size() - 1; i >= 0; i--) { - tableList.get(i).writeUnlock(); - } + MetaLockUtils.writeUnlockTables(tableList); } DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(db.getId()); return dbTransactionMgr.publishTransaction(db, transactionId, timeoutMillis); diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/AnotherDemoTest.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/AnotherDemoTest.java index 0a91f40fecbdc3..5ea6919fea8ddb 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/utframe/AnotherDemoTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/AnotherDemoTest.java @@ -161,15 +161,15 @@ public void testCreateDbAndTable() throws Exception { // 4. get and test the created db and table Database db = Catalog.getCurrentCatalog().getDb("default_cluster:db1"); Assert.assertNotNull(db); - db.readLock(); + OlapTable tbl = (OlapTable) db.getTable("tbl1"); + tbl.readLock(); try { - OlapTable tbl = (OlapTable) db.getTable("tbl1"); Assert.assertNotNull(tbl); System.out.println(tbl.getName()); Assert.assertEquals("Doris", tbl.getEngine()); Assert.assertEquals(1, tbl.getBaseSchema().size()); } finally { - db.readUnlock(); + tbl.readUnlock(); } // 5. query // TODO: we can not process real query for now. So it has to be a explain query diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoTest.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoTest.java index 499f9594b3044b..906d6c1f8222cb 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoTest.java @@ -90,15 +90,15 @@ public void testCreateDbAndTable() throws Exception { // 4. get and test the created db and table Database db = Catalog.getCurrentCatalog().getDb("default_cluster:db1"); Assert.assertNotNull(db); - db.readLock(); + OlapTable tbl = (OlapTable) db.getTable("tbl1"); + tbl.readLock(); try { - OlapTable tbl = (OlapTable) db.getTable("tbl1"); Assert.assertNotNull(tbl); System.out.println(tbl.getName()); Assert.assertEquals("Doris", tbl.getEngine()); Assert.assertEquals(1, tbl.getBaseSchema().size()); } finally { - db.readUnlock(); + tbl.readUnlock(); } // 5. process a schema change job String alterStmtStr = "alter table db1.tbl1 add column k2 int default '1'"; @@ -115,17 +115,17 @@ public void testCreateDbAndTable() throws Exception { System.out.println("alter job " + alterJobV2.getJobId() + " is done. state: " + alterJobV2.getJobState()); Assert.assertEquals(AlterJobV2.JobState.FINISHED, alterJobV2.getJobState()); } - db.readLock(); - try { - OlapTable tbl = (OlapTable) db.getTable("tbl1"); - Assert.assertEquals(2, tbl.getBaseSchema().size()); - String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId()); - Assert.assertEquals(baseIndexName, tbl.getName()); - MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId()); + OlapTable tbl1 = (OlapTable) db.getTable("tbl1"); + tbl1.readLock(); + try { + Assert.assertEquals(2, tbl1.getBaseSchema().size()); + String baseIndexName = tbl1.getIndexNameById(tbl.getBaseIndexId()); + Assert.assertEquals(baseIndexName, tbl1.getName()); + MaterializedIndexMeta indexMeta = tbl1.getIndexMetaByIndexId(tbl1.getBaseIndexId()); Assert.assertNotNull(indexMeta); } finally { - db.readUnlock(); + tbl1.readUnlock(); } // 7. query // TODO: we can not process real query for now. So it has to be a explain query From 40f5cf1ce1e0bd078a23d0d761772b14e1b32b0d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Wed, 21 Oct 2020 14:17:44 +0800 Subject: [PATCH 26/50] Fix unittest for AlterTest failed --- fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index 96c4681d5a1332..e27e1e4e029d64 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -359,15 +359,13 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { // entry of processing replace table private void processReplaceTable(Database db, OlapTable origTable, List alterClauses) throws UserException { ReplaceTableClause clause = (ReplaceTableClause) alterClauses.get(0); - Preconditions.checkState(db.isWriteLockHeldByCurrentThread()); + Preconditions.checkState(origTable.isWriteLockHeldByCurrentThread()); String oldTblName = origTable.getName(); String newTblName = clause.getTblName(); Table newTbl = db.getTableOrThrowException(newTblName, TableType.OLAP); OlapTable olapNewTbl = (OlapTable) newTbl; - boolean swapTable = clause.isSwapTable(); - // First, we need to check whether the table to be operated on can be renamed olapNewTbl.checkAndSetName(oldTblName, true); if (swapTable) { From d3c287cd6d10023fd1f2117eb4fccb591e8d09f2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Wed, 21 Oct 2020 20:19:56 +0800 Subject: [PATCH 27/50] Change db level lock to table level lock for query --- .../org/apache/doris/analysis/InsertStmt.java | 16 +++--- .../org/apache/doris/analysis/QueryStmt.java | 10 ++-- .../org/apache/doris/analysis/SelectStmt.java | 22 ++++--- .../doris/analysis/SetOperationStmt.java | 10 ++-- .../apache/doris/analysis/ShowDataStmt.java | 5 +- .../org/apache/doris/analysis/WithClause.java | 7 +-- .../org/apache/doris/backup/RestoreJob.java | 39 ++++++------- .../org/apache/doris/catalog/Catalog.java | 19 ++----- .../org/apache/doris/clone/TabletChecker.java | 57 +++++++------------ .../org/apache/doris/qe/StmtExecutor.java | 37 +++--------- .../doris/service/FrontendServiceImpl.java | 1 - .../transaction/PublishVersionDaemon.java | 26 +++++---- .../org/apache/doris/qe/StmtExecutorTest.java | 2 +- 13 files changed, 106 insertions(+), 145 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertStmt.java index c2845b828d7aa4..bd6f0acfcaa129 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertStmt.java @@ -179,29 +179,31 @@ public String getDb() { return tblName.getDb(); } - // TODO(zc): used to get all dbs for lock - public void getDbs(Analyzer analyzer, Map dbs, Set parentViewNameSet) throws AnalysisException { + public void getTables(Analyzer analyzer, Map tableMap, Set parentViewNameSet) throws AnalysisException { // get dbs of statement - queryStmt.getDbs(analyzer, dbs, parentViewNameSet); - // get db of target table + queryStmt.getTables(analyzer, tableMap, parentViewNameSet); tblName.analyze(analyzer); String dbName = tblName.getDb(); - + String tableName = tblName.getTbl(); // check exist Database db = analyzer.getCatalog().getDb(dbName); if (db == null) { ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName); } + Table table = db.getTable(tblName.getTbl()); + if (table == null) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } // check access - if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), tblName.getDb(), tblName.getTbl(), + if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), dbName, tableName, PrivPredicate.LOAD)) { ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD", ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), tblName.getTbl()); } - dbs.put(dbName, db); + tableMap.put(table.getId(), table); } public QueryStmt getQueryStmt() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/QueryStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/QueryStmt.java index 73e53583f9eec8..c457892ae274af 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/QueryStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/QueryStmt.java @@ -17,7 +17,7 @@ package org.apache.doris.analysis; -import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Table; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; @@ -427,20 +427,20 @@ private Expr trySubstituteOrdinal(Expr expr, String errorPrefix, return resultExprs.get((int) pos - 1).clone(); } - public void getWithClauseDbs(Analyzer analyzer, Map dbs, Set parentViewNameSet) throws AnalysisException { + public void getWithClauseTables(Analyzer analyzer, Map tableMap, Set parentViewNameSet) throws AnalysisException { if (withClause_ != null) { - withClause_.getDbs(analyzer, dbs, parentViewNameSet); + withClause_.getTables(analyzer, tableMap, parentViewNameSet); } } - // get database used by this query. + // get tables used by this query. // Set parentViewNameSet contain parent stmt view name // to make sure query like "with tmp as (select * from db1.table1) " + // "select a.siteid, b.citycode, a.siteid from (select siteid, citycode from tmp) a " + // "left join (select siteid, citycode from tmp) b on a.siteid = b.siteid;"; // tmp in child stmt "(select siteid, citycode from tmp)" do not contain with_Clause // so need to check is view name by parentViewNameSet. issue link: https://github.com/apache/incubator-doris/issues/4598 - public abstract void getDbs(Analyzer analyzer, Map dbs, Set parentViewNameSet) throws AnalysisException; + public abstract void getTables(Analyzer analyzer, Map tables, Set parentViewNameSet) throws AnalysisException; /** * UnionStmt and SelectStmt have different implementations. diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java index 5560f7b77f00c4..0f7c60c4a945ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.FunctionSet; import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Table.TableType; import org.apache.doris.catalog.Type; import org.apache.doris.catalog.View; @@ -60,6 +61,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.SortedMap; import java.util.UUID; /** @@ -276,15 +278,16 @@ public ExprSubstitutionMap getBaseTblSmap() { } @Override - public void getDbs(Analyzer analyzer, Map dbs, Set parentViewNameSet) throws AnalysisException { - getWithClauseDbs(analyzer, dbs, parentViewNameSet); + public void getTables(Analyzer analyzer, Map tableMap, Set parentViewNameSet) throws AnalysisException { + getWithClauseTables(analyzer, tableMap, parentViewNameSet); for (TableRef tblRef : fromClause_) { if (tblRef instanceof InlineViewRef) { // Inline view reference QueryStmt inlineStmt = ((InlineViewRef) tblRef).getViewStmt(); - inlineStmt.getDbs(analyzer, dbs, parentViewNameSet); + inlineStmt.getTables(analyzer, tableMap, parentViewNameSet); } else { String dbName = tblRef.getName().getDb(); + String tableName = tblRef.getName().getTbl(); if (Strings.isNullOrEmpty(dbName)) { dbName = analyzer.getDefaultDb(); } else { @@ -296,23 +299,28 @@ public void getDbs(Analyzer analyzer, Map dbs, Set par if (Strings.isNullOrEmpty(dbName)) { ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); } - + if (Strings.isNullOrEmpty(tableName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR); + } Database db = analyzer.getCatalog().getDb(dbName); if (db == null) { ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName); } + Table table = db.getTable(tableName); + if (table == null) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } // check auth if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), dbName, - tblRef.getName().getTbl(), + tableName, PrivPredicate.SELECT)) { ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "SELECT", ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), tblRef.getName().getTbl()); } - - dbs.put(dbName, db); + tableMap.put(table.getId(), table); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetOperationStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetOperationStmt.java index 583d647c8640da..a1f2f4177621c4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetOperationStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetOperationStmt.java @@ -17,7 +17,7 @@ package org.apache.doris.analysis; -import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Table; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; import org.apache.doris.rewrite.ExprRewriter; @@ -172,10 +172,10 @@ public void removeAllOperands() { public List getSetOpsResultExprs() { return setOpsResultExprs_; } @Override - public void getDbs(Analyzer analyzer, Map dbs, Set parentViewNameSet) throws AnalysisException { - getWithClauseDbs(analyzer, dbs, parentViewNameSet); + public void getTables(Analyzer analyzer, Map tableMap, Set parentViewNameSet) throws AnalysisException { + getWithClauseTables(analyzer, tableMap, parentViewNameSet); for (SetOperand op : operands) { - op.getQueryStmt().getDbs(analyzer, dbs, parentViewNameSet); + op.getQueryStmt().getTables(analyzer, tableMap, parentViewNameSet); } } @@ -184,7 +184,7 @@ public void getDbs(Analyzer analyzer, Map dbs, Set par * set operands are set compatible, adding implicit casts if necessary. */ @Override - public void analyze(Analyzer analyzer) throws AnalysisException, UserException { + public void analyze(Analyzer analyzer) throws UserException { if (isAnalyzed()) return; super.analyze(analyzer); Preconditions.checkState(operands.size() > 0); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java index 1fd73845283462..e8eed90f67b4a7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java @@ -17,6 +17,7 @@ package org.apache.doris.analysis; +import com.google.common.collect.Lists; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Database; @@ -41,7 +42,6 @@ import java.util.Arrays; import java.util.Comparator; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.SortedSet; @@ -73,8 +73,7 @@ public class ShowDataStmt extends ShowStmt { public ShowDataStmt(String dbName, String tableName) { this.dbName = dbName; this.tableName = tableName; - - this.totalRows = new LinkedList>(); + this.totalRows = Lists.newArrayList(); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/WithClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/WithClause.java index 416490ce47513d..064021d27e9f45 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/WithClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/WithClause.java @@ -22,7 +22,7 @@ import java.util.Map; import java.util.Set; -import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Table; import org.apache.doris.catalog.View; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; @@ -106,11 +106,10 @@ public void reset() { for (View view: views_) view.getQueryStmt().reset(); } - public void getDbs(Analyzer analyzer, Map dbs, Set parentViewNameSet) throws AnalysisException { + public void getTables(Analyzer analyzer, Map tableMap, Set parentViewNameSet) throws AnalysisException { for (View view : views_) { QueryStmt stmt = view.getQueryStmt(); - parentViewNameSet.add(view.getName()); - stmt.getDbs(analyzer, dbs, parentViewNameSet); + stmt.getTables(analyzer, tableMap, parentViewNameSet); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java index fcc10c66352f22..8a562d24aa2cb0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -355,33 +355,28 @@ private void checkIfNeedCancel() { return; } - db.readLock(); - try { - for (IdChain idChain : fileMapping.getMapping().keySet()) { - OlapTable tbl = (OlapTable) db.getTable(idChain.getTblId()); - if (tbl == null) { - status = new Status(ErrCode.NOT_FOUND, "table " + idChain.getTblId() + " has been dropped"); + for (IdChain idChain : fileMapping.getMapping().keySet()) { + OlapTable tbl = (OlapTable) db.getTable(idChain.getTblId()); + if (tbl == null) { + status = new Status(ErrCode.NOT_FOUND, "table " + idChain.getTblId() + " has been dropped"); + return; + } + tbl.readLock(); + try { + Partition part = tbl.getPartition(idChain.getPartId()); + if (part == null) { + status = new Status(ErrCode.NOT_FOUND, "partition " + idChain.getPartId() + " has been dropped"); return; } - tbl.readLock(); - try { - Partition part = tbl.getPartition(idChain.getPartId()); - if (part == null) { - status = new Status(ErrCode.NOT_FOUND, "partition " + idChain.getPartId() + " has been dropped"); - return; - } - MaterializedIndex index = part.getIndex(idChain.getIdxId()); - if (index == null) { - status = new Status(ErrCode.NOT_FOUND, "index " + idChain.getIdxId() + " has been dropped"); - return; - } - } finally { - tbl.readUnlock(); + MaterializedIndex index = part.getIndex(idChain.getIdxId()); + if (index == null) { + status = new Status(ErrCode.NOT_FOUND, "index " + idChain.getIdxId() + " has been dropped"); + return; } + } finally { + tbl.readUnlock(); } - } finally { - db.readUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index 8fe5c6b7a54939..c8a49055733848 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -2046,15 +2046,11 @@ public long saveDb(DataOutputStream dos, long checksum) throws IOException { checksum ^= entry.getKey(); db.readLock(); List
tableList = db.getTablesOnIdOrder(); - for (Table table : tableList) { - table.readLock(); - } + MetaLockUtils.readLockTables(tableList); try { db.write(dos); } finally { - for (int i = tableList.size() - 1; i >= 0; i--) { - tableList.get(i).readUnlock(); - } + MetaLockUtils.readUnlockTables(tableList); db.readUnlock(); } } @@ -4718,14 +4714,7 @@ public HashMap getPartitionIdToStorageMediumMap() { LOG.warn("db {} does not exist while doing backend report", dbId); continue; } - List
tableList = null; - db.readLock(); - try { - tableList = db.getTables(); - } finally { - db.readUnlock(); - } - + List
tableList = db.getTables(); for (Table table : tableList) { if (table.getType() != TableType.OLAP) { continue; @@ -6369,11 +6358,11 @@ public String dumpImage() { } } finally { // unlock all - load.readUnlock(); for (int i = databases.size() - 1; i >= 0; i--) { MetaLockUtils.readUnlockTables(tableLists.get(i)); databases.get(i).readUnlock(); } + load.readUnlock(); unlock(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java index 922f308af3336f..ede26bef34e3be 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java @@ -204,16 +204,8 @@ private void checkTablets() { continue; } - int availableBackendsNum = infoService.getClusterBackendIds(db.getClusterName(), true).size(); - db.readLock(); - List
tableList = null; - List aliveBeIdsInCluster = null; - try { - aliveBeIdsInCluster = infoService.getClusterBackendIds(db.getClusterName(), true); - tableList = db.getTables(); - } finally { - db.readUnlock(); - } + List
tableList = db.getTables(); + List aliveBeIdsInCluster = infoService.getClusterBackendIds(db.getClusterName(), true); for (Table table : tableList) { table.readLock(); @@ -342,35 +334,30 @@ private void removePriosIfNecessary() { continue; } - db.readLock(); - try { - Iterator>> jter = dbEntry.getValue().entrySet().iterator(); - while (jter.hasNext()) { - Map.Entry> tblEntry = jter.next(); - long tblId = tblEntry.getKey(); - OlapTable tbl = (OlapTable) db.getTable(tblId); - if (tbl == null) { + Iterator>> jter = dbEntry.getValue().entrySet().iterator(); + while (jter.hasNext()) { + Map.Entry> tblEntry = jter.next(); + long tblId = tblEntry.getKey(); + OlapTable tbl = (OlapTable) db.getTable(tblId); + if (tbl == null) { + deletedPrios.add(Pair.create(dbId, tblId)); + continue; + } + tbl.readLock(); + try { + Set parts = tblEntry.getValue(); + parts = parts.stream().filter(p -> (tbl.getPartition(p.partId) != null && !p.isTimeout())).collect( + Collectors.toSet()); + if (parts.isEmpty()) { deletedPrios.add(Pair.create(dbId, tblId)); - continue; - } - tbl.readLock(); - try { - Set parts = tblEntry.getValue(); - parts = parts.stream().filter(p -> (tbl.getPartition(p.partId) != null && !p.isTimeout())).collect( - Collectors.toSet()); - if (parts.isEmpty()) { - deletedPrios.add(Pair.create(dbId, tblId)); - } - } finally { - tbl.readUnlock(); } + } finally { + tbl.readUnlock(); } + } - if (dbEntry.getValue().isEmpty()) { - iter.remove(); - } - } finally { - db.readUnlock(); + if (dbEntry.getValue().isEmpty()) { + iter.remove(); } } for (Pair prio : deletedPrios) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index bbf89b4b1b4241..61b50505692017 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -17,6 +17,7 @@ package org.apache.doris.qe; +import com.google.common.collect.Maps; import org.apache.doris.analysis.Analyzer; import org.apache.doris.analysis.CreateTableAsSelectStmt; import org.apache.doris.analysis.DdlStmt; @@ -40,8 +41,8 @@ import org.apache.doris.analysis.UseStmt; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.Database; import org.apache.doris.catalog.ScalarType; +import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Table.TableType; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; @@ -54,6 +55,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.Version; import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.common.util.ProfileManager; import org.apache.doris.common.util.QueryPlannerProfile; import org.apache.doris.common.util.RuntimeProfile; @@ -87,7 +89,6 @@ import com.google.common.base.Strings; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -399,26 +400,6 @@ private void writeProfile() { ProfileManager.getInstance().pushProfile(profile); } - // Lock all database before analyze - private void lock(Map dbs) { - if (dbs == null) { - return; - } - for (Database db : dbs.values()) { - db.readLock(); - } - } - - // unLock all database after analyze - private void unLock(Map dbs) { - if (dbs == null) { - return; - } - for (Database db : dbs.values()) { - db.readUnlock(); - } - } - // Analyze one statement to structure in memory. public void analyze(TQueryOptions tQueryOptions) throws UserException { LOG.info("begin to analyze stmt: {}, forwarded stmt id: {}", context.getStmtId(), context.getForwardedStmtId()); @@ -472,12 +453,12 @@ public void analyze(TQueryOptions tQueryOptions) throws UserException { if (parsedStmt instanceof QueryStmt || parsedStmt instanceof InsertStmt || parsedStmt instanceof CreateTableAsSelectStmt) { - Map dbs = Maps.newTreeMap(); + Map tableMap = Maps.newTreeMap(); QueryStmt queryStmt; Set parentViewNameSet = Sets.newHashSet(); if (parsedStmt instanceof QueryStmt) { queryStmt = (QueryStmt) parsedStmt; - queryStmt.getDbs(analyzer, dbs, parentViewNameSet); + queryStmt.getTables(analyzer, tableMap, parentViewNameSet); } else { InsertStmt insertStmt; if (parsedStmt instanceof InsertStmt) { @@ -485,10 +466,10 @@ public void analyze(TQueryOptions tQueryOptions) throws UserException { } else { insertStmt = ((CreateTableAsSelectStmt) parsedStmt).getInsertStmt(); } - insertStmt.getDbs(analyzer, dbs, parentViewNameSet); + insertStmt.getTables(analyzer, tableMap, parentViewNameSet); } - - lock(dbs); + List
tables = Lists.newArrayList(tableMap.values()); + MetaLockUtils.readLockTables(tables); try { analyzeAndGenerateQueryPlan(tQueryOptions); } catch (MVSelectFailedException e) { @@ -505,7 +486,7 @@ public void analyze(TQueryOptions tQueryOptions) throws UserException { LOG.warn("Analyze failed because ", e); throw new AnalysisException("Unexpected exception: " + e.getMessage()); } finally { - unLock(dbs); + MetaLockUtils.readUnlockTables(tables); } } else { try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index 88377b9d00603c..cf9bd9de8df58d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -268,7 +268,6 @@ public TListTableStatusResult listTableStatus(TGetTablesParams params) throws TE table.getName(), PrivPredicate.SHOW)) { continue; } - table.readLock(); try { if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(currentUser, params.db, diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java index a1116172734b89..291ab881a19218 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java @@ -190,16 +190,18 @@ private void publishVersion() throws UserException { LOG.warn("Database [{}] has been dropped.", transactionState.getDbId()); continue; } - db.readLock(); - try { - for (int i = 0; i < transactionState.getTableIdList().size(); i++) { - long tableId = transactionState.getTableIdList().get(i); - Table table = db.getTable(tableId); - if (table == null || table.getType() != Table.TableType.OLAP) { - LOG.warn("Table [{}] in database [{}] has been dropped.", tableId, db.getFullName()); - continue; - } - OlapTable olapTable = (OlapTable) table; + + + for (int i = 0; i < transactionState.getTableIdList().size(); i++) { + long tableId = transactionState.getTableIdList().get(i); + Table table = db.getTable(tableId); + if (table == null || table.getType() != Table.TableType.OLAP) { + LOG.warn("Table [{}] in database [{}] has been dropped.", tableId, db.getFullName()); + continue; + } + OlapTable olapTable = (OlapTable) table; + olapTable.readLock(); + try { for (Long errorPartitionId : errorPartitionIds) { Partition partition = olapTable.getPartition(errorPartitionId); if (partition != null) { @@ -214,9 +216,9 @@ private void publishVersion() throws UserException { } } } + } finally { + olapTable.readUnlock(); } - } finally { - db.readUnlock(); } } shouldFinishTxn = true; diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/StmtExecutorTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/StmtExecutorTest.java index 4c09fdb5d7b30f..5d7ddb5a0b707e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/StmtExecutorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/StmtExecutorTest.java @@ -197,7 +197,7 @@ public void testSelect(@Mocked QueryStmt queryStmt, minTimes = 0; result = false; - queryStmt.getDbs((Analyzer) any, (SortedMap) any, Sets.newHashSet()); + queryStmt.getTables((Analyzer) any, (SortedMap) any, Sets.newHashSet()); minTimes = 0; queryStmt.getRedirectStatus(); From 1860b70ce5b2a90777bc9fef4dd1159230c322a1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Thu, 22 Oct 2020 21:01:43 +0800 Subject: [PATCH 28/50] fix unittest failed --- .../src/main/java/org/apache/doris/analysis/SelectStmt.java | 1 - 1 file changed, 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java index 0f7c60c4a945ed..af30850bb1a02f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java @@ -61,7 +61,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.SortedMap; import java.util.UUID; /** From 46b8d0459dd33f4c654f86a876194db0fb34edfe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Wed, 4 Nov 2020 17:21:33 +0800 Subject: [PATCH 29/50] fix conflict for table lock and db lock --- .../apache/doris/alter/SchemaChangeJob.java | 94 ++++++++++++++++++- .../org/apache/doris/catalog/Catalog.java | 11 ++- .../org/apache/doris/catalog/Database.java | 18 ---- .../doris/load/loadv2/BrokerLoadJob.java | 35 +++---- .../doris/load/loadv2/SparkLoadJob.java | 45 +++------ 5 files changed, 131 insertions(+), 72 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java index d106e812466428..e591f878f3b27a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java @@ -33,9 +33,11 @@ import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Tablet; import org.apache.doris.common.Config; +import org.apache.doris.common.FeConstants; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.io.Text; +import org.apache.doris.common.util.TimeUtils; import org.apache.doris.persist.ReplicaPersistInfo; import org.apache.doris.persist.ReplicaPersistInfo.ReplicaOperationType; import org.apache.doris.task.AgentBatchTask; @@ -66,6 +68,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.LinkedList; @@ -1104,7 +1107,96 @@ public void replayCancel(Database db) { @Override public void getJobInfo(List> jobInfos, OlapTable tbl) { - //TODO + if (changedIndexIdToSchemaVersion == null) { + // for compatibility + if (state == JobState.FINISHED || state == JobState.CANCELLED) { + List jobInfo = new ArrayList(); + jobInfo.add(tableId); // job id + jobInfo.add(tbl.getName()); // table name + jobInfo.add(TimeUtils.longToTimeString(createTime)); + jobInfo.add(TimeUtils.longToTimeString(finishedTime)); + jobInfo.add(FeConstants.null_string); // index name + jobInfo.add(FeConstants.null_string); // index id + jobInfo.add(FeConstants.null_string); // origin id + jobInfo.add(FeConstants.null_string); // schema version + jobInfo.add(-1); // transaction id + jobInfo.add(state.name()); // job state + jobInfo.add(cancelMsg); + jobInfo.add(FeConstants.null_string); // progress + jobInfo.add(Config.alter_table_timeout_second); // timeout + jobInfos.add(jobInfo); + return; + } + + // in previous version, changedIndexIdToSchema is set to null + // when job is finished or cancelled. + // so if changedIndexIdToSchema == null, the job'state must be FINISHED or CANCELLED + return; + } + + Map indexProgress = new HashMap(); + Map indexState = new HashMap(); + + // calc progress and state for each table + for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { + if (tbl.getIndexNameById(indexId) == null) { + // this index may be dropped, and this should be a FINISHED job, just use a dummy info to show + indexState.put(indexId, IndexState.NORMAL.name()); + indexProgress.put(indexId, "100%"); + } else { + int totalReplicaNum = 0; + int finishedReplicaNum = 0; + String idxState = IndexState.NORMAL.name(); + for (Partition partition : tbl.getPartitions()) { + MaterializedIndex index = partition.getIndex(indexId); + if (state == JobState.RUNNING) { + int tableReplicaNum = getTotalReplicaNumByIndexId(indexId); + int tableFinishedReplicaNum = getFinishedReplicaNumByIndexId(indexId); + Preconditions.checkState(!(tableReplicaNum == 0 && tableFinishedReplicaNum == -1)); + Preconditions.checkState(tableFinishedReplicaNum <= tableReplicaNum, + tableFinishedReplicaNum + "/" + tableReplicaNum); + totalReplicaNum += tableReplicaNum; + finishedReplicaNum += tableFinishedReplicaNum; + } + + if (index.getState() != IndexState.NORMAL) { + idxState = index.getState().name(); + } + } + + indexState.put(indexId, idxState); + + if (Catalog.getCurrentCatalog().isMaster() && state == JobState.RUNNING && totalReplicaNum != 0) { + indexProgress.put(indexId, (finishedReplicaNum * 100 / totalReplicaNum) + "%"); + } else { + indexProgress.put(indexId, "0%"); + } + } + } + + for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { + List jobInfo = new ArrayList(); + + jobInfo.add(tableId); + jobInfo.add(tbl.getName()); + jobInfo.add(TimeUtils.longToTimeString(createTime)); + jobInfo.add(TimeUtils.longToTimeString(finishedTime)); + jobInfo.add(tbl.getIndexNameById(indexId) == null ? FeConstants.null_string : tbl.getIndexNameById(indexId)); // index name + jobInfo.add(indexId); + jobInfo.add(indexId); // origin index id + // index schema version and schema hash + jobInfo.add(changedIndexIdToSchemaVersion.get(indexId) + ":" + changedIndexIdToSchemaHash.get(indexId)); + jobInfo.add(transactionId); + jobInfo.add(state.name()); // job state + jobInfo.add(cancelMsg); + if (state == JobState.RUNNING) { + jobInfo.add(indexProgress.get(indexId) == null ? FeConstants.null_string : indexProgress.get(indexId)); // progress + } else { + jobInfo.add(FeConstants.null_string); + } + jobInfo.add(Config.alter_table_timeout_second); + jobInfos.add(jobInfo); + } // end for indexIds } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index c8a49055733848..7a1c95c13f40d4 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -4308,7 +4308,16 @@ public void replayCreateTable(String dbName, Table table) { public void replayAlterExteranlTableSchema(String dbName, String tableName, List newSchema) throws DdlException { Database db = this.fullNameToDb.get(dbName); - db.allterExternalTableSchemaWithLock(tableName, newSchema); + Table table = db.getTable(tableName); + if (table == null) { + throw new DdlException("Do not contain proper table " + tableName + " in refresh table"); + } + table.writeLock(); + try { + table.setNewFullSchema(newSchema); + } finally { + table.writeUnlock(); + } } private void createTablets(String clusterName, MaterializedIndex index, ReplicaState replicaState, diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 316fe60b585727..efdfd73ee0b16f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -154,10 +154,6 @@ public boolean tryWriteLock(long timeout, TimeUnit unit) { } } - public boolean isWriteLockHeldByCurrentThread() { - return this.rwLock.writeLock().isHeldByCurrentThread(); - } - public long getId() { return id; } @@ -316,20 +312,6 @@ public boolean createTableWithLock(Table table, boolean isReplay, boolean setIfN } } - public void allterExternalTableSchemaWithLock(String tableName, List newSchema) throws DdlException{ - writeLock(); - try { - if (!nameToTable.containsKey(tableName)) { - throw new DdlException("Do not contain proper table " + tableName + " in refresh table"); - } else { - Table table = nameToTable.get(tableName); - table.setNewFullSchema(newSchema); - } - } finally { - writeUnlock(); - } - } - public boolean createTable(Table table) { boolean result = true; String tableName = table.getName(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java index c80a8f6dfb814e..55d15c5a098961 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java @@ -22,6 +22,7 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Table; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.DataQualityException; @@ -35,6 +36,7 @@ import org.apache.doris.common.util.ProfileManager; import org.apache.doris.common.util.RuntimeProfile; import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.load.BrokerFileGroup; import org.apache.doris.load.BrokerFileGroupAggInfo.FileGroupAggKey; import org.apache.doris.load.EtlJobType; @@ -184,26 +186,17 @@ private void onPendingTaskFinished(BrokerPendingTaskAttachment attachment) { } private void createLoadingTask(Database db, BrokerPendingTaskAttachment attachment) throws UserException { + List
tableList = db.getTablesOnIdOrderOrThrowException(Lists.newArrayList(fileGroupAggInfo.getAllTableIds())); // divide job into broker loading task by table List newLoadingTasks = Lists.newArrayList(); this.jobProfile = new RuntimeProfile("BrokerLoadJob " + id + ". " + label); - db.readLock(); + MetaLockUtils.readLockTables(tableList); try { for (Map.Entry> entry : fileGroupAggInfo.getAggKeyToFileGroups().entrySet()) { FileGroupAggKey aggKey = entry.getKey(); List brokerFileGroups = entry.getValue(); long tableId = aggKey.getTableId(); OlapTable table = (OlapTable) db.getTable(tableId); - if (table == null) { - LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) - .add("database_id", dbId) - .add("table_id", tableId) - .add("error_msg", "Failed to divide job into loading task when table not found") - .build()); - throw new MetaNotFoundException("Failed to divide job into loading task when table " - + tableId + " not found"); - } - // Generate loading task and init the plan of task LoadLoadingTask task = new LoadLoadingTask(db, table, brokerDesc, brokerFileGroups, getDeadlineMs(), execMemLimit, @@ -226,11 +219,9 @@ brokerFileGroups, getDeadlineMs(), execMemLimit, } txnState.addTableIndexes(table); } - } finally { - db.readUnlock(); + MetaLockUtils.readUnlockTables(tableList); } - // Submit task outside the database lock, cause it may take a while if task queue is full. for (LoadTask loadTask : newLoadingTasks) { Catalog.getCurrentCatalog().getLoadingLoadTaskScheduler().submit(loadTask); @@ -282,8 +273,10 @@ private void onLoadingTaskFinished(BrokerLoadingTaskAttachment attachment) { return; } Database db = null; + List
tableList = null; try { db = getDb(); + tableList = db.getTablesOnIdOrderOrThrowException(Lists.newArrayList(fileGroupAggInfo.getAllTableIds())); } catch (MetaNotFoundException e) { LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) .add("database_id", dbId) @@ -292,27 +285,25 @@ private void onLoadingTaskFinished(BrokerLoadingTaskAttachment attachment) { cancelJobWithoutCheck(new FailMsg(FailMsg.CancelType.LOAD_RUN_FAIL, e.getMessage()), true, true); return; } - db.writeLock(); + MetaLockUtils.writeLockTables(tableList); try { LOG.info(new LogBuilder(LogKey.LOAD_JOB, id) .add("txn_id", transactionId) .add("msg", "Load job try to commit txn") .build()); MetricRepo.COUNTER_LOAD_FINISHED.increase(1L); -// Catalog.getCurrentGlobalTransactionMgr().commitTransaction( -// dbId, transactionId, commitInfos, -// new LoadJobFinalOperation(id, loadingStatus, progress, loadStartTimestamp, -// finishTimestamp, state, failMsg)); - throw new UserException(""); + Catalog.getCurrentGlobalTransactionMgr().commitTransaction( + dbId, tableList, transactionId, commitInfos, + new LoadJobFinalOperation(id, loadingStatus, progress, loadStartTimestamp, + finishTimestamp, state, failMsg)); } catch (UserException e) { LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) .add("database_id", dbId) .add("error_msg", "Failed to commit txn with error:" + e.getMessage()) .build(), e); cancelJobWithoutCheck(new FailMsg(FailMsg.CancelType.LOAD_RUN_FAIL, e.getMessage()), true, true); - return; } finally { - db.writeUnlock(); + MetaLockUtils.writeUnlockTables(tableList); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java index b2c904440e9851..d8907f55b9e5ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java @@ -56,6 +56,7 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.util.LogBuilder; import org.apache.doris.common.util.LogKey; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.load.EtlJobType; import org.apache.doris.load.EtlStatus; import org.apache.doris.load.FailMsg; @@ -414,7 +415,8 @@ private Set submitPushTasks() throws UserException { AgentBatchTask batchTask = new AgentBatchTask(); boolean hasLoadPartitions = false; Set totalTablets = Sets.newHashSet(); - db.readLock(); + List
tableList = db.getTablesOnIdOrderOrThrowException(Lists.newArrayList(tableToLoadPartitions.keySet())); + MetaLockUtils.readLockTables(tableList); try { writeLock(); try { @@ -426,24 +428,18 @@ private Set submitPushTasks() throws UserException { return totalTablets; } - for (Map.Entry> entry : tableToLoadPartitions.entrySet()) { - long tableId = entry.getKey(); - OlapTable table = (OlapTable) db.getTable(tableId); - if (table == null) { - LOG.warn("table does not exist. id: {}", tableId); - continue; - } - - Set partitionIds = entry.getValue(); + for (Table table : tableList) { + Set partitionIds = tableToLoadPartitions.get(table.getId()); + OlapTable olapTable = (OlapTable) table; for (long partitionId : partitionIds) { - Partition partition = table.getPartition(partitionId); + Partition partition = olapTable.getPartition(partitionId); if (partition == null) { LOG.warn("partition does not exist. id: {}", partitionId); continue; } hasLoadPartitions = true; - int quorumReplicaNum = table.getPartitionInfo().getReplicationNum(partitionId) / 2 + 1; + int quorumReplicaNum = olapTable.getPartitionInfo().getReplicationNum(partitionId) / 2 + 1; List indexes = partition.getMaterializedIndices(IndexExtState.ALL); for (MaterializedIndex index : indexes) { @@ -454,7 +450,7 @@ private Set submitPushTasks() throws UserException { for (Tablet tablet : index.getTablets()) { long tabletId = tablet.getId(); totalTablets.add(tabletId); - String tabletMetaStr = String.format("%d.%d.%d.%d.%d", tableId, partitionId, + String tabletMetaStr = String.format("%d.%d.%d.%d.%d", olapTable.getId(), partitionId, indexId, bucket++, schemaHash); Set tabletAllReplicas = Sets.newHashSet(); Set tabletFinishedReplicas = Sets.newHashSet(); @@ -467,7 +463,7 @@ private Set submitPushTasks() throws UserException { long taskSignature = Catalog.getCurrentGlobalTransactionMgr() .getTransactionIDGenerator().getNextTransactionId(); - PushBrokerReaderParams params = getPushBrokerReaderParams(table, indexId); + PushBrokerReaderParams params = getPushBrokerReaderParams(olapTable, indexId); // deep copy TBrokerScanRange because filePath and fileSize will be updated // in different tablet push task TBrokerScanRange tBrokerScanRange = new TBrokerScanRange(params.tBrokerScanRange); @@ -493,7 +489,7 @@ private Set submitPushTasks() throws UserException { replicaId, fsBroker.ip, fsBroker.port, backendId, tBrokerRangeDesc.path, tBrokerRangeDesc.file_size); - PushTask pushTask = new PushTask(backendId, dbId, tableId, partitionId, + PushTask pushTask = new PushTask(backendId, dbId, olapTable.getId(), partitionId, indexId, tabletId, replicaId, schemaHash, 0, id, TPushType.LOAD_V2, TPriority.NORMAL, transactionId, taskSignature, @@ -546,7 +542,7 @@ private Set submitPushTasks() throws UserException { writeUnlock(); } } finally { - db.readUnlock(); + MetaLockUtils.readUnlockTables(tableList); } } @@ -621,18 +617,9 @@ private void tryCommitJob() throws UserException { .add("txn_id", transactionId) .add("msg", "Load job try to commit txn") .build()); - List
tableList = Lists.newArrayList(); Database db = getDb(); - // The database will not be locked in here. - // The getTable is a thread-safe method called without read lock of database - for (long tableId : fileGroupAggInfo.getAllTableIds()) { - Table table = db.getTable(tableId); - if (table == null) { - throw new MetaNotFoundException("Failed to find table " + tableId + " in db " + dbId); - } else { - tableList.add(table); - } - } + List
tableList = db.getTablesOnIdOrderOrThrowException(Lists.newArrayList(tableToLoadPartitions.keySet())); + MetaLockUtils.writeLockTables(tableList); try { Catalog.getCurrentGlobalTransactionMgr().commitTransaction( dbId, tableList, transactionId, commitInfos, @@ -641,9 +628,7 @@ private void tryCommitJob() throws UserException { } catch (TabletQuorumFailedException e) { // retry in next loop } finally { - for (int i = tableList.size() - 1; i >= 0; i--) { - tableList.get(i).writeUnlock(); - } + MetaLockUtils.writeUnlockTables(tableList); } } From dc877535de1d0b0f5bb8d97a8547de82ff27464c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Wed, 4 Nov 2020 20:55:07 +0800 Subject: [PATCH 30/50] fix unittest failed --- .../java/org/apache/doris/alter/Alter.java | 249 ++++++++---------- .../doris/alter/MaterializedViewHandler.java | 1 + .../doris/alter/SchemaChangeHandler.java | 2 +- .../org/apache/doris/catalog/Catalog.java | 53 ++-- .../org/apache/doris/catalog/Database.java | 4 + .../org/apache/doris/alter/AlterTest.java | 1 + 6 files changed, 141 insertions(+), 169 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index e27e1e4e029d64..2c63d6075c1dfe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -150,55 +150,70 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable, "Table[" + olapTable.getName() + "]'s state is not NORMAL. Do not allow doing ALTER ops"); } - boolean needProcessOutsideDatabaseLock = false; + boolean needProcessOutsideTableLock = false; if (currentAlterOps.hasSchemaChangeOp()) { // if modify storage type to v2, do schema change to convert all related tablets to segment v2 format - schemaChangeHandler.process(alterClauses, clusterName, db, olapTable); + olapTable.writeLock(); + try { + schemaChangeHandler.process(alterClauses, clusterName, db, olapTable); + } finally { + olapTable.writeUnlock(); + } } else if (currentAlterOps.hasRollupOp()) { - materializedViewHandler.process(alterClauses, clusterName, db, olapTable); + olapTable.writeLock(); + try { + materializedViewHandler.process(alterClauses, clusterName, db, olapTable); + } finally { + olapTable.writeUnlock(); + } } else if (currentAlterOps.hasPartitionOp()) { Preconditions.checkState(alterClauses.size() == 1); AlterClause alterClause = alterClauses.get(0); - if (alterClause instanceof DropPartitionClause) { - if (!((DropPartitionClause) alterClause).isTempPartition()) { - DynamicPartitionUtil.checkAlterAllowed((OlapTable) db.getTable(olapTable.getName())); - } - Catalog.getCurrentCatalog().dropPartition(db, olapTable, ((DropPartitionClause) alterClause)); - } else if (alterClause instanceof ReplacePartitionClause) { - Catalog.getCurrentCatalog().replaceTempPartition(db, olapTable.getName(), (ReplacePartitionClause) alterClause); - } else if (alterClause instanceof ModifyPartitionClause) { - ModifyPartitionClause clause = ((ModifyPartitionClause) alterClause); - // expand the partition names if it is 'Modify Partition(*)' - if (clause.isNeedExpand()) { - List partitionNames = clause.getPartitionNames(); - partitionNames.clear(); - for (Partition partition : olapTable.getPartitions()) { - partitionNames.add(partition.getName()); + olapTable.writeLock(); + try { + if (alterClause instanceof DropPartitionClause) { + if (!((DropPartitionClause) alterClause).isTempPartition()) { + DynamicPartitionUtil.checkAlterAllowed(olapTable); } - } - Map properties = clause.getProperties(); - if (properties.containsKey(PropertyAnalyzer.PROPERTIES_INMEMORY)) { - needProcessOutsideDatabaseLock = true; + Catalog.getCurrentCatalog().dropPartition(db, olapTable, ((DropPartitionClause) alterClause)); + } else if (alterClause instanceof ReplacePartitionClause) { + Catalog.getCurrentCatalog().replaceTempPartition(db, olapTable, (ReplacePartitionClause) alterClause); + } else if (alterClause instanceof ModifyPartitionClause) { + ModifyPartitionClause clause = ((ModifyPartitionClause) alterClause); + // expand the partition names if it is 'Modify Partition(*)' + if (clause.isNeedExpand()) { + List partitionNames = clause.getPartitionNames(); + partitionNames.clear(); + for (Partition partition : olapTable.getPartitions()) { + partitionNames.add(partition.getName()); + } + } + Map properties = clause.getProperties(); + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_INMEMORY)) { + needProcessOutsideTableLock = true; + } else { + List partitionNames = clause.getPartitionNames(); + modifyPartitionsProperty(db, olapTable, partitionNames, properties); + } + } else if (alterClause instanceof AddPartitionClause) { + needProcessOutsideTableLock = true; } else { - List partitionNames = clause.getPartitionNames(); - modifyPartitionsProperty(db, olapTable, partitionNames, properties); + throw new DdlException("Invalid alter operation: " + alterClause.getOpType()); } - } else if (alterClause instanceof AddPartitionClause) { - needProcessOutsideDatabaseLock = true; - } else { - throw new DdlException("Invalid alter operation: " + alterClause.getOpType()); + } finally { + olapTable.writeUnlock(); } } else if (currentAlterOps.hasRenameOp()) { processRename(db, olapTable, alterClauses); } else if (currentAlterOps.hasReplaceTableOp()) { processReplaceTable(db, olapTable, alterClauses); } else if (currentAlterOps.contains(AlterOpType.MODIFY_TABLE_PROPERTY_SYNC)) { - needProcessOutsideDatabaseLock = true; + needProcessOutsideTableLock = true; } else { throw new DdlException("Invalid alter operations: " + currentAlterOps); } - return needProcessOutsideDatabaseLock; + return needProcessOutsideTableLock; } private void processAlterExternalTable(AlterTableStmt stmt, Table externalTable, Database db) throws UserException { @@ -208,11 +223,15 @@ private void processAlterExternalTable(AlterTableStmt stmt, Table externalTable, List alterClauses = stmt.getOps(); AlterOperations currentAlterOps = new AlterOperations(); currentAlterOps.checkConflict(alterClauses); - if (currentAlterOps.hasRenameOp()) { processRename(db, externalTable, alterClauses); } else if (currentAlterOps.hasSchemaChangeOp()) { - schemaChangeHandler.processExternalTable(alterClauses, db, externalTable); + externalTable.writeLock(); + try { + schemaChangeHandler.processExternalTable(alterClauses, db, externalTable); + } finally { + externalTable.writeUnlock(); + } } } @@ -237,88 +256,24 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { // check conflict alter ops first AlterOperations currentAlterOps = new AlterOperations(); currentAlterOps.checkConflict(alterClauses); - // check cluster capacity and db quota, only need to check once. - currentAlterOps.checkConflict(stmt.getOps()); // check cluster capacity and db quota outside table lock to escape dead lock, only need to check once. if (currentAlterOps.needCheckCapacity()) { Catalog.getCurrentSystemInfo().checkClusterCapacity(clusterName); db.checkQuota(); } - table.writeLock(); - try { - switch (table.getType()) { - case OLAP: - OlapTable olapTable = (OlapTable) table; - needProcessOutsideTableLock = processAlterOlapTable(stmt, olapTable, alterClauses, clusterName, db); - break; - case ODBC: - case MYSQL: - case ELASTICSEARCH: - processAlterExternalTable(stmt, table, db); - return; - default: - throw new DdlException("Do not support alter " + table.getType().toString() + " table[" + tableName + "]"); - } - - OlapTable olapTable = (OlapTable) table; - stmt.rewriteAlterClause(olapTable); - alterClauses = stmt.getOps(); - currentAlterOps = new AlterOperations(); - currentAlterOps.checkConflict(alterClauses); - if (olapTable.getState() != OlapTableState.NORMAL) { - throw new DdlException( - "Table[" + olapTable.getName() + "]'s state is not NORMAL. Do not allow doing ALTER ops"); - } - - if (currentAlterOps.hasSchemaChangeOp()) { - // if modify storage type to v2, do schema change to convert all related tablets to segment v2 format - schemaChangeHandler.process(alterClauses, clusterName, db, olapTable); - } else if (currentAlterOps.hasRollupOp()) { - materializedViewHandler.process(alterClauses, clusterName, db, olapTable); - } else if (currentAlterOps.hasPartitionOp()) { - Preconditions.checkState(alterClauses.size() == 1); - AlterClause alterClause = alterClauses.get(0); - if (alterClause instanceof DropPartitionClause) { - if (!((DropPartitionClause) alterClause).isTempPartition()) { - DynamicPartitionUtil.checkAlterAllowed((OlapTable) db.getTable(tableName)); - } - Catalog.getCurrentCatalog().dropPartition(db, olapTable, ((DropPartitionClause) alterClause)); - } else if (alterClause instanceof ReplacePartitionClause) { - Catalog.getCurrentCatalog().replaceTempPartition(db, tableName, (ReplacePartitionClause) alterClause); - } else if (alterClause instanceof ModifyPartitionClause) { - ModifyPartitionClause clause = ((ModifyPartitionClause) alterClause); - // expand the partition names if it is 'Modify Partition(*)' - if (clause.isNeedExpand()) { - List partitionNames = clause.getPartitionNames(); - partitionNames.clear(); - for (Partition partition : olapTable.getPartitions()) { - partitionNames.add(partition.getName()); - } - } - Map properties = clause.getProperties(); - if (properties.containsKey(PropertyAnalyzer.PROPERTIES_INMEMORY)) { - needProcessOutsideTableLock = true; - } else { - List partitionNames = clause.getPartitionNames(); - modifyPartitionsProperty(db, olapTable, partitionNames, properties); - } - } else if (alterClause instanceof AddPartitionClause) { - needProcessOutsideTableLock = true; - } else { - throw new DdlException("Invalid alter operation: " + alterClause.getOpType()); - } - } else if (currentAlterOps.hasRenameOp()) { - processRename(db, olapTable, alterClauses); - } else if (currentAlterOps.hasReplaceTableOp()) { - processReplaceTable(db, olapTable, alterClauses); - } else if (currentAlterOps.contains(AlterOpType.MODIFY_TABLE_PROPERTY_SYNC)) { - needProcessOutsideTableLock = true; - } else { - throw new DdlException("Invalid alter operations: " + currentAlterOps); - } - } finally { - table.writeUnlock(); + switch (table.getType()) { + case OLAP: + OlapTable olapTable = (OlapTable) table; + needProcessOutsideTableLock = processAlterOlapTable(stmt, olapTable, alterClauses, clusterName, db); + break; + case ODBC: + case MYSQL: + case ELASTICSEARCH: + processAlterExternalTable(stmt, table, db); + return; + default: + throw new DdlException("Do not support alter " + table.getType().toString() + " table[" + tableName + "]"); } // the following ops should done outside table lock. because it contain synchronized create operation @@ -359,26 +314,29 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { // entry of processing replace table private void processReplaceTable(Database db, OlapTable origTable, List alterClauses) throws UserException { ReplaceTableClause clause = (ReplaceTableClause) alterClauses.get(0); - Preconditions.checkState(origTable.isWriteLockHeldByCurrentThread()); - - String oldTblName = origTable.getName(); String newTblName = clause.getTblName(); + boolean swapTable = clause.isSwapTable(); Table newTbl = db.getTableOrThrowException(newTblName, TableType.OLAP); OlapTable olapNewTbl = (OlapTable) newTbl; - boolean swapTable = clause.isSwapTable(); - // First, we need to check whether the table to be operated on can be renamed - olapNewTbl.checkAndSetName(oldTblName, true); - if (swapTable) { - origTable.checkAndSetName(newTblName, true); + db.writeLock(); + origTable.writeLock(); + try { + String oldTblName = origTable.getName(); + // First, we need to check whether the table to be operated on can be renamed + olapNewTbl.checkAndSetName(oldTblName, true); + if (swapTable) { + origTable.checkAndSetName(newTblName, true); + } + replaceTableInternal(db, origTable, olapNewTbl, swapTable); + // write edit log + ReplaceTableOperationLog log = new ReplaceTableOperationLog(db.getId(), origTable.getId(), olapNewTbl.getId(), swapTable); + Catalog.getCurrentCatalog().getEditLog().logReplaceTable(log); + LOG.info("finish replacing table {} with table {}, is swap: {}", oldTblName, newTblName, swapTable); + } finally { + origTable.writeUnlock(); + db.writeUnlock(); } - replaceTableInternal(db, origTable, olapNewTbl, swapTable); - - // write edit log - ReplaceTableOperationLog log = new ReplaceTableOperationLog(db.getId(), origTable.getId(), olapNewTbl.getId(), swapTable); - Catalog.getCurrentCatalog().getEditLog().logReplaceTable(log); - - LOG.info("finish replacing table {} with table {}, is swap: {}", oldTblName, newTblName, swapTable); } public void replayReplaceTable(ReplaceTableOperationLog log) { @@ -506,19 +464,33 @@ public void processAlterCluster(AlterSystemStmt stmt) throws UserException { private void processRename(Database db, OlapTable table, List alterClauses) throws DdlException { for (AlterClause alterClause : alterClauses) { if (alterClause instanceof TableRenameClause) { - Catalog.getCurrentCatalog().renameTable(db, table, (TableRenameClause) alterClause); - break; - } else if (alterClause instanceof RollupRenameClause) { - Catalog.getCurrentCatalog().renameRollup(db, table, (RollupRenameClause) alterClause); - break; - } else if (alterClause instanceof PartitionRenameClause) { - Catalog.getCurrentCatalog().renamePartition(db, table, (PartitionRenameClause) alterClause); - break; - } else if (alterClause instanceof ColumnRenameClause) { - Catalog.getCurrentCatalog().renameColumn(db, table, (ColumnRenameClause) alterClause); + db.writeLock(); + table.writeLock(); + try { + Catalog.getCurrentCatalog().renameTable(db, table, (TableRenameClause) alterClause); + } finally { + table.writeUnlock(); + db.writeUnlock(); + } break; } else { - Preconditions.checkState(false); + table.writeLock(); + try { + if (alterClause instanceof RollupRenameClause) { + Catalog.getCurrentCatalog().renameRollup(db, table, (RollupRenameClause) alterClause); + break; + } else if (alterClause instanceof PartitionRenameClause) { + Catalog.getCurrentCatalog().renamePartition(db, table, (PartitionRenameClause) alterClause); + break; + } else if (alterClause instanceof ColumnRenameClause) { + Catalog.getCurrentCatalog().renameColumn(db, table, (ColumnRenameClause) alterClause); + break; + } else { + Preconditions.checkState(false); + } + } finally { + table.writeUnlock(); + } } } } @@ -526,7 +498,14 @@ private void processRename(Database db, OlapTable table, List alter private void processRename(Database db, Table table, List alterClauses) throws DdlException { for (AlterClause alterClause : alterClauses) { if (alterClause instanceof TableRenameClause) { - Catalog.getCurrentCatalog().renameTable(db, table, (TableRenameClause) alterClause); + db.writeLock(); + table.writeLock(); + try { + Catalog.getCurrentCatalog().renameTable(db, table, (TableRenameClause) alterClause); + } finally { + table.writeUnlock(); + db.writeUnlock(); + } break; } else { Preconditions.checkState(false); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java index 17a01fb31029bc..32781b9b62c6e6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java @@ -1173,6 +1173,7 @@ private void getOldAlterJobInfos(Database db, List> rollupJobIn @Override public void process(List alterClauses, String clusterName, Database db, OlapTable olapTable) throws DdlException, AnalysisException, MetaNotFoundException { + Preconditions.checkState(olapTable.isWriteLockHeldByCurrentThread()); Optional alterClauseOptional = alterClauses.stream().findAny(); if (alterClauseOptional.isPresent()) { if (alterClauseOptional.get() instanceof AddRollupClause) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index 4061d64214e14f..ae72eb229c7d52 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -1586,7 +1586,7 @@ private void getOldAlterJobInfos(Database db, List> schemaChang @Override public void process(List alterClauses, String clusterName, Database db, OlapTable olapTable) throws UserException { - + Preconditions.checkState(olapTable.isWriteLockHeldByCurrentThread()); // index id -> index schema Map> indexSchemaMap = new HashMap<>(); for (Map.Entry> entry : olapTable.getIndexIdToSchema(true).entrySet()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index 7a1c95c13f40d4..cd6217bfdd2b63 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -5107,6 +5107,8 @@ public void cancelBackup(CancelBackupStmt stmt) throws DdlException { // entry of rename table operation public void renameTable(Database db, Table table, TableRenameClause tableRenameClause) throws DdlException { + Preconditions.checkState(db.isWriteLockHeldByCurrentThread()); + Preconditions.checkState(table.isWriteLockHeldByCurrentThread()); if (table instanceof OlapTable) { OlapTable olapTable = (OlapTable) table; if ( olapTable.getState() != OlapTableState.NORMAL) { @@ -6732,46 +6734,31 @@ public void replayConvertDistributionType(TableInfo tableInfo) { /* * The entry of replacing partitions with temp partitions. */ - public void replaceTempPartition(Database db, String tableName, ReplacePartitionClause clause) throws DdlException { + public void replaceTempPartition(Database db, OlapTable olapTable, ReplacePartitionClause clause) throws DdlException { + Preconditions.checkState(olapTable.isWriteLockHeldByCurrentThread()); List partitionNames = clause.getPartitionNames(); List tempPartitionNames = clause.getTempPartitionNames(); boolean isStrictRange = clause.isStrictRange(); boolean useTempPartitionName = clause.useTempPartitionName(); - Table table = db.getTable(tableName); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } - - if (table.getType() != TableType.OLAP) { - throw new DdlException("Table[" + tableName + "] is not OLAP table"); - } - - OlapTable olapTable = (OlapTable) table; - olapTable.writeLock(); - try { - // check partition exist - for (String partName : partitionNames) { - if (!olapTable.checkPartitionNameExist(partName, false)) { - throw new DdlException("Partition[" + partName + "] does not exist"); - } + // check partition exist + for (String partName : partitionNames) { + if (!olapTable.checkPartitionNameExist(partName, false)) { + throw new DdlException("Partition[" + partName + "] does not exist"); } - for (String partName : tempPartitionNames) { - if (!olapTable.checkPartitionNameExist(partName, true)) { - throw new DdlException("Temp partition[" + partName + "] does not exist"); - } + } + for (String partName : tempPartitionNames) { + if (!olapTable.checkPartitionNameExist(partName, true)) { + throw new DdlException("Temp partition[" + partName + "] does not exist"); } - - olapTable.replaceTempPartitions(partitionNames, tempPartitionNames, isStrictRange, useTempPartitionName); - - // write log - ReplacePartitionOperationLog info = new ReplacePartitionOperationLog(db.getId(), olapTable.getId(), - partitionNames, tempPartitionNames, isStrictRange, useTempPartitionName); - editLog.logReplaceTempPartition(info); - LOG.info("finished to replace partitions {} with temp partitions {} from table: {}", - clause.getPartitionNames(), clause.getTempPartitionNames(), tableName); - } finally { - olapTable.writeUnlock(); } + olapTable.replaceTempPartitions(partitionNames, tempPartitionNames, isStrictRange, useTempPartitionName); + + // write log + ReplacePartitionOperationLog info = new ReplacePartitionOperationLog(db.getId(), olapTable.getId(), + partitionNames, tempPartitionNames, isStrictRange, useTempPartitionName); + editLog.logReplaceTempPartition(info); + LOG.info("finished to replace partitions {} with temp partitions {} from table: {}", + clause.getPartitionNames(), clause.getTempPartitionNames(), olapTable.getName()); } public void replayReplaceTempPartition(ReplacePartitionOperationLog replaceTempPartitionLog) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index efdfd73ee0b16f..bc93dcc52392ee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -154,6 +154,10 @@ public boolean tryWriteLock(long timeout, TimeUnit unit) { } } + public boolean isWriteLockHeldByCurrentThread() { + return this.rwLock.writeLock().isHeldByCurrentThread(); + } + public long getId() { return id; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java index 41cd00441fd851..694c9d8104c6e9 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java @@ -571,6 +571,7 @@ public void testReplaceTable() throws Exception { Assert.assertNotNull(replace3.getIndexIdByName("r2")); } + @Test public void testExternalTableAlterOperations() throws Exception { // external table do not support partition operation String stmt = "alter table test.odbc_table add partition p3 values less than('2020-04-01'), add partition p4 values less than('2020-05-01')"; From 07d7314e8cf12fc2840a980397b23a33ae5738b0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Wed, 4 Nov 2020 21:23:29 +0800 Subject: [PATCH 31/50] fix unit test failed --- .../apache/doris/alter/RollupJobV2Test.java | 28 ++++++-- .../doris/alter/SchemaChangeJobV2Test.java | 72 ++++++++++++++++--- 2 files changed, 85 insertions(+), 15 deletions(-) diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java index 7293740e4fa85a..5931f825448443 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java +++ b/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java @@ -158,7 +158,12 @@ public void testRunRollupJobConcurrentLimit() throws UserException { alterClauses.add(clause2); Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); - materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); + olapTable.writeLock(); + try { + materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); + } finally { + olapTable.writeUnlock(); + } Map alterJobsV2 = materializedViewHandler.getAlterJobsV2(); materializedViewHandler.runAfterCatalogReady(); @@ -178,7 +183,12 @@ public void testAddSchemaChange() throws UserException { alterClauses.add(clause); Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); - materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); + olapTable.writeLock(); + try { + materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); + } finally { + olapTable.writeUnlock(); + } Map alterJobsV2 = materializedViewHandler.getAlterJobsV2(); Assert.assertEquals(1, alterJobsV2.size()); Assert.assertEquals(OlapTableState.ROLLUP, olapTable.getState()); @@ -198,7 +208,12 @@ public void testSchemaChange1() throws Exception { Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); - materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); + olapTable.writeLock(); + try { + materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); + } finally { + olapTable.writeUnlock(); + } Map alterJobsV2 = materializedViewHandler.getAlterJobsV2(); Assert.assertEquals(1, alterJobsV2.size()); RollupJobV2 rollupJob = (RollupJobV2) alterJobsV2.values().stream().findAny().get(); @@ -302,7 +317,12 @@ public void testSchemaChangeWhileTabletNotStable() throws Exception { Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); - materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); + olapTable.writeLock(); + try { + materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); + } finally { + olapTable.writeUnlock(); + } Map alterJobsV2 = materializedViewHandler.getAlterJobsV2(); Assert.assertEquals(1, alterJobsV2.size()); RollupJobV2 rollupJob = (RollupJobV2) alterJobsV2.values().stream().findAny().get(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java index b48c8b811d5fe9..1433b50590450e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java +++ b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java @@ -65,6 +65,7 @@ import com.google.common.collect.Maps; +import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -136,7 +137,12 @@ public void testAddSchemaChange() throws UserException { alterClauses.add(addColumnClause); Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); - schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); + olapTable.writeLock(); + try { + schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); + } finally { + olapTable.writeUnlock(); + } Map alterJobsV2 = schemaChangeHandler.getAlterJobsV2(); Assert.assertEquals(1, alterJobsV2.size()); Assert.assertEquals(OlapTableState.SCHEMA_CHANGE, olapTable.getState()); @@ -156,7 +162,12 @@ public void testSchemaChange1() throws Exception { Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); - schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); + olapTable.writeLock(); + try { + schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); + } finally { + olapTable.writeUnlock(); + } Map alterJobsV2 = schemaChangeHandler.getAlterJobsV2(); Assert.assertEquals(1, alterJobsV2.size()); SchemaChangeJobV2 schemaChangeJob = (SchemaChangeJobV2) alterJobsV2.values().stream().findAny().get(); @@ -232,7 +243,12 @@ public void testSchemaChangeWhileTabletNotStable() throws Exception { Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); - schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); + olapTable.writeLock(); + try { + schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); + } finally { + olapTable.writeUnlock(); + } Map alterJobsV2 = schemaChangeHandler.getAlterJobsV2(); Assert.assertEquals(1, alterJobsV2.size()); SchemaChangeJobV2 schemaChangeJob = (SchemaChangeJobV2) alterJobsV2.values().stream().findAny().get(); @@ -317,7 +333,12 @@ public void testModifyDynamicPartitionNormal() throws UserException { alterClauses.add(new ModifyTablePropertiesClause(properties)); Database db = CatalogMocker.mockDb(); OlapTable olapTable = (OlapTable) db.getTable(CatalogMocker.TEST_TBL2_ID); - schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); + olapTable.writeLock(); + try { + schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); + } finally { + olapTable.writeUnlock(); + } Assert.assertTrue(olapTable.getTableProperty().getDynamicPartitionProperty().isExist()); Assert.assertTrue(olapTable.getTableProperty().getDynamicPartitionProperty().getEnable()); Assert.assertEquals("day", olapTable.getTableProperty().getDynamicPartitionProperty().getTimeUnit()); @@ -330,31 +351,56 @@ public void testModifyDynamicPartitionNormal() throws UserException { ArrayList tmpAlterClauses = new ArrayList<>(); properties.put(DynamicPartitionProperty.ENABLE, "false"); tmpAlterClauses.add(new ModifyTablePropertiesClause(properties)); - schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); + olapTable.writeLock(); + try { + schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); + } finally { + olapTable.writeUnlock(); + } Assert.assertFalse(olapTable.getTableProperty().getDynamicPartitionProperty().getEnable()); // set dynamic_partition.time_unit = week tmpAlterClauses = new ArrayList<>(); properties.put(DynamicPartitionProperty.TIME_UNIT, "week"); tmpAlterClauses.add(new ModifyTablePropertiesClause(properties)); - schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); + olapTable.writeLock(); + try { + schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); + } finally { + olapTable.writeUnlock(); + } Assert.assertEquals("week", olapTable.getTableProperty().getDynamicPartitionProperty().getTimeUnit()); // set dynamic_partition.end = 10 tmpAlterClauses = new ArrayList<>(); properties.put(DynamicPartitionProperty.END, "10"); tmpAlterClauses.add(new ModifyTablePropertiesClause(properties)); - schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); + olapTable.writeLock(); + try { + schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); + } finally { + olapTable.writeUnlock(); + } Assert.assertEquals(10, olapTable.getTableProperty().getDynamicPartitionProperty().getEnd()); // set dynamic_partition.prefix = p1 tmpAlterClauses = new ArrayList<>(); properties.put(DynamicPartitionProperty.PREFIX, "p1"); tmpAlterClauses.add(new ModifyTablePropertiesClause(properties)); - schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); + olapTable.writeLock(); + try { + schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); + } finally { + olapTable.writeUnlock(); + } Assert.assertEquals("p1", olapTable.getTableProperty().getDynamicPartitionProperty().getPrefix()); // set dynamic_partition.buckets = 3 tmpAlterClauses = new ArrayList<>(); properties.put(DynamicPartitionProperty.BUCKETS, "3"); tmpAlterClauses.add(new ModifyTablePropertiesClause(properties)); - schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); + olapTable.writeLock(); + try { + schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); + } finally { + olapTable.writeUnlock(); + } Assert.assertEquals(3, olapTable.getTableProperty().getDynamicPartitionProperty().getBuckets()); } @@ -374,8 +420,12 @@ public void modifyDynamicPartitionWithoutTableProperty(String propertyKey, Strin expectedEx.expect(DdlException.class); expectedEx.expectMessage("errCode = 2, detailMessage = Table test_db.test_tbl2 is not a dynamic partition table. " + "Use command `HELP ALTER TABLE` to see how to change a normal table to a dynamic partition table."); - - schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); + olapTable.writeLock(); + try { + schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); + } finally { + olapTable.writeUnlock(); + } } @Test From 6a983540b048297f5d793f2c3d52e4d2e153aa65 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Thu, 5 Nov 2020 16:11:47 +0800 Subject: [PATCH 32/50] Fix SparkLoadJobTest Failed --- .../java/org/apache/doris/load/loadv2/SparkLoadJobTest.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java index b40f2fe9e6ef9a..b975b2e070e581 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java @@ -355,8 +355,10 @@ public void testUpdateEtlStatusFinishedAndCommitTransaction( result = filePathToSize; catalog.getDb(dbId); result = db; - db.getTable(tableId); - result = table; + db.getTablesOnIdOrderOrThrowException((List) any); + result = Lists.newArrayList(table); + table.getId(); + result = tableId; table.getPartition(partitionId); result = partition; table.getPartitionInfo(); From b3b30a75c7377053dd3881618264c8d081d9d70b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Thu, 5 Nov 2020 18:02:45 +0800 Subject: [PATCH 33/50] Fix save image lock --- .../org/apache/doris/catalog/Catalog.java | 21 +++----- .../doris/common/util/MetaLockUtils.java | 49 ------------------- .../apache/doris/http/meta/MetaService.java | 1 - 3 files changed, 7 insertions(+), 64 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index cd6217bfdd2b63..f02bf5c2d3aaf9 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -2044,15 +2044,7 @@ public long saveDb(DataOutputStream dos, long checksum) throws IOException { // Don't write information_schema db meta if (!InfoSchemaDb.isInfoSchemaDb(dbName)) { checksum ^= entry.getKey(); - db.readLock(); - List
tableList = db.getTablesOnIdOrder(); - MetaLockUtils.readLockTables(tableList); - try { - db.write(dos); - } finally { - MetaLockUtils.readUnlockTables(tableList); - db.readUnlock(); - } + db.write(dos); } } return checksum; @@ -6347,16 +6339,17 @@ public String dumpImage() { databases.sort(Comparator.comparing(Database::getId)); // lock all dbs + MetaLockUtils.readLockDatabases(databases); + LOG.info("acquired all the dbs' read lock."); + // lock all tables for (Database db : databases) { - db.readLock(); List
tableList = db.getTablesOnIdOrder(); MetaLockUtils.readLockTables(tableList); tableLists.add(tableList); } - LOG.info("acquired all the dbs' read lock."); + LOG.info("acquired all the tables' read lock."); load.readLock(); - LOG.info("acquired all jobs' read lock."); long journalId = getMaxJournalId(); File dumpFile = new File(Config.meta_dir, "image." + journalId); @@ -6369,11 +6362,11 @@ public String dumpImage() { } } finally { // unlock all + load.readUnlock(); for (int i = databases.size() - 1; i >= 0; i--) { MetaLockUtils.readUnlockTables(tableLists.get(i)); - databases.get(i).readUnlock(); } - load.readUnlock(); + MetaLockUtils.readUnlockDatabases(databases); unlock(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java index e0278f05ad9245..d8506b30afc56b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java @@ -31,67 +31,18 @@ public static void readLockDatabases(List databaseList) { } } - public static boolean tryReadLockDatabases(List databaseList, long timeout, TimeUnit unit) { - for (int i = 0; i < databaseList.size(); i++) { - if (!databaseList.get(i).tryReadLock(timeout, unit)) { - for (int j = i - 1; j >= 0; j--) { - databaseList.get(j).readUnlock(); - } - return false; - } - } - return true; - } - public static void readUnlockDatabases(List databaseList) { for (int i = databaseList.size() - 1; i >= 0; i--) { databaseList.get(i).readUnlock(); } } - public static void writeLockDatabases(List databaseList) { - for (Database database : databaseList) { - database.writeLock(); - } - } - - public static boolean tryWriteLockDatabases(List databaseList, long timeout, TimeUnit unit) { - for (int i = 0; i < databaseList.size(); i++) { - if (!databaseList.get(i).tryWriteLock(timeout, unit)) { - for (int j = i - 1; j >= 0; j--) { - databaseList.get(j).writeUnlock(); - } - return false; - } - } - return true; - } - - public static void writeUnlockDatabases(List databaseList) { - for (int i = databaseList.size() - 1; i >= 0; i--) { - databaseList.get(i).writeUnlock(); - } - } - - public static void readLockTables(List
tableList) { for (Table table : tableList) { table.readLock(); } } - public static boolean tryReadLockTables(List
tableList, long timeout, TimeUnit unit) { - for (int i = 0; i < tableList.size(); i++) { - if (!tableList.get(i).tryReadLock(timeout, unit)) { - for (int j = i - 1; j >= 0; j--) { - tableList.get(j).readUnlock(); - } - return false; - } - } - return true; - } - public static void readUnlockTables(List
tableList) { for (int i = tableList.size() - 1; i >= 0; i--) { tableList.get(i).readUnlock(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/meta/MetaService.java b/fe/fe-core/src/main/java/org/apache/doris/http/meta/MetaService.java index 4f2d51ef6fa1c4..631b579f021b51 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/http/meta/MetaService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/http/meta/MetaService.java @@ -330,7 +330,6 @@ public void executeGet(BaseRequest request, BaseResponse response) { response.appendContent("dump finished. " + dumpFilePath); writeResponse(request, response); - return; } } } From 619b71cb15599bd9a015155d1922df56a770957f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Thu, 5 Nov 2020 20:19:36 +0800 Subject: [PATCH 34/50] Add unit test for MetaLockUtils --- .../doris/common/util/MetaLockUtilsTest.java | 76 +++++++++++++++++++ 1 file changed, 76 insertions(+) create mode 100644 fe/fe-core/src/test/java/org/apache/doris/common/util/MetaLockUtilsTest.java diff --git a/fe/fe-core/src/test/java/org/apache/doris/common/util/MetaLockUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/common/util/MetaLockUtilsTest.java new file mode 100644 index 00000000000000..47fb1fffb05fb5 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/common/util/MetaLockUtilsTest.java @@ -0,0 +1,76 @@ +// 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.util; + +import com.google.common.collect.Lists; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Table; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +public class MetaLockUtilsTest { + + @Test + public void testReadLockDatabases() { + List databaseList = Lists.newArrayList(new Database(), new Database()); + MetaLockUtils.readLockDatabases(databaseList); + Assert.assertFalse(databaseList.get(0).tryWriteLock(1, TimeUnit.MILLISECONDS)); + Assert.assertFalse(databaseList.get(1).tryWriteLock(1, TimeUnit.MILLISECONDS)); + MetaLockUtils.readUnlockDatabases(databaseList); + Assert.assertTrue(databaseList.get(0).tryWriteLock(1, TimeUnit.MILLISECONDS)); + Assert.assertTrue(databaseList.get(1).tryWriteLock(1, TimeUnit.MILLISECONDS)); + databaseList.get(0).writeUnlock(); + databaseList.get(1).writeUnlock(); + } + + @Test + public void testReadLockTables() { + List
tableList = Lists.newArrayList(new Table(Table.TableType.OLAP), new Table(Table.TableType.OLAP)); + MetaLockUtils.readLockTables(tableList); + Assert.assertFalse(tableList.get(0).tryWriteLock(1, TimeUnit.MILLISECONDS)); + Assert.assertFalse(tableList.get(1).tryWriteLock(1, TimeUnit.MILLISECONDS)); + MetaLockUtils.readUnlockTables(tableList); + Assert.assertTrue(tableList.get(0).tryWriteLock(1, TimeUnit.MILLISECONDS)); + Assert.assertTrue(tableList.get(1).tryWriteLock(1, TimeUnit.MILLISECONDS)); + tableList.get(0).writeUnlock(); + tableList.get(1).writeUnlock(); + } + + @Test + public void testWriteLockTables() { + List
tableList = Lists.newArrayList(new Table(Table.TableType.OLAP), new Table(Table.TableType.OLAP)); + MetaLockUtils.writeLockTables(tableList); + Assert.assertTrue(tableList.get(0).isWriteLockHeldByCurrentThread()); + Assert.assertTrue(tableList.get(1).isWriteLockHeldByCurrentThread()); + MetaLockUtils.writeUnlockTables(tableList); + Assert.assertFalse(tableList.get(0).isWriteLockHeldByCurrentThread()); + Assert.assertFalse(tableList.get(1).isWriteLockHeldByCurrentThread()); + Assert.assertTrue(MetaLockUtils.tryWriteLockTables(tableList, 1, TimeUnit.MILLISECONDS)); + Assert.assertTrue(tableList.get(0).isWriteLockHeldByCurrentThread()); + Assert.assertTrue(tableList.get(1).isWriteLockHeldByCurrentThread()); + MetaLockUtils.writeUnlockTables(tableList); + tableList.get(1).readLock(); + Assert.assertFalse(MetaLockUtils.tryWriteLockTables(tableList, 1, TimeUnit.MILLISECONDS)); + Assert.assertFalse(tableList.get(0).isWriteLockHeldByCurrentThread()); + Assert.assertFalse(tableList.get(1).isWriteLockHeldByCurrentThread()); + tableList.get(1).readUnlock(); + } +} From 8fb1e329c56a993c609335368f3fa660dd32f357 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Fri, 6 Nov 2020 20:38:50 +0800 Subject: [PATCH 35/50] use MetadataLockUtils to lock table when finish transaction --- .../apache/doris/transaction/DatabaseTransactionMgr.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java index d8a0d276f5ff85..cda888d58f4612 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java @@ -40,6 +40,7 @@ import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.common.util.Util; import org.apache.doris.metric.MetricRepo; @@ -679,9 +680,7 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) thr } List
tableList = db.getTablesOnIdOrderOrThrowException(tableIdList); - for (Table table : tableList) { - table.writeLock(); - } + MetaLockUtils.writeLockTables(tableList); try { boolean hasError = false; Iterator tableCommitInfoIterator = transactionState.getIdToTableCommitInfos().values().iterator(); @@ -822,9 +821,7 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) thr } updateCatalogAfterVisible(transactionState, db); } finally { - for (int i = tableList.size() - 1; i >=0; i--) { - tableList.get(i).writeUnlock(); - } + MetaLockUtils.writeUnlockTables(tableList); } LOG.info("finish transaction {} successfully", transactionState); } From 11e4e07052268206f44aeb85cb67814d2f7fbaa3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Mon, 9 Nov 2020 14:28:58 +0800 Subject: [PATCH 36/50] remove useless code of dropTableWithLock for Database --- .../java/org/apache/doris/catalog/Database.java | 13 ------------- .../java/org/apache/doris/catalog/InfoSchemaDb.java | 5 ----- .../java/org/apache/doris/catalog/DatabaseTest.java | 6 ------ .../org/apache/doris/catalog/InfoSchemaDbTest.java | 1 - 4 files changed, 25 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index bc93dcc52392ee..96269804e13158 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -328,19 +328,6 @@ public boolean createTable(Table table) { return result; } - public void dropTableWithLock(String tableName) { - writeLock(); - try { - Table table = this.nameToTable.get(tableName); - if (table != null) { - this.nameToTable.remove(tableName); - this.idToTable.remove(table.getId()); - } - } finally { - writeUnlock(); - } - } - public void dropTable(String tableName) { Table table = this.nameToTable.get(tableName); if (table != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/InfoSchemaDb.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/InfoSchemaDb.java index 7fcc9b4818205a..e577d4f256b351 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/InfoSchemaDb.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/InfoSchemaDb.java @@ -49,11 +49,6 @@ public boolean createTable(Table table) { return false; } - @Override - public void dropTableWithLock(String name) { - // Do nothing. - } - @Override public void dropTable(String name) { // Do nothing. diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DatabaseTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DatabaseTest.java index dff692d0e9b734..89aeec17b404f6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DatabaseTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DatabaseTest.java @@ -131,12 +131,6 @@ public void createAndDropPartitionTest() { // drop not exist tableFamily db.dropTable("invalid"); Assert.assertEquals(1, db.getTables().size()); - db.dropTableWithLock("invalid"); - Assert.assertEquals(1, db.getTables().size()); - - // drop normal - db.dropTableWithLock(table.getName()); - Assert.assertEquals(0, db.getTables().size()); db.createTable(table); db.dropTable(table.getName()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/InfoSchemaDbTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/InfoSchemaDbTest.java index 00ae89bc47696b..08a47a8ab1805d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/InfoSchemaDbTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/InfoSchemaDbTest.java @@ -30,7 +30,6 @@ public void testNormal() throws IOException { Assert.assertFalse(db.createTable(null)); Assert.assertFalse(db.createTableWithLock(null, false, false)); db.dropTable("authors"); - db.dropTableWithLock("authors"); db.write(null); Assert.assertNull(db.getTable("authors")); } From 1b089e40d678926205811ecdedd56f8c60c20ec7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Mon, 9 Nov 2020 16:48:40 +0800 Subject: [PATCH 37/50] remove useless db read lock --- .../apache/doris/catalog/TabletStatMgr.java | 9 +-- .../doris/common/proc/TablesProcDir.java | 9 +-- .../doris/consistency/ConsistencyChecker.java | 9 +-- .../doris/http/rest/MigrationAction.java | 8 +-- .../doris/http/rest/ShowDataAction.java | 11 ++-- .../http/rest/StorageTypeCheckAction.java | 8 +-- .../load/loadv2/SparkLoadPendingTask.java | 56 +++++++++++-------- .../load/routineload/KafkaRoutineLoadJob.java | 12 +--- .../load/routineload/RoutineLoadJob.java | 14 +---- 9 files changed, 52 insertions(+), 84 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java index 92543515c02d97..898a97a9088738 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java @@ -86,14 +86,7 @@ protected void runAfterCatalogReady() { if (db == null) { continue; } - List
tableList = null; - db.readLock(); - try { - tableList = db.getTables(); - } finally { - db.readUnlock(); - } - + List
tableList = db.getTables(); for (Table table : tableList) { if (table.getType() != TableType.OLAP) { continue; diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TablesProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TablesProcDir.java index d837c17a95057c..380618d3d3667f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TablesProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TablesProcDir.java @@ -86,14 +86,7 @@ public ProcResult fetchResult() throws AnalysisException { // get info List> tableInfos = new ArrayList>(); - List
tableList = null; - db.readLock(); - try { - tableList = db.getTables(); - } finally { - db.readUnlock(); - } - + List
tableList = db.getTables(); for (Table table : tableList) { List tableInfo = new ArrayList(); int partitionNum = 1; diff --git a/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java b/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java index f60073b66f122d..052847b4e6382e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java @@ -260,14 +260,7 @@ private List chooseTablets() { try { while ((chosenOne = dbQueue.poll()) != null) { Database db = (Database) chosenOne; - List
tables = null; - db.readLock(); - try { - tables = db.getTables(); - } finally { - db.readUnlock(); - } - + List
tables = db.getTables(); // sort tables Queue tableQueue = new PriorityQueue<>(Math.max(tables.size(), 1), COMPARATOR); for (Table table : tables) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/MigrationAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/MigrationAction.java index 32abdb69928978..50bbdd6d62086d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/MigrationAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/MigrationAction.java @@ -119,13 +119,7 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response olapTable.readUnlock(); } } else { - List
tableList = null; - db.readLock(); - try { - tableList = db.getTables(); - } finally { - db.readUnlock(); - } + List
tableList = db.getTables(); // get all olap table for (Table table : tableList) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/ShowDataAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/ShowDataAction.java index 2f4e77a8d91328..af8036eb8b8c38 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/ShowDataAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/ShowDataAction.java @@ -48,18 +48,21 @@ public static void registerAction (ActionController controller) throws IllegalAr public long getDataSizeOfDatabase(Database db) { long totalSize = 0; - db.readLock(); + long tableSize = 0; // sort by table name List
tables = db.getTables(); for (Table table : tables) { if (table.getType() != TableType.OLAP) { continue; } - - long tableSize = ((OlapTable)table).getDataSize(); + table.readLock(); + try { + tableSize = ((OlapTable)table).getDataSize(); + } finally { + table.readUnlock(); + } totalSize += tableSize; } // end for tables - db.readUnlock(); return totalSize; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/StorageTypeCheckAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/StorageTypeCheckAction.java index a0a3fe68e51055..b1906c51d636ac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/StorageTypeCheckAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/StorageTypeCheckAction.java @@ -67,13 +67,7 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response } JSONObject root = new JSONObject(); - List
tableList = null; - db.readLock(); - try { - tableList = db.getTables(); - } finally { - db.readUnlock(); - } + List
tableList = db.getTables(); for (Table tbl : tableList) { if (tbl.getType() != TableType.OLAP) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadPendingTask.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadPendingTask.java index 7445e2b4e9e0ff..f129f463cfca5f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadPendingTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadPendingTask.java @@ -38,9 +38,12 @@ import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.RangePartitionInfo; import org.apache.doris.catalog.SparkResource; +import org.apache.doris.catalog.Table; import org.apache.doris.common.LoadException; +import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.load.BrokerFileGroup; import org.apache.doris.load.BrokerFileGroupAggInfo.FileGroupAggKey; import org.apache.doris.load.FailMsg; @@ -133,13 +136,18 @@ private void createEtlJobConf() throws LoadException { } Map tables = Maps.newHashMap(); - db.readLock(); + Map> tableIdToPartitionIds = Maps.newHashMap(); + Set allPartitionsTableIds = Sets.newHashSet(); + prepareTablePartitionInfos(db, tableIdToPartitionIds, allPartitionsTableIds); + List
tableList = null; try { - Map> tableIdToPartitionIds = Maps.newHashMap(); - Set allPartitionsTableIds = Sets.newHashSet(); - prepareTablePartitionInfos(db, tableIdToPartitionIds, allPartitionsTableIds); - + tableList = db.getTablesOnIdOrderOrThrowException(Lists.newArrayList(allPartitionsTableIds)); + } catch (MetaNotFoundException e) { + throw new LoadException(e.getMessage()); + } + MetaLockUtils.readLockTables(tableList); + try { for (Map.Entry> entry : aggKeyToBrokerFileGroups.entrySet()) { FileGroupAggKey aggKey = entry.getKey(); long tableId = aggKey.getTableId(); @@ -176,7 +184,7 @@ private void createEtlJobConf() throws LoadException { } } } finally { - db.readUnlock(); + MetaLockUtils.readUnlockTables(tableList); } String outputFilePattern = EtlJobConfig.getOutputFilePattern(loadLabel, FilePatternVersion.V1); @@ -199,25 +207,29 @@ private void prepareTablePartitionInfos(Database db, Map> tableI if (table == null) { throw new LoadException("table does not exist. id: " + tableId); } + table.readLock(); + try { + Set partitionIds = null; + if (tableIdToPartitionIds.containsKey(tableId)) { + partitionIds = tableIdToPartitionIds.get(tableId); + } else { + partitionIds = Sets.newHashSet(); + tableIdToPartitionIds.put(tableId, partitionIds); + } - Set partitionIds = null; - if (tableIdToPartitionIds.containsKey(tableId)) { - partitionIds = tableIdToPartitionIds.get(tableId); - } else { - partitionIds = Sets.newHashSet(); - tableIdToPartitionIds.put(tableId, partitionIds); - } + Set groupPartitionIds = aggKey.getPartitionIds(); + // if not assign partition, use all partitions + if (groupPartitionIds == null || groupPartitionIds.isEmpty()) { + for (Partition partition : table.getPartitions()) { + partitionIds.add(partition.getId()); + } - Set groupPartitionIds = aggKey.getPartitionIds(); - // if not assign partition, use all partitions - if (groupPartitionIds == null || groupPartitionIds.isEmpty()) { - for (Partition partition : table.getPartitions()) { - partitionIds.add(partition.getId()); + allPartitionsTableIds.add(tableId); + } else { + partitionIds.addAll(groupPartitionIds); } - - allPartitionsTableIds.add(tableId); - } else { - partitionIds.addAll(groupPartitionIds); + } finally { + table.readUnlock(); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java index bb464ee1520bd7..856ae0d23c5845 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java @@ -346,15 +346,9 @@ public static KafkaRoutineLoadJob fromCreateStmt(CreateRoutineLoadStmt stmt) thr ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, stmt.getDBName()); } - long tableId = -1L; - db.readLock(); - try { - checkMeta(db, stmt.getTableName(), stmt.getRoutineLoadDesc()); - Table table = db.getTable(stmt.getTableName()); - tableId = table.getId(); - } finally { - db.readUnlock(); - } + checkMeta(db, stmt.getTableName(), stmt.getRoutineLoadDesc()); + Table table = db.getTable(stmt.getTableName()); + long tableId = table.getId(); // init kafka routine load job long id = Catalog.getCurrentCatalog().getNextId(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java index 1f1b70133f10ef..1dacd6169408b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java @@ -408,12 +408,8 @@ public String getDbFullName() throws MetaNotFoundException { if (database == null) { throw new MetaNotFoundException("Database " + dbId + "has been deleted"); } - database.readLock(); - try { - return database.getFullName(); - } finally { - database.readUnlock(); - } + + return database.getFullName(); } public long getTableId() { @@ -1267,11 +1263,7 @@ protected abstract boolean checkCommitInfo(RLTaskTxnCommitAttachment rlTaskTxnCo public List getShowInfo() { Database db = Catalog.getCurrentCatalog().getDb(dbId); - Table tbl = null; - if (db != null) { - db.readLock(); - tbl = db.getTable(tableId); - } + Table tbl = (db == null) ? null : db.getTable(tableId); readLock(); try { From 841d3f88daba40579d5d44a5f6130d4e136f0c1d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Mon, 9 Nov 2020 17:17:30 +0800 Subject: [PATCH 38/50] remove useless db read lock and change some db read lock to table locks --- .../org/apache/doris/catalog/Database.java | 9 -- .../load/routineload/RoutineLoadJob.java | 1 - .../org/apache/doris/qe/ShowExecutor.java | 95 ++++++++----------- 3 files changed, 39 insertions(+), 66 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 96269804e13158..5b8b49cfa446b2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -124,15 +124,6 @@ public void readLock() { this.rwLock.readLock().lock(); } - public boolean tryReadLock(long timeout, TimeUnit unit) { - try { - return this.rwLock.readLock().tryLock(timeout, unit); - } catch (InterruptedException e) { - LOG.warn("failed to try read lock at db[" + id + "]", e); - return false; - } - } - public void readUnlock() { this.rwLock.readLock().unlock(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java index 1dacd6169408b9..fc39a1cd8792d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java @@ -1210,7 +1210,6 @@ public void update() throws UserException { } // check table belong to database - database.readLock(); Table table = database.getTable(tableId); if (table == null) { LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id).add("db_id", dbId) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java index a6b52c80dfbf27..41dd674285935a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -456,27 +456,22 @@ private void handleShowTable() throws AnalysisException { Database db = ctx.getCatalog().getDb(showTableStmt.getDb()); if (db != null) { Map tableMap = Maps.newTreeMap(); - db.readLock(); - try { - PatternMatcher matcher = null; - if (showTableStmt.getPattern() != null) { - matcher = PatternMatcher.createMysqlPattern(showTableStmt.getPattern(), - CaseSensibility.TABLE.getCaseSensibility()); + PatternMatcher matcher = null; + if (showTableStmt.getPattern() != null) { + matcher = PatternMatcher.createMysqlPattern(showTableStmt.getPattern(), + CaseSensibility.TABLE.getCaseSensibility()); + } + for (Table tbl : db.getTables()) { + if (matcher != null && !matcher.match(tbl.getName())) { + continue; } - for (Table tbl : db.getTables()) { - if (matcher != null && !matcher.match(tbl.getName())) { - continue; - } - // check tbl privs - if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), - db.getFullName(), tbl.getName(), - PrivPredicate.SHOW)) { - continue; - } - tableMap.put(tbl.getName(), tbl.getMysqlType()); + // check tbl privs + if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), + db.getFullName(), tbl.getName(), + PrivPredicate.SHOW)) { + continue; } - } finally { - db.readUnlock(); + tableMap.put(tbl.getName(), tbl.getMysqlType()); } for (Map.Entry entry : tableMap.entrySet()) { @@ -498,39 +493,34 @@ private void handleShowTableStatus() throws AnalysisException { List> rows = Lists.newArrayList(); Database db = ctx.getCatalog().getDb(showStmt.getDb()); if (db != null) { - db.readLock(); - try { - PatternMatcher matcher = null; - if (showStmt.getPattern() != null) { - matcher = PatternMatcher.createMysqlPattern(showStmt.getPattern(), - CaseSensibility.TABLE.getCaseSensibility()); + PatternMatcher matcher = null; + if (showStmt.getPattern() != null) { + matcher = PatternMatcher.createMysqlPattern(showStmt.getPattern(), + CaseSensibility.TABLE.getCaseSensibility()); + } + for (Table table : db.getTables()) { + if (matcher != null && !matcher.match(table.getName())) { + continue; } - for (Table table : db.getTables()) { - if (matcher != null && !matcher.match(table.getName())) { - continue; - } - // check tbl privs - if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), - db.getFullName(), table.getName(), - PrivPredicate.SHOW)) { - continue; - } + // check tbl privs + if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), + db.getFullName(), table.getName(), + PrivPredicate.SHOW)) { + continue; + } - List row = Lists.newArrayList(); - // Name - row.add(table.getName()); - // Engine - row.add(table.getEngine()); - // version, ra - for (int i = 0; i < 15; ++i) { - row.add(null); - } - row.add(table.getComment()); - rows.add(row); + List row = Lists.newArrayList(); + // Name + row.add(table.getName()); + // Engine + row.add(table.getEngine()); + // version, ra + for (int i = 0; i < 15; ++i) { + row.add(null); } - } finally { - db.readUnlock(); + row.add(table.getComment()); + rows.add(row); } } resultSet = new ShowResultSet(showStmt.getMetaData(), rows); @@ -1536,14 +1526,7 @@ private void handleShowDynamicPartition() { List> rows = Lists.newArrayList(); Database db = ctx.getCatalog().getDb(showDynamicPartitionStmt.getDb()); if (db != null) { - List
tableList = null; - db.readLock(); - try { - tableList = db.getTables(); - } finally { - db.readUnlock(); - } - + List
tableList = db.getTables(); for (Table tbl : tableList) { if (!(tbl instanceof OlapTable)) { continue; From 520cd273fdb196ad5eae1f21f9ffcb63c76100d2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Mon, 9 Nov 2020 18:29:28 +0800 Subject: [PATCH 39/50] fix --- .../src/main/java/org/apache/doris/alter/AlterHandler.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java index 4c33934365f63b..54e7d0cbf27628 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java @@ -410,11 +410,7 @@ public void handleFinishAlterTask(AlterReplicaTask task) throws MetaNotFoundExce throw new MetaNotFoundException("database " + task.getDbId() + " does not exist"); } - OlapTable tbl = (OlapTable) db.getTable(task.getTableId()); - if (tbl == null) { - throw new MetaNotFoundException("tbl " + task.getTableId() + " does not exist"); - } - + OlapTable tbl = (OlapTable) db.getTableOrThrowException(task.getTableId(), Table.TableType.OLAP); tbl.writeLock(); try { Partition partition = tbl.getPartition(task.getPartitionId()); From a2faf9d395b0c70a61900442cdcb576089ccdf10 Mon Sep 17 00:00:00 2001 From: caiconghui Date: Mon, 9 Nov 2020 23:33:14 +0800 Subject: [PATCH 40/50] fix --- .../org/apache/doris/catalog/Catalog.java | 27 +++---------------- 1 file changed, 3 insertions(+), 24 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index f02bf5c2d3aaf9..bf1c6bc331167f 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -3414,9 +3414,6 @@ public void dropPartition(Database db, OlapTable olapTable, DropPartitionClause public void replayDropPartition(DropPartitionInfo info) { Database db = this.getDb(info.getDbId()); OlapTable olapTable = (OlapTable) db.getTable(info.getTableId()); - if (olapTable == null) { - return; - } olapTable.writeLock(); try { if (info.isTempPartition()) { @@ -4301,9 +4298,6 @@ public void replayCreateTable(String dbName, Table table) { public void replayAlterExteranlTableSchema(String dbName, String tableName, List newSchema) throws DdlException { Database db = this.fullNameToDb.get(dbName); Table table = db.getTable(tableName); - if (table == null) { - throw new DdlException("Do not contain proper table " + tableName + " in refresh table"); - } table.writeLock(); try { table.setNewFullSchema(newSchema); @@ -4474,18 +4468,12 @@ public boolean unprotectDropTable(Database db, Table table, boolean isForceDrop) public void replayDropTable(Database db, long tableId, boolean isForceDrop) { Table table = db.getTable(tableId); // delete from db meta - if (table == null) { - return; - } db.writeLock(); + table.writeLock(); try { - table.writeLock(); - try { - unprotectDropTable(db, table, isForceDrop); - } finally { - table.writeUnlock(); - } + unprotectDropTable(db, table, isForceDrop); } finally { + table.writeUnlock(); db.writeUnlock(); } } @@ -4545,9 +4533,6 @@ private void unprotectUpdateReplica(ReplicaPersistInfo info) { public void replayAddReplica(ReplicaPersistInfo info) { Database db = getDb(info.getDbId()); OlapTable olapTable = (OlapTable) db.getTable(info.getTableId()); - if (olapTable == null) { - return; - } olapTable.writeLock(); try { unprotectAddReplica(info); @@ -4559,9 +4544,6 @@ public void replayAddReplica(ReplicaPersistInfo info) { public void replayUpdateReplica(ReplicaPersistInfo info) { Database db = getDb(info.getDbId()); OlapTable olapTable = (OlapTable) db.getTable(info.getTableId()); - if (olapTable == null) { - return; - } olapTable.writeLock(); try { unprotectUpdateReplica(info); @@ -4582,9 +4564,6 @@ public void unprotectDeleteReplica(ReplicaPersistInfo info) { public void replayDeleteReplica(ReplicaPersistInfo info) { Database db = getDb(info.getDbId()); OlapTable tbl = (OlapTable) db.getTable(info.getTableId()); - if (tbl == null) { - return; - } tbl.writeLock(); try { unprotectDeleteReplica(info); From 26f39d1f923bf52f25ec15c09ea5775ad4590524 Mon Sep 17 00:00:00 2001 From: caiconghui Date: Tue, 10 Nov 2020 00:17:48 +0800 Subject: [PATCH 41/50] fix --- .../src/main/java/org/apache/doris/alter/AlterJobV2.java | 2 +- .../main/java/org/apache/doris/alter/SchemaChangeJob.java | 6 ------ 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java index 134df36bd59fd9..f0bc947ad79bbf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java @@ -178,7 +178,7 @@ public synchronized void run() { } } - public final boolean cancel(String errMsg) { + public synchronized final boolean cancel(String errMsg) { return cancelImpl(errMsg); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java index e591f878f3b27a..c2ba17fdfe894b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java @@ -920,9 +920,6 @@ public synchronized void clear() { @Override public void replayInitJob(Database db) { OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - return; - } olapTable.writeLock(); try { // change the state of table/partition and replica, then add object to related List and Set @@ -965,9 +962,6 @@ public void replayInitJob(Database db) { @Override public void replayFinishing(Database db) { OlapTable olapTable = (OlapTable) db.getTable(tableId); - if (olapTable == null) { - return; - } olapTable.writeLock(); try { // set the status to normal From e518f1e1cd4a5b05298cd5ff0f15f0feffc4ff33 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Tue, 10 Nov 2020 19:30:18 +0800 Subject: [PATCH 42/50] Fix dead lock bug --- fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java | 2 +- .../main/java/org/apache/doris/service/FrontendServiceImpl.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index bf1c6bc331167f..3ed58f0f336619 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -4747,7 +4747,7 @@ public HashMap getPartitionIdToStorageMediumMap() { // use try lock to avoid blocking a long time. // if block too long, backend report rpc will timeout. if (!olapTable.tryWriteLock(Table.TRY_LOCK_TIMEOUT_MS, TimeUnit.MILLISECONDS)) { - LOG.warn("try get db {} writelock but failed when checking backend storage medium", dbId); + LOG.warn("try get table {} writelock but failed when checking backend storage medium", table.getName()); continue; } Preconditions.checkState(olapTable.isWriteLockHeldByCurrentThread()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index cf9bd9de8df58d..739c7620fbbe6e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -922,7 +922,6 @@ private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request) } long timeoutMs = request.isSetThriftRpcTimeoutMs() ? request.getThriftRpcTimeoutMs() : 5000; Table table = db.getTableOrThrowException(request.getTbl(), TableType.OLAP); - table.readLock(); if (!table.tryReadLock(timeoutMs, TimeUnit.MILLISECONDS)) { throw new UserException("get table read lock timeout, database=" + fullDbName + ",table=" + table.getName()); } From 7563bfbc0b0bc4f684071050f8ef3ad834fad909 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Thu, 19 Nov 2020 17:32:08 +0800 Subject: [PATCH 43/50] fix unit test failed for SelectStmt --- .../src/main/java/org/apache/doris/analysis/WithClause.java | 1 + 1 file changed, 1 insertion(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/WithClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/WithClause.java index 064021d27e9f45..6ee877578d8a03 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/WithClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/WithClause.java @@ -109,6 +109,7 @@ public void reset() { public void getTables(Analyzer analyzer, Map tableMap, Set parentViewNameSet) throws AnalysisException { for (View view : views_) { QueryStmt stmt = view.getQueryStmt(); + parentViewNameSet.add(view.getName()); stmt.getTables(analyzer, tableMap, parentViewNameSet); } } From 7dfc2f8612383ac395cd8b8d0665aa935f74c330 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Fri, 27 Nov 2020 19:51:40 +0800 Subject: [PATCH 44/50] Fix by review --- .../java/org/apache/doris/alter/Alter.java | 75 ++---- .../doris/alter/MaterializedViewHandler.java | 59 ++--- .../doris/alter/SchemaChangeHandler.java | 225 +++++++++--------- .../org/apache/doris/backup/RestoreJob.java | 7 +- .../org/apache/doris/catalog/Catalog.java | 176 +++++++------- .../org/apache/doris/catalog/Database.java | 13 + .../org/apache/doris/load/AsyncDeleteJob.java | 2 +- .../main/java/org/apache/doris/load/Load.java | 57 +---- .../apache/doris/master/ReportHandler.java | 16 -- .../org/apache/doris/qe/StmtExecutor.java | 1 + .../doris/task/HadoopLoadPendingTask.java | 44 ++-- .../apache/doris/alter/RollupJobV2Test.java | 28 +-- .../doris/alter/SchemaChangeJobV2Test.java | 70 +----- 13 files changed, 322 insertions(+), 451 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index 2c63d6075c1dfe..0cb6ab58d17577 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -105,15 +105,8 @@ public void processCreateMaterializedView(CreateMaterializedViewStmt stmt) db.checkQuota(); OlapTable olapTable = (OlapTable) db.getTableOrThrowException(tableName, TableType.OLAP); - - olapTable.writeLock(); - try { - olapTable.checkStableAndNormal(db.getClusterName()); - ((MaterializedViewHandler)materializedViewHandler).processCreateMaterializedView(stmt, db, + ((MaterializedViewHandler)materializedViewHandler).processCreateMaterializedView(stmt, db, olapTable); - } finally { - olapTable.writeUnlock(); - } } public void processDropMaterializedView(DropMaterializedViewStmt stmt) throws DdlException, MetaNotFoundException { @@ -153,19 +146,9 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable, boolean needProcessOutsideTableLock = false; if (currentAlterOps.hasSchemaChangeOp()) { // if modify storage type to v2, do schema change to convert all related tablets to segment v2 format - olapTable.writeLock(); - try { - schemaChangeHandler.process(alterClauses, clusterName, db, olapTable); - } finally { - olapTable.writeUnlock(); - } + schemaChangeHandler.process(alterClauses, clusterName, db, olapTable); } else if (currentAlterOps.hasRollupOp()) { - olapTable.writeLock(); - try { - materializedViewHandler.process(alterClauses, clusterName, db, olapTable); - } finally { - olapTable.writeUnlock(); - } + materializedViewHandler.process(alterClauses, clusterName, db, olapTable); } else if (currentAlterOps.hasPartitionOp()) { Preconditions.checkState(alterClauses.size() == 1); AlterClause alterClause = alterClauses.get(0); @@ -226,12 +209,7 @@ private void processAlterExternalTable(AlterTableStmt stmt, Table externalTable, if (currentAlterOps.hasRenameOp()) { processRename(db, externalTable, alterClauses); } else if (currentAlterOps.hasSchemaChangeOp()) { - externalTable.writeLock(); - try { - schemaChangeHandler.processExternalTable(alterClauses, db, externalTable); - } finally { - externalTable.writeUnlock(); - } + schemaChangeHandler.processExternalTable(alterClauses, db, externalTable); } } @@ -464,32 +442,20 @@ public void processAlterCluster(AlterSystemStmt stmt) throws UserException { private void processRename(Database db, OlapTable table, List alterClauses) throws DdlException { for (AlterClause alterClause : alterClauses) { if (alterClause instanceof TableRenameClause) { - db.writeLock(); - table.writeLock(); - try { - Catalog.getCurrentCatalog().renameTable(db, table, (TableRenameClause) alterClause); - } finally { - table.writeUnlock(); - db.writeUnlock(); - } + Catalog.getCurrentCatalog().renameTable(db, table, (TableRenameClause) alterClause); break; } else { - table.writeLock(); - try { - if (alterClause instanceof RollupRenameClause) { - Catalog.getCurrentCatalog().renameRollup(db, table, (RollupRenameClause) alterClause); - break; - } else if (alterClause instanceof PartitionRenameClause) { - Catalog.getCurrentCatalog().renamePartition(db, table, (PartitionRenameClause) alterClause); - break; - } else if (alterClause instanceof ColumnRenameClause) { - Catalog.getCurrentCatalog().renameColumn(db, table, (ColumnRenameClause) alterClause); - break; - } else { - Preconditions.checkState(false); - } - } finally { - table.writeUnlock(); + if (alterClause instanceof RollupRenameClause) { + Catalog.getCurrentCatalog().renameRollup(db, table, (RollupRenameClause) alterClause); + break; + } else if (alterClause instanceof PartitionRenameClause) { + Catalog.getCurrentCatalog().renamePartition(db, table, (PartitionRenameClause) alterClause); + break; + } else if (alterClause instanceof ColumnRenameClause) { + Catalog.getCurrentCatalog().renameColumn(db, table, (ColumnRenameClause) alterClause); + break; + } else { + Preconditions.checkState(false); } } } @@ -498,14 +464,7 @@ private void processRename(Database db, OlapTable table, List alter private void processRename(Database db, Table table, List alterClauses) throws DdlException { for (AlterClause alterClause : alterClauses) { if (alterClause instanceof TableRenameClause) { - db.writeLock(); - table.writeLock(); - try { - Catalog.getCurrentCatalog().renameTable(db, table, (TableRenameClause) alterClause); - } finally { - table.writeUnlock(); - db.writeUnlock(); - } + Catalog.getCurrentCatalog().renameTable(db, table, (TableRenameClause) alterClause); break; } else { Preconditions.checkState(false); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java index 32781b9b62c6e6..2b5b8e81e6f5e1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java @@ -176,38 +176,43 @@ private boolean removeAlterJobV2FromTableNotFinalStateJobMap(AlterJobV2 alterJob */ public void processCreateMaterializedView(CreateMaterializedViewStmt addMVClause, Database db, OlapTable olapTable) throws DdlException, AnalysisException { + olapTable.writeLock(); + try { + olapTable.checkStableAndNormal(db.getClusterName()); + if (olapTable.existTempPartitions()) { + throw new DdlException("Can not alter table when there are temp partitions in table"); + } - if (olapTable.existTempPartitions()) { - throw new DdlException("Can not alter table when there are temp partitions in table"); - } - - // Step1.1: semantic analysis - // TODO(ML): support the materialized view as base index - if (!addMVClause.getBaseIndexName().equals(olapTable.getName())) { - throw new DdlException("The name of table in from clause must be same as the name of alter table"); - } - // Step1.2: base table validation - String baseIndexName = addMVClause.getBaseIndexName(); - String mvIndexName = addMVClause.getMVName(); - LOG.info("process add materialized view[{}] based on [{}]", mvIndexName, baseIndexName); + // Step1.1: semantic analysis + // TODO(ML): support the materialized view as base index + if (!addMVClause.getBaseIndexName().equals(olapTable.getName())) { + throw new DdlException("The name of table in from clause must be same as the name of alter table"); + } + // Step1.2: base table validation + String baseIndexName = addMVClause.getBaseIndexName(); + String mvIndexName = addMVClause.getMVName(); + LOG.info("process add materialized view[{}] based on [{}]", mvIndexName, baseIndexName); - // avoid conflict against with batch add rollup job - Preconditions.checkState(olapTable.getState() == OlapTableState.NORMAL); + // avoid conflict against with batch add rollup job + Preconditions.checkState(olapTable.getState() == OlapTableState.NORMAL); - long baseIndexId = checkAndGetBaseIndex(baseIndexName, olapTable); - // Step1.3: mv clause validation - List mvColumns = checkAndPrepareMaterializedView(addMVClause, olapTable); + long baseIndexId = checkAndGetBaseIndex(baseIndexName, olapTable); + // Step1.3: mv clause validation + List mvColumns = checkAndPrepareMaterializedView(addMVClause, olapTable); - // Step2: create mv job - RollupJobV2 rollupJobV2 = createMaterializedViewJob(mvIndexName, baseIndexName, mvColumns, addMVClause - .getProperties(), olapTable, db, baseIndexId, addMVClause.getMVKeysType(), addMVClause.getOrigStmt()); + // Step2: create mv job + RollupJobV2 rollupJobV2 = createMaterializedViewJob(mvIndexName, baseIndexName, mvColumns, addMVClause + .getProperties(), olapTable, db, baseIndexId, addMVClause.getMVKeysType(), addMVClause.getOrigStmt()); - addAlterJobV2(rollupJobV2); + addAlterJobV2(rollupJobV2); - olapTable.setState(OlapTableState.ROLLUP); + olapTable.setState(OlapTableState.ROLLUP); - Catalog.getCurrentCatalog().getEditLog().logAlterJob(rollupJobV2); - LOG.info("finished to create materialized view job: {}", rollupJobV2.getJobId()); + Catalog.getCurrentCatalog().getEditLog().logAlterJob(rollupJobV2); + LOG.info("finished to create materialized view job: {}", rollupJobV2.getJobId()); + } finally { + olapTable.writeUnlock(); + } } /** @@ -742,7 +747,6 @@ public void processBatchDropRollup(List dropRollupClauses, Database public void processDropMaterializedView(DropMaterializedViewStmt dropMaterializedViewStmt, Database db, OlapTable olapTable) throws DdlException, MetaNotFoundException { - Preconditions.checkState(olapTable.isWriteLockHeldByCurrentThread()); olapTable.writeLock(); try { // check table state @@ -766,6 +770,8 @@ public void processDropMaterializedView(DropMaterializedViewStmt dropMaterialize } else { throw e; } + } finally { + olapTable.writeUnlock(); } } @@ -1173,7 +1179,6 @@ private void getOldAlterJobInfos(Database db, List> rollupJobIn @Override public void process(List alterClauses, String clusterName, Database db, OlapTable olapTable) throws DdlException, AnalysisException, MetaNotFoundException { - Preconditions.checkState(olapTable.isWriteLockHeldByCurrentThread()); Optional alterClauseOptional = alterClauses.stream().findAny(); if (alterClauseOptional.isPresent()) { if (alterClauseOptional.get() instanceof AddRollupClause) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index ae72eb229c7d52..334f9b10348bb8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -1586,128 +1586,137 @@ private void getOldAlterJobInfos(Database db, List> schemaChang @Override public void process(List alterClauses, String clusterName, Database db, OlapTable olapTable) throws UserException { - Preconditions.checkState(olapTable.isWriteLockHeldByCurrentThread()); - // index id -> index schema - Map> indexSchemaMap = new HashMap<>(); - for (Map.Entry> entry : olapTable.getIndexIdToSchema(true).entrySet()) { - indexSchemaMap.put(entry.getKey(), new LinkedList<>(entry.getValue())); - } - List newIndexes = olapTable.getCopiedIndexes(); - Map propertyMap = new HashMap<>(); - for (AlterClause alterClause : alterClauses) { - Map properties = alterClause.getProperties(); - if (properties != null) { - if (propertyMap.isEmpty()) { - propertyMap.putAll(properties); - } else { - throw new DdlException("reduplicated PROPERTIES"); - } + olapTable.writeLock(); + try { + // index id -> index schema + Map> indexSchemaMap = new HashMap<>(); + for (Map.Entry> entry : olapTable.getIndexIdToSchema(true).entrySet()) { + indexSchemaMap.put(entry.getKey(), new LinkedList<>(entry.getValue())); + } + List newIndexes = olapTable.getCopiedIndexes(); + Map propertyMap = new HashMap<>(); + for (AlterClause alterClause : alterClauses) { + Map properties = alterClause.getProperties(); + if (properties != null) { + if (propertyMap.isEmpty()) { + propertyMap.putAll(properties); + } else { + 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.getCurrentCatalog().modifyTableColocate(db, olapTable, colocateGroup, false, null); - return; - } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_DISTRIBUTION_TYPE)) { - Catalog.getCurrentCatalog().convertDistributionType(db, olapTable); - return; - } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_SEND_CLEAR_ALTER_TASK)) { + // 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.getCurrentCatalog().modifyTableColocate(db, olapTable, colocateGroup, false, null); + return; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_DISTRIBUTION_TYPE)) { + Catalog.getCurrentCatalog().convertDistributionType(db, olapTable); + return; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_SEND_CLEAR_ALTER_TASK)) { /* * This is only for fixing bug when upgrading Doris from 0.9.x to 0.10.x. */ - sendClearAlterTask(db, olapTable); - return; - } else if (DynamicPartitionUtil.checkDynamicPartitionPropertiesExist(properties)) { - if (!olapTable.dynamicPartitionExists()) { - try { - DynamicPartitionUtil.checkInputDynamicPartitionProperties(properties, olapTable.getPartitionInfo()); - } catch (DdlException e) { - // This table is not a dynamic partition table and didn't supply all dynamic partition properties - throw new DdlException("Table " + db.getFullName() + "." + - olapTable.getName() + " is not a dynamic partition table. Use command `HELP ALTER TABLE` " + - "to see how to change a normal table to a dynamic partition table."); + sendClearAlterTask(db, olapTable); + return; + } else if (DynamicPartitionUtil.checkDynamicPartitionPropertiesExist(properties)) { + if (!olapTable.dynamicPartitionExists()) { + try { + DynamicPartitionUtil.checkInputDynamicPartitionProperties(properties, olapTable.getPartitionInfo()); + } catch (DdlException e) { + // This table is not a dynamic partition table and didn't supply all dynamic partition properties + throw new DdlException("Table " + db.getFullName() + "." + + olapTable.getName() + " is not a dynamic partition table. Use command `HELP ALTER TABLE` " + + "to see how to change a normal table to a dynamic partition table."); + } } + Catalog.getCurrentCatalog().modifyTableDynamicPartition(db, olapTable, properties); + return; + } else if (properties.containsKey("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)) { + Preconditions.checkNotNull(properties.get(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)); + Catalog.getCurrentCatalog().modifyTableDefaultReplicationNum(db, olapTable, properties); + return; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)) { + Catalog.getCurrentCatalog().modifyTableReplicationNum(db, olapTable, properties); + return; } - Catalog.getCurrentCatalog().modifyTableDynamicPartition(db, olapTable, properties); - return; - } else if (properties.containsKey("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)) { - Preconditions.checkNotNull(properties.get(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)); - Catalog.getCurrentCatalog().modifyTableDefaultReplicationNum(db, olapTable, properties); - return; - } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)) { - Catalog.getCurrentCatalog().modifyTableReplicationNum(db, olapTable, properties); - return; } - } - // the following operations can not be done when there are temp partitions exist. - if (olapTable.existTempPartitions()) { - throw new DdlException("Can not alter table when there are temp partitions in table"); - } - - if (alterClause instanceof AddColumnClause) { - // add column - processAddColumn((AddColumnClause) alterClause, olapTable, indexSchemaMap); - } else if (alterClause instanceof AddColumnsClause) { - // add columns - processAddColumns((AddColumnsClause) alterClause, olapTable, indexSchemaMap); - } else if (alterClause instanceof DropColumnClause) { - // drop column and drop indexes on this column - processDropColumn((DropColumnClause) alterClause, olapTable, indexSchemaMap, newIndexes); - } else if (alterClause instanceof ModifyColumnClause) { - // modify column - processModifyColumn((ModifyColumnClause) alterClause, olapTable, indexSchemaMap); - } else if (alterClause instanceof ReorderColumnsClause) { - // reorder column - processReorderColumn((ReorderColumnsClause) alterClause, olapTable, indexSchemaMap); - } else if (alterClause instanceof ModifyTablePropertiesClause) { - // modify table properties - // do nothing, properties are already in propertyMap - } else if (alterClause instanceof CreateIndexClause) { - processAddIndex((CreateIndexClause) alterClause, olapTable, newIndexes); - } else if (alterClause instanceof DropIndexClause) { - processDropIndex((DropIndexClause) alterClause, olapTable, newIndexes); - } else { - Preconditions.checkState(false); - } - } // end for alter clauses + // the following operations can not be done when there are temp partitions exist. + if (olapTable.existTempPartitions()) { + throw new DdlException("Can not alter table when there are temp partitions in table"); + } - createJob(db.getId(), olapTable, indexSchemaMap, propertyMap, newIndexes); + if (alterClause instanceof AddColumnClause) { + // add column + processAddColumn((AddColumnClause) alterClause, olapTable, indexSchemaMap); + } else if (alterClause instanceof AddColumnsClause) { + // add columns + processAddColumns((AddColumnsClause) alterClause, olapTable, indexSchemaMap); + } else if (alterClause instanceof DropColumnClause) { + // drop column and drop indexes on this column + processDropColumn((DropColumnClause) alterClause, olapTable, indexSchemaMap, newIndexes); + } else if (alterClause instanceof ModifyColumnClause) { + // modify column + processModifyColumn((ModifyColumnClause) alterClause, olapTable, indexSchemaMap); + } else if (alterClause instanceof ReorderColumnsClause) { + // reorder column + processReorderColumn((ReorderColumnsClause) alterClause, olapTable, indexSchemaMap); + } else if (alterClause instanceof ModifyTablePropertiesClause) { + // modify table properties + // do nothing, properties are already in propertyMap + } else if (alterClause instanceof CreateIndexClause) { + processAddIndex((CreateIndexClause) alterClause, olapTable, newIndexes); + } else if (alterClause instanceof DropIndexClause) { + processDropIndex((DropIndexClause) alterClause, olapTable, newIndexes); + } else { + Preconditions.checkState(false); + } + } // end for alter clauses + + createJob(db.getId(), olapTable, indexSchemaMap, propertyMap, newIndexes); + } finally { + olapTable.writeUnlock(); + } } @Override public void processExternalTable(List alterClauses, Database db, Table externalTable) throws UserException { - // copy the external table schema columns - List newSchema = Lists.newArrayList(); - newSchema.addAll(externalTable.getBaseSchema(true)); - - for (AlterClause alterClause : alterClauses) { - if (alterClause instanceof AddColumnClause) { - // add column - processAddColumn((AddColumnClause) alterClause, externalTable, newSchema); - } else if (alterClause instanceof AddColumnsClause) { - // add columns - processAddColumns((AddColumnsClause) alterClause, externalTable, newSchema); - } else if (alterClause instanceof DropColumnClause) { - // drop column and drop indexes on this column - processDropColumn((DropColumnClause) alterClause, externalTable, newSchema); - } else if (alterClause instanceof ModifyColumnClause) { - // modify column - processModifyColumn((ModifyColumnClause) alterClause, externalTable, newSchema); - } else if (alterClause instanceof ReorderColumnsClause) { - // reorder column - processReorderColumn((ReorderColumnsClause) alterClause, externalTable, newSchema); - } else { - Preconditions.checkState(false); - } - } // end for alter clauses - // replace the old column list - externalTable.setNewFullSchema(newSchema); - // refresh external table column in edit log - Catalog.getCurrentCatalog().refreshExternalTableSchema(db, externalTable, newSchema); + externalTable.writeLock(); + try { + // copy the external table schema columns + List newSchema = Lists.newArrayList(); + newSchema.addAll(externalTable.getBaseSchema(true)); + + for (AlterClause alterClause : alterClauses) { + if (alterClause instanceof AddColumnClause) { + // add column + processAddColumn((AddColumnClause) alterClause, externalTable, newSchema); + } else if (alterClause instanceof AddColumnsClause) { + // add columns + processAddColumns((AddColumnsClause) alterClause, externalTable, newSchema); + } else if (alterClause instanceof DropColumnClause) { + // drop column and drop indexes on this column + processDropColumn((DropColumnClause) alterClause, externalTable, newSchema); + } else if (alterClause instanceof ModifyColumnClause) { + // modify column + processModifyColumn((ModifyColumnClause) alterClause, externalTable, newSchema); + } else if (alterClause instanceof ReorderColumnsClause) { + // reorder column + processReorderColumn((ReorderColumnsClause) alterClause, externalTable, newSchema); + } else { + Preconditions.checkState(false); + } + } // end for alter clauses + // replace the old column list + externalTable.setNewFullSchema(newSchema); + // refresh external table column in edit log + Catalog.getCurrentCatalog().refreshExternalTableSchema(db, externalTable, newSchema); + } finally { + externalTable.writeUnlock(); + } } private void sendClearAlterTask(Database db, OlapTable olapTable) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java index 8a562d24aa2cb0..fa3e7f4cd10f03 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -1392,12 +1392,7 @@ public void cancelInternal(boolean isReplay) { } finally { restoreTbl.writeUnlock(); } - db.writeLock(); - try { - db.dropTable(restoreTbl.getName()); - } finally { - db.writeUnlock(); - } + db.dropTableWithLock(restoreTbl.getName()); } // remove restored partitions diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index 3ed58f0f336619..ae87e89119acf7 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -5078,39 +5078,44 @@ public void cancelBackup(CancelBackupStmt stmt) throws DdlException { // entry of rename table operation public void renameTable(Database db, Table table, TableRenameClause tableRenameClause) throws DdlException { - Preconditions.checkState(db.isWriteLockHeldByCurrentThread()); - Preconditions.checkState(table.isWriteLockHeldByCurrentThread()); - if (table instanceof OlapTable) { - OlapTable olapTable = (OlapTable) table; - if ( olapTable.getState() != OlapTableState.NORMAL) { - throw new DdlException("Table[" + olapTable.getName() + "] is under " + olapTable.getState()); + db.writeLock(); + table.writeLock(); + try { + if (table instanceof OlapTable) { + OlapTable olapTable = (OlapTable) table; + if ( olapTable.getState() != OlapTableState.NORMAL) { + throw new DdlException("Table[" + olapTable.getName() + "] is under " + olapTable.getState()); + } } - } - String oldTableName = table.getName(); - String newTableName = tableRenameClause.getNewTableName(); - if (oldTableName.equals(newTableName)) { - throw new DdlException("Same table name"); - } + String oldTableName = table.getName(); + String newTableName = tableRenameClause.getNewTableName(); + if (oldTableName.equals(newTableName)) { + throw new DdlException("Same table name"); + } - // check if name is already used - if (db.getTable(newTableName) != null) { - throw new DdlException("Table name[" + newTableName + "] is already used"); - } + // check if name is already used + if (db.getTable(newTableName) != null) { + throw new DdlException("Table name[" + newTableName + "] is already used"); + } - if (table.getType() == TableType.OLAP) { - // olap table should also check if any rollup has same name as "newTableName" - ((OlapTable) table).checkAndSetName(newTableName, false); - } else { - table.setName(newTableName); - } + if (table.getType() == TableType.OLAP) { + // olap table should also check if any rollup has same name as "newTableName" + ((OlapTable) table).checkAndSetName(newTableName, false); + } else { + table.setName(newTableName); + } - db.dropTable(oldTableName); - db.createTable(table); + db.dropTable(oldTableName); + db.createTable(table); - TableInfo tableInfo = TableInfo.createForTableRename(db.getId(), table.getId(), newTableName); - editLog.logTableRename(tableInfo); - LOG.info("rename table[{}] to {}", oldTableName, newTableName); + TableInfo tableInfo = TableInfo.createForTableRename(db.getId(), table.getId(), newTableName); + editLog.logTableRename(tableInfo); + LOG.info("rename table[{}] to {}", oldTableName, newTableName); + } finally { + table.writeUnlock(); + db.writeUnlock(); + } } public void refreshExternalTableSchema(Database db, Table table, List newSchema) { @@ -5242,38 +5247,43 @@ public void replayModifyTableColocate(TablePropertyInfo info) { } public void renameRollup(Database db, OlapTable table, RollupRenameClause renameClause) throws DdlException { - if (table.getState() != OlapTableState.NORMAL) { - throw new DdlException("Table[" + table.getName() + "] is under " + table.getState()); - } + table.writeLock(); + try { + if (table.getState() != OlapTableState.NORMAL) { + throw new DdlException("Table[" + table.getName() + "] is under " + table.getState()); + } - String rollupName = renameClause.getRollupName(); - // check if it is base table name - if (rollupName.equals(table.getName())) { - throw new DdlException("Using ALTER TABLE RENAME to change table name"); - } + String rollupName = renameClause.getRollupName(); + // check if it is base table name + if (rollupName.equals(table.getName())) { + throw new DdlException("Using ALTER TABLE RENAME to change table name"); + } - String newRollupName = renameClause.getNewRollupName(); - if (rollupName.equals(newRollupName)) { - throw new DdlException("Same rollup name"); - } + String newRollupName = renameClause.getNewRollupName(); + if (rollupName.equals(newRollupName)) { + throw new DdlException("Same rollup name"); + } - Map indexNameToIdMap = table.getIndexNameToId(); - if (indexNameToIdMap.get(rollupName) == null) { - throw new DdlException("Rollup index[" + rollupName + "] does not exists"); - } + Map indexNameToIdMap = table.getIndexNameToId(); + if (indexNameToIdMap.get(rollupName) == null) { + throw new DdlException("Rollup index[" + rollupName + "] does not exists"); + } - // check if name is already used - if (indexNameToIdMap.get(newRollupName) != null) { - throw new DdlException("Rollup name[" + newRollupName + "] is already used"); - } + // check if name is already used + if (indexNameToIdMap.get(newRollupName) != null) { + throw new DdlException("Rollup name[" + newRollupName + "] is already used"); + } - long indexId = indexNameToIdMap.remove(rollupName); - indexNameToIdMap.put(newRollupName, indexId); + long indexId = indexNameToIdMap.remove(rollupName); + indexNameToIdMap.put(newRollupName, indexId); - // log - TableInfo tableInfo = TableInfo.createForRollupRename(db.getId(), table.getId(), indexId, newRollupName); - editLog.logRollupRename(tableInfo); - LOG.info("rename rollup[{}] to {}", rollupName, newRollupName); + // log + TableInfo tableInfo = TableInfo.createForRollupRename(db.getId(), table.getId(), indexId, newRollupName); + editLog.logRollupRename(tableInfo); + LOG.info("rename rollup[{}] to {}", rollupName, newRollupName); + } finally { + table.writeUnlock(); + } } public void replayRenameRollup(TableInfo tableInfo) throws DdlException { @@ -5298,38 +5308,43 @@ public void replayRenameRollup(TableInfo tableInfo) throws DdlException { } public void renamePartition(Database db, OlapTable table, PartitionRenameClause renameClause) throws DdlException { - if (table.getState() != OlapTableState.NORMAL) { - throw new DdlException("Table[" + table.getName() + "] is under " + table.getState()); - } + table.writeLock(); + try { + if (table.getState() != OlapTableState.NORMAL) { + throw new DdlException("Table[" + table.getName() + "] is under " + table.getState()); + } - if (table.getPartitionInfo().getType() != PartitionType.RANGE) { - throw new DdlException("Table[" + table.getName() + "] is single partitioned. " - + "no need to rename partition name."); - } + if (table.getPartitionInfo().getType() != PartitionType.RANGE) { + throw new DdlException("Table[" + table.getName() + "] is single partitioned. " + + "no need to rename partition name."); + } - String partitionName = renameClause.getPartitionName(); - String newPartitionName = renameClause.getNewPartitionName(); - if (partitionName.equalsIgnoreCase(newPartitionName)) { - throw new DdlException("Same partition name"); - } + String partitionName = renameClause.getPartitionName(); + String newPartitionName = renameClause.getNewPartitionName(); + if (partitionName.equalsIgnoreCase(newPartitionName)) { + throw new DdlException("Same partition name"); + } - Partition partition = table.getPartition(partitionName); - if (partition == null) { - throw new DdlException("Partition[" + partitionName + "] does not exists"); - } + Partition partition = table.getPartition(partitionName); + if (partition == null) { + throw new DdlException("Partition[" + partitionName + "] does not exists"); + } - // check if name is already used - if (table.checkPartitionNameExist(newPartitionName)) { - throw new DdlException("Partition name[" + newPartitionName + "] is already used"); - } + // check if name is already used + if (table.checkPartitionNameExist(newPartitionName)) { + throw new DdlException("Partition name[" + newPartitionName + "] is already used"); + } - table.renamePartition(partitionName, newPartitionName); + table.renamePartition(partitionName, newPartitionName); - // log - TableInfo tableInfo = TableInfo.createForPartitionRename(db.getId(), table.getId(), partition.getId(), - newPartitionName); - editLog.logPartitionRename(tableInfo); - LOG.info("rename partition[{}] to {}", partitionName, newPartitionName); + // log + TableInfo tableInfo = TableInfo.createForPartitionRename(db.getId(), table.getId(), partition.getId(), + newPartitionName); + editLog.logPartitionRename(tableInfo); + LOG.info("rename partition[{}] to {}", partitionName, newPartitionName); + } finally { + table.writeUnlock(); + } } public void replayRenamePartition(TableInfo tableInfo) throws DdlException { @@ -6691,9 +6706,6 @@ public void convertDistributionType(Database db, OlapTable tbl) throws DdlExcept public void replayConvertDistributionType(TableInfo tableInfo) { Database db = getDb(tableInfo.getDbId()); OlapTable tbl = (OlapTable) db.getTable(tableInfo.getTableId()); - if (tbl == null) { - return; - } tbl.writeLock(); try { tbl.convertRandomDistributionToHashDistribution(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 5b8b49cfa446b2..85da3bb2ff4d61 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -319,6 +319,19 @@ public boolean createTable(Table table) { return result; } + public void dropTableWithLock(String tableName) { + writeLock(); + try { + Table table = this.nameToTable.get(tableName); + if (table != null) { + this.nameToTable.remove(tableName); + this.idToTable.remove(table.getId()); + } + } finally { + writeUnlock(); + } + } + public void dropTable(String tableName) { Table table = this.nameToTable.get(tableName); if (table != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/AsyncDeleteJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/AsyncDeleteJob.java index baa8f0ab0f211a..ede4c8523aad72 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/AsyncDeleteJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/AsyncDeleteJob.java @@ -50,7 +50,7 @@ public enum DeleteState { FINISHED } - private DeleteState state; + private volatile DeleteState state; private long jobId; private long dbId; diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java index 4446f9597821ec..73ad11d502c663 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java @@ -1930,31 +1930,6 @@ public Map> getDbToDeleteInfos() { return dbToDeleteInfos; } - public Set getTxnIdsByDb(Long dbId) { - Set txnIds = Sets.newHashSet(); - readLock(); - try { - List jobs = dbToLoadJobs.get(dbId); - if (jobs != null) { - for (LoadJob loadJob : jobs) { - txnIds.add(loadJob.getTransactionId()); - } - } - } finally { - readUnlock(); - } - return txnIds; - } - - public List getDbLoadJobs(long dbId) { - readLock(); - try { - return dbToLoadJobs.get(dbId); - } finally { - readUnlock(); - } - } - public List getLoadJobs(JobState jobState) { List jobs = new ArrayList(); Collection stateJobs = null; @@ -3287,19 +3262,14 @@ public void unprotectDelete(DeleteInfo deleteInfo, Database db) { public void replayFinishAsyncDeleteJob(AsyncDeleteJob deleteJob, Catalog catalog) { Database db = catalog.getDb(deleteJob.getDbId()); - writeLock(); + OlapTable table = (OlapTable) db.getTable(deleteJob.getTableId()); + table.writeLock(); + readLock(); try { // Update database information Map replicaInfos = deleteJob.getReplicaPersistInfos(); if (replicaInfos != null) { for (ReplicaPersistInfo info : replicaInfos.values()) { - OlapTable table = (OlapTable) db.getTable(info.getTableId()); - if (table == null) { - LOG.warn("the table[{}] is missing", info.getIndexId()); - continue; - } - table.writeLock(); - try { Partition partition = table.getPartition(info.getPartitionId()); if (partition == null) { LOG.warn("the partition[{}] is missing", info.getIndexId()); @@ -3323,14 +3293,11 @@ public void replayFinishAsyncDeleteJob(AsyncDeleteJob deleteJob, Catalog catalog } replica.updateVersionInfo(info.getVersion(), info.getVersionHash(), info.getDataSize(), info.getRowCount()); - - } finally { - table.writeUnlock(); - } } } } finally { - writeUnlock(); + readUnlock(); + table.writeUnlock(); } removeDeleteJobAndSetState(deleteJob); LOG.info("unprotected finish asyncDeleteJob: {}", deleteJob.getJobId()); @@ -3342,16 +3309,13 @@ public void replayDelete(DeleteInfo deleteInfo, Catalog catalog) { if (table == null) { return; } + table.writeLock(); writeLock(); try { - table.writeLock(); - try { - unprotectDelete(deleteInfo, db); - } finally { - table.writeUnlock(); - } + unprotectDelete(deleteInfo, db); } finally { writeUnlock(); + table.writeUnlock(); } } @@ -3516,11 +3480,6 @@ private void checkHasRunningAsyncDeleteJob(long partitionId, String partitionNam } } - public void checkHashRunningDeleteJob(long partitionId, String partitionName) throws DdlException { - checkHasRunningSyncDeleteJob(partitionId, partitionName); - checkHasRunningAsyncDeleteJob(partitionId, partitionName); - } - public void delete(DeleteStmt stmt) throws DdlException { String dbName = stmt.getDbName(); String tableName = stmt.getTableName(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java index 27717f27f88233..129f32f20fc64e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java @@ -14,22 +14,6 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -// 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.master; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index 61b50505692017..0a991effd77a0d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -468,6 +468,7 @@ public void analyze(TQueryOptions tQueryOptions) throws UserException { } insertStmt.getTables(analyzer, tableMap, parentViewNameSet); } + // table id in tableList is in ascending order because that table map is a sorted map List
tables = Lists.newArrayList(tableMap.values()); MetaLockUtils.readLockTables(tables); try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java index ebbfbce002e3fa..09140e960954d6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java @@ -136,32 +136,36 @@ private Map createEtlPartitions() throws LoadException if (table == null) { throw new LoadException("table does not exist. id: " + tableId); } + table.readLock(); + try { + // columns + Map etlColumns = createEtlColumns(table); - // columns - Map etlColumns = createEtlColumns(table); - - // partitions - Map idToPartitionLoadInfo = tableLoadInfo.getIdToPartitionLoadInfo(); - for (Entry partitionEntry : idToPartitionLoadInfo.entrySet()) { - long partitionId = partitionEntry.getKey(); - PartitionLoadInfo partitionLoadInfo = partitionEntry.getValue(); + // partitions + Map idToPartitionLoadInfo = tableLoadInfo.getIdToPartitionLoadInfo(); + for (Entry partitionEntry : idToPartitionLoadInfo.entrySet()) { + long partitionId = partitionEntry.getKey(); + PartitionLoadInfo partitionLoadInfo = partitionEntry.getValue(); - EtlPartitionConf etlPartitionConf = new EtlPartitionConf(); - // columns - etlPartitionConf.setColumns(etlColumns); + EtlPartitionConf etlPartitionConf = new EtlPartitionConf(); + // columns + etlPartitionConf.setColumns(etlColumns); - // indices (views) - Map etlIndices = createEtlIndices(table, partitionId); - Preconditions.checkNotNull(etlIndices); - etlPartitionConf.setIndices(etlIndices); + // indices (views) + Map etlIndices = createEtlIndices(table, partitionId); + Preconditions.checkNotNull(etlIndices); + etlPartitionConf.setIndices(etlIndices); - // source file schema - etlPartitionConf.setSources(createSources(partitionLoadInfo)); + // source file schema + etlPartitionConf.setSources(createSources(partitionLoadInfo)); - // partition info - etlPartitionConf.setPartitionInfo(createPartitionInfo(table, partitionId)); + // partition info + etlPartitionConf.setPartitionInfo(createPartitionInfo(table, partitionId)); - etlPartitions.put(String.valueOf(partitionId), etlPartitionConf); + etlPartitions.put(String.valueOf(partitionId), etlPartitionConf); + } + } finally { + table.readUnlock(); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java index 5931f825448443..7293740e4fa85a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java +++ b/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java @@ -158,12 +158,7 @@ public void testRunRollupJobConcurrentLimit() throws UserException { alterClauses.add(clause2); Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); - olapTable.writeLock(); - try { - materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); - } finally { - olapTable.writeUnlock(); - } + materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); Map alterJobsV2 = materializedViewHandler.getAlterJobsV2(); materializedViewHandler.runAfterCatalogReady(); @@ -183,12 +178,7 @@ public void testAddSchemaChange() throws UserException { alterClauses.add(clause); Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); - olapTable.writeLock(); - try { - materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); - } finally { - olapTable.writeUnlock(); - } + materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); Map alterJobsV2 = materializedViewHandler.getAlterJobsV2(); Assert.assertEquals(1, alterJobsV2.size()); Assert.assertEquals(OlapTableState.ROLLUP, olapTable.getState()); @@ -208,12 +198,7 @@ public void testSchemaChange1() throws Exception { Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); - olapTable.writeLock(); - try { - materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); - } finally { - olapTable.writeUnlock(); - } + materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); Map alterJobsV2 = materializedViewHandler.getAlterJobsV2(); Assert.assertEquals(1, alterJobsV2.size()); RollupJobV2 rollupJob = (RollupJobV2) alterJobsV2.values().stream().findAny().get(); @@ -317,12 +302,7 @@ public void testSchemaChangeWhileTabletNotStable() throws Exception { Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); - olapTable.writeLock(); - try { - materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); - } finally { - olapTable.writeUnlock(); - } + materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable); Map alterJobsV2 = materializedViewHandler.getAlterJobsV2(); Assert.assertEquals(1, alterJobsV2.size()); RollupJobV2 rollupJob = (RollupJobV2) alterJobsV2.values().stream().findAny().get(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java index 1433b50590450e..d299de2b6ca9f9 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java +++ b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java @@ -137,12 +137,7 @@ public void testAddSchemaChange() throws UserException { alterClauses.add(addColumnClause); Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); - olapTable.writeLock(); - try { - schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); - } finally { - olapTable.writeUnlock(); - } + schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); Map alterJobsV2 = schemaChangeHandler.getAlterJobsV2(); Assert.assertEquals(1, alterJobsV2.size()); Assert.assertEquals(OlapTableState.SCHEMA_CHANGE, olapTable.getState()); @@ -162,12 +157,7 @@ public void testSchemaChange1() throws Exception { Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); - olapTable.writeLock(); - try { - schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); - } finally { - olapTable.writeUnlock(); - } + schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); Map alterJobsV2 = schemaChangeHandler.getAlterJobsV2(); Assert.assertEquals(1, alterJobsV2.size()); SchemaChangeJobV2 schemaChangeJob = (SchemaChangeJobV2) alterJobsV2.values().stream().findAny().get(); @@ -243,12 +233,7 @@ public void testSchemaChangeWhileTabletNotStable() throws Exception { Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); - olapTable.writeLock(); - try { - schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); - } finally { - olapTable.writeUnlock(); - } + schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); Map alterJobsV2 = schemaChangeHandler.getAlterJobsV2(); Assert.assertEquals(1, alterJobsV2.size()); SchemaChangeJobV2 schemaChangeJob = (SchemaChangeJobV2) alterJobsV2.values().stream().findAny().get(); @@ -333,12 +318,7 @@ public void testModifyDynamicPartitionNormal() throws UserException { alterClauses.add(new ModifyTablePropertiesClause(properties)); Database db = CatalogMocker.mockDb(); OlapTable olapTable = (OlapTable) db.getTable(CatalogMocker.TEST_TBL2_ID); - olapTable.writeLock(); - try { - schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); - } finally { - olapTable.writeUnlock(); - } + schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); Assert.assertTrue(olapTable.getTableProperty().getDynamicPartitionProperty().isExist()); Assert.assertTrue(olapTable.getTableProperty().getDynamicPartitionProperty().getEnable()); Assert.assertEquals("day", olapTable.getTableProperty().getDynamicPartitionProperty().getTimeUnit()); @@ -351,56 +331,31 @@ public void testModifyDynamicPartitionNormal() throws UserException { ArrayList tmpAlterClauses = new ArrayList<>(); properties.put(DynamicPartitionProperty.ENABLE, "false"); tmpAlterClauses.add(new ModifyTablePropertiesClause(properties)); - olapTable.writeLock(); - try { - schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); - } finally { - olapTable.writeUnlock(); - } + schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); Assert.assertFalse(olapTable.getTableProperty().getDynamicPartitionProperty().getEnable()); // set dynamic_partition.time_unit = week tmpAlterClauses = new ArrayList<>(); properties.put(DynamicPartitionProperty.TIME_UNIT, "week"); tmpAlterClauses.add(new ModifyTablePropertiesClause(properties)); - olapTable.writeLock(); - try { - schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); - } finally { - olapTable.writeUnlock(); - } + schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); Assert.assertEquals("week", olapTable.getTableProperty().getDynamicPartitionProperty().getTimeUnit()); // set dynamic_partition.end = 10 tmpAlterClauses = new ArrayList<>(); properties.put(DynamicPartitionProperty.END, "10"); tmpAlterClauses.add(new ModifyTablePropertiesClause(properties)); - olapTable.writeLock(); - try { - schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); - } finally { - olapTable.writeUnlock(); - } + schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); Assert.assertEquals(10, olapTable.getTableProperty().getDynamicPartitionProperty().getEnd()); // set dynamic_partition.prefix = p1 tmpAlterClauses = new ArrayList<>(); properties.put(DynamicPartitionProperty.PREFIX, "p1"); tmpAlterClauses.add(new ModifyTablePropertiesClause(properties)); - olapTable.writeLock(); - try { - schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); - } finally { - olapTable.writeUnlock(); - } + schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); Assert.assertEquals("p1", olapTable.getTableProperty().getDynamicPartitionProperty().getPrefix()); // set dynamic_partition.buckets = 3 tmpAlterClauses = new ArrayList<>(); properties.put(DynamicPartitionProperty.BUCKETS, "3"); tmpAlterClauses.add(new ModifyTablePropertiesClause(properties)); - olapTable.writeLock(); - try { - schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); - } finally { - olapTable.writeUnlock(); - } + schemaChangeHandler.process(tmpAlterClauses, "default_cluster", db, olapTable); Assert.assertEquals(3, olapTable.getTableProperty().getDynamicPartitionProperty().getBuckets()); } @@ -420,12 +375,7 @@ public void modifyDynamicPartitionWithoutTableProperty(String propertyKey, Strin expectedEx.expect(DdlException.class); expectedEx.expectMessage("errCode = 2, detailMessage = Table test_db.test_tbl2 is not a dynamic partition table. " + "Use command `HELP ALTER TABLE` to see how to change a normal table to a dynamic partition table."); - olapTable.writeLock(); - try { - schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); - } finally { - olapTable.writeUnlock(); - } + schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable); } @Test From 63195339ad1ba0ee1598eb3511b03744ae0d0564 Mon Sep 17 00:00:00 2001 From: caiconghui Date: Sun, 29 Nov 2020 16:20:39 +0800 Subject: [PATCH 45/50] Add table lock when update BackendReportVersion in SystemInfoService --- .../apache/doris/alter/SchemaChangeJob.java | 2 +- .../org/apache/doris/master/MasterImpl.java | 8 +++--- .../doris/system/SystemInfoService.java | 25 +++++++++++++++---- .../doris/cluster/SystemInfoServiceTest.java | 13 +++++++--- 4 files changed, 35 insertions(+), 13 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java index c2ba17fdfe894b..feb442b042a3e2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java @@ -638,7 +638,7 @@ public void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo, } Catalog.getCurrentSystemInfo().updateBackendReportVersion(schemaChangeTask.getBackendId(), - reportVersion, dbId); + reportVersion, dbId, tableId); setReplicaFinished(indexId, replicaId); LOG.info("finish schema change replica[{}]. index[{}]. tablet[{}], backend[{}]", diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java b/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java index c28581600d6efa..728089dce198ca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java @@ -265,7 +265,7 @@ private void finishCreateReplica(AgentTask task, TFinishTaskRequest request) { } // this should be called before 'countDownLatch()' - Catalog.getCurrentSystemInfo().updateBackendReportVersion(task.getBackendId(), request.getReportVersion(), task.getDbId()); + Catalog.getCurrentSystemInfo().updateBackendReportVersion(task.getBackendId(), request.getReportVersion(), task.getDbId(), task.getTableId()); createReplicaTask.countDownLatch(task.getBackendId(), task.getSignature()); LOG.debug("finish create replica. tablet id: {}, be: {}, report version: {}", @@ -358,7 +358,7 @@ private void finishRealtimePush(AgentTask task, TFinishTaskRequest request) { // should be done before addReplicaPersistInfos and countDownLatch long reportVersion = request.getReportVersion(); Catalog.getCurrentSystemInfo().updateBackendReportVersion(task.getBackendId(), reportVersion, - task.getDbId()); + task.getDbId(), task.getTableId()); List tabletIds = finishTabletInfos.stream().map( tTabletInfo -> tTabletInfo.getTabletId()).collect(Collectors.toList()); @@ -583,7 +583,7 @@ private void finishPush(AgentTask task, TFinishTaskRequest request) { // should be done before addReplicaPersistInfos and countDownLatch long reportVersion = request.getReportVersion(); Catalog.getCurrentSystemInfo().updateBackendReportVersion(task.getBackendId(), reportVersion, - task.getDbId()); + task.getDbId(), task.getTableId()); if (pushTask.getPushType() == TPushType.LOAD || pushTask.getPushType() == TPushType.LOAD_DELETE) { // handle load job @@ -648,7 +648,7 @@ private void finishPublishVersion(AgentTask task, TFinishTaskRequest request) { if (request.isSetReportVersion()) { // report version is required. here we check if set, for compatibility. long reportVersion = request.getReportVersion(); - Catalog.getCurrentSystemInfo().updateBackendReportVersion(task.getBackendId(), reportVersion, task.getDbId()); + Catalog.getCurrentSystemInfo().updateBackendReportVersion(task.getBackendId(), reportVersion, task.getDbId(), task.getTableId()); } PublishVersionTask publishVersionTask = (PublishVersionTask) task; diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java index 3ca54f32c4184d..b3048ac6512a1d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java @@ -18,7 +18,9 @@ package org.apache.doris.system; import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Database; import org.apache.doris.catalog.DiskInfo; +import org.apache.doris.catalog.Table; import org.apache.doris.cluster.Cluster; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; @@ -890,13 +892,26 @@ public long getBackendReportVersion(long backendId) { } } - public void updateBackendReportVersion(long backendId, long newReportVersion, long dbId) { + public void updateBackendReportVersion(long backendId, long newReportVersion, long dbId, long tableId) { AtomicLong atomicLong = null; if ((atomicLong = idToReportVersionRef.get(backendId)) != null) { - atomicLong.set(newReportVersion); - LOG.debug("update backend {} report version: {}, db: {}", backendId, newReportVersion, dbId); - } else { - LOG.warn("failed to update backend report version, backend {} does not exist", backendId); + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + LOG.warn("failed to update backend report version, db {} does not exist", dbId); + return; + } + Table table = db.getTable(tableId); + if (table == null) { + LOG.warn("failed to update backend report version, table {} in db {} does not exist", tableId, dbId); + return; + } + table.readLock(); + try { + atomicLong.set(newReportVersion); + LOG.debug("update backend {} report version: {}, db: {}, table: {}", backendId, newReportVersion, dbId, tableId); + } finally { + table.readUnlock(); + } } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/cluster/SystemInfoServiceTest.java b/fe/fe-core/src/test/java/org/apache/doris/cluster/SystemInfoServiceTest.java index ef96703d6ca06e..5473264e0c4760 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/cluster/SystemInfoServiceTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/cluster/SystemInfoServiceTest.java @@ -24,6 +24,7 @@ import org.apache.doris.analysis.DropBackendClause; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TabletInvertedIndex; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; @@ -57,6 +58,8 @@ public class SystemInfoServiceTest { private TabletInvertedIndex invertedIndex; @Mocked private Database db; + @Mocked + private Table table; private Analyzer analyzer; @@ -77,10 +80,10 @@ public void setUp() throws IOException { editLog.logBackendStateChange((Backend) any); minTimes = 0; - db.readLock(); + table.readLock(); minTimes = 0; - db.readUnlock(); + table.readUnlock(); minTimes = 0; catalog.getNextId(); @@ -95,6 +98,10 @@ public void setUp() throws IOException { minTimes = 0; result = db; + db.getTable(anyLong); + minTimes = 0; + result = table; + catalog.getCluster(anyString); minTimes = 0; result = new Cluster("cluster", 1); @@ -223,7 +230,7 @@ public void addBackendTest() throws AnalysisException { Assert.assertTrue(Catalog.getCurrentSystemInfo().getBackendReportVersion(backendId) == 0L); - Catalog.getCurrentSystemInfo().updateBackendReportVersion(backendId, 2L, 20000L); + Catalog.getCurrentSystemInfo().updateBackendReportVersion(backendId, 2L, 20000L, 30000L); Assert.assertTrue(Catalog.getCurrentSystemInfo().getBackendReportVersion(backendId) == 2L); } From 8bb19a231982f4fed54f1dfdc71c518a47c0b2c0 Mon Sep 17 00:00:00 2001 From: caiconghui Date: Sun, 29 Nov 2020 17:18:40 +0800 Subject: [PATCH 46/50] change db level lock to table lock level in http2 --- .../doris/httpv2/rest/GetDdlStmtAction.java | 14 +++++----- .../doris/httpv2/rest/MetaInfoAction.java | 27 ++++++++----------- .../doris/httpv2/rest/RowCountAction.java | 22 +++++++-------- .../apache/doris/httpv2/rest/ShowAction.java | 10 ++++--- .../httpv2/rest/StorageTypeCheckAction.java | 20 +++++++------- .../httpv2/rest/TableQueryPlanAction.java | 20 +++++++------- .../httpv2/rest/TableRowCountAction.java | 22 +++++++-------- .../doris/httpv2/rest/TableSchemaAction.java | 21 +++++++-------- 8 files changed, 72 insertions(+), 84 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetDdlStmtAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetDdlStmtAction.java index 3adda7a25e57c1..45ac72bdb6e475 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetDdlStmtAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetDdlStmtAction.java @@ -72,16 +72,16 @@ public Object execute(HttpServletRequest request, HttpServletResponse response) List addPartitionStmt = Lists.newArrayList(); List createRollupStmt = Lists.newArrayList(); - db.readLock(); - try { - Table table = db.getTable(tableName); - if (table == null) { - return ResponseEntityBuilder.okWithCommonError("Table[" + tableName + "] does not exist"); - } + Table table = db.getTable(tableName); + if (table == null) { + return ResponseEntityBuilder.okWithCommonError("Table[" + tableName + "] does not exist"); + } + table.readLock(); + try { Catalog.getDdlStmt(table, createTableStmt, addPartitionStmt, createRollupStmt, true, false /* show password */); } finally { - db.readUnlock(); + table.readUnlock(); } Map> results = Maps.newHashMap(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/MetaInfoAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/MetaInfoAction.java index 1e9e66045801c3..aedc4690b87b26 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/MetaInfoAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/MetaInfoAction.java @@ -146,17 +146,12 @@ public Object getTables( } List tblNames = Lists.newArrayList(); - db.readLock(); - try { - for (Table tbl : db.getTables()) { - if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), fullDbName, tbl.getName(), - PrivPredicate.SHOW)) { - continue; - } - tblNames.add(tbl.getName()); + for (Table tbl : db.getTables()) { + if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), fullDbName, tbl.getName(), + PrivPredicate.SHOW)) { + continue; } - } finally { - db.readUnlock(); + tblNames.add(tbl.getName()); } Collections.sort(tblNames); @@ -230,12 +225,12 @@ public Object getTableSchema( // get all proc paths Map> result = Maps.newHashMap(); - db.readLock(); + Table tbl = db.getTable(tblName); + if (tbl == null) { + return ResponseEntityBuilder.okWithCommonError("Table does not exist: " + tblName); + } + tbl.readLock(); try { - Table tbl = db.getTable(tblName); - if (tbl == null) { - return ResponseEntityBuilder.okWithCommonError("Table does not exist: " + tblName); - } long baseId = -1; if (tbl.getType() == Table.TableType.OLAP) { baseId = ((OlapTable) tbl).getBaseIndexId(); @@ -253,7 +248,7 @@ public Object getTableSchema( } } } finally { - db.readUnlock(); + tbl.readUnlock(); } return ResponseEntityBuilder.ok(result); diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RowCountAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RowCountAction.java index 4c4a4adbadd505..bfe75633c50a3f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RowCountAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RowCountAction.java @@ -25,8 +25,8 @@ import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.Table; -import org.apache.doris.catalog.Table.TableType; import org.apache.doris.catalog.Tablet; +import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.httpv2.entity.ResponseEntityBuilder; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; @@ -71,18 +71,14 @@ protected Object rowcount(HttpServletRequest request, HttpServletResponse respon if (db == null) { return ResponseEntityBuilder.okWithCommonError("Database[" + fullDbName + "] does not exist"); } - db.writeLock(); + OlapTable olapTable = null; + try { + olapTable = (OlapTable) db.getTableOrThrowException(tableName, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + return ResponseEntityBuilder.okWithCommonError(e.getMessage()); + } + olapTable.readLock(); try { - Table table = db.getTable(tableName); - if (table == null) { - return ResponseEntityBuilder.okWithCommonError("Table[" + tableName + "] does not exist"); - } - - if (table.getType() != TableType.OLAP) { - return ResponseEntityBuilder.okWithCommonError("Table[" + tableName + "] is not OLAP table"); - } - - OlapTable olapTable = (OlapTable) table; for (Partition partition : olapTable.getAllPartitions()) { long version = partition.getVisibleVersion(); long versionHash = partition.getVisibleVersionHash(); @@ -103,7 +99,7 @@ protected Object rowcount(HttpServletRequest request, HttpServletResponse respon } // end for indices } // end for partitions } finally { - db.writeUnlock(); + olapTable.readUnlock(); } return ResponseEntityBuilder.ok(indexRowCountMap); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ShowAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ShowAction.java index f08a7a2a9f8632..d98e2489bd660a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ShowAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ShowAction.java @@ -259,9 +259,13 @@ public long getDataSizeOfDatabase(Database db) { if (table.getType() != TableType.OLAP) { continue; } - - long tableSize = ((OlapTable) table).getDataSize(); - totalSize += tableSize; + table.readLock(); + try { + long tableSize = ((OlapTable) table).getDataSize(); + totalSize += tableSize; + } finally { + table.readUnlock(); + } } // end for tables } finally { db.readUnlock(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StorageTypeCheckAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StorageTypeCheckAction.java index 90c730f936ced5..cf35a8e932e40b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StorageTypeCheckAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StorageTypeCheckAction.java @@ -60,15 +60,15 @@ protected Object check_storagetype(HttpServletRequest request, HttpServletRespon } Map> result = Maps.newHashMap(); - db.readLock(); - try { - List
tbls = db.getTables(); - for (Table tbl : tbls) { - if (tbl.getType() != TableType.OLAP) { - continue; - } + List
tbls = db.getTables(); + for (Table tbl : tbls) { + if (tbl.getType() != TableType.OLAP) { + continue; + } - OlapTable olapTbl = (OlapTable) tbl; + OlapTable olapTbl = (OlapTable) tbl; + olapTbl.readLock(); + try { Map indexMap = Maps.newHashMap(); for (Map.Entry entry : olapTbl.getIndexIdToMeta().entrySet()) { MaterializedIndexMeta indexMeta = entry.getValue(); @@ -77,9 +77,9 @@ protected Object check_storagetype(HttpServletRequest request, HttpServletRespon } } result.put(tbl.getName(), indexMap); + } finally { + olapTbl.readUnlock(); } - } finally { - db.readUnlock(); } return ResponseEntityBuilder.ok(result); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableQueryPlanAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableQueryPlanAction.java index f143ea2c849f47..c9a87d9248796a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableQueryPlanAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableQueryPlanAction.java @@ -28,6 +28,7 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Table; import org.apache.doris.common.DorisHttpException; +import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.httpv2.entity.ResponseEntityBuilder; import org.apache.doris.httpv2.util.HttpUtil; import org.apache.doris.mysql.privilege.PrivPredicate; @@ -117,22 +118,19 @@ public Object query_plan( if (db == null) { return ResponseEntityBuilder.okWithCommonError("Database [" + dbName + "] " + "does not exists"); } - // may be should acquire writeLock - db.readLock(); + Table table = null; try { - Table table = db.getTable(tblName); - if (table == null) { - return ResponseEntityBuilder.okWithCommonError("Table [" + tblName + "] " + "does not exists"); - } // just only support OlapTable, ignore others such as ESTable - if (table.getType() != Table.TableType.OLAP) { - return ResponseEntityBuilder.okWithCommonError("only support OlapTable currently, " - + "but Table [" + tblName + "] " + "is not a OlapTable"); - } + table = db.getTableOrThrowException(tblName, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + return ResponseEntityBuilder.okWithCommonError(e.getMessage()); + } + table.readLock(); + try { // parse/analysis/plan the sql and acquire tablet distributions handleQuery(ConnectContext.get(), fullDbName, tblName, sql, resultMap); } finally { - db.readUnlock(); + table.readUnlock(); } } catch (DorisHttpException e) { // status code should conforms to HTTP semantic diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableRowCountAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableRowCountAction.java index 429d1d0a8c9bd5..534e7741ad1238 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableRowCountAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableRowCountAction.java @@ -22,6 +22,7 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Table; import org.apache.doris.common.DorisHttpException; +import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.httpv2.entity.ResponseEntityBuilder; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; @@ -64,22 +65,19 @@ public Object count( if (db == null) { return ResponseEntityBuilder.okWithCommonError("Database [" + dbName + "] " + "does not exists"); } - db.writeLock(); + OlapTable olapTable = null; + try { + olapTable = (OlapTable) db.getTableOrThrowException(tblName, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + return ResponseEntityBuilder.okWithCommonError(e.getMessage()); + } + + olapTable.readLock(); try { - Table table = db.getTable(tblName); - if (table == null) { - return ResponseEntityBuilder.okWithCommonError("Table [" + tblName + "] " + "does not exists"); - } - // just only support OlapTable, ignore others such as ESTable - if (!(table instanceof OlapTable)) { - return ResponseEntityBuilder.okWithCommonError("Table [" + tblName + "] " - + "is not a OlapTable, only support OlapTable currently"); - } - OlapTable olapTable = (OlapTable) table; resultMap.put("status", 200); resultMap.put("size", olapTable.proximateRowCount()); } finally { - db.writeUnlock(); + olapTable.readLock(); } } catch (DorisHttpException e) { // status code should conforms to HTTP semantic diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableSchemaAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableSchemaAction.java index 840770b942d334..a68abdb439c2fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableSchemaAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableSchemaAction.java @@ -20,12 +20,12 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Type; import org.apache.doris.common.DorisHttpException; +import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.httpv2.entity.ResponseEntityBuilder; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; @@ -66,17 +66,14 @@ protected Object schema( if (db == null) { return ResponseEntityBuilder.okWithCommonError("Database [" + dbName + "] " + "does not exists"); } - db.readLock(); + Table table = null; + try { + table = db.getTableOrThrowException(tblName, Table.TableType.OLAP); + } catch (MetaNotFoundException e) { + return ResponseEntityBuilder.okWithCommonError(e.getMessage()); + } + table.readLock(); try { - Table table = db.getTable(tblName); - if (table == null) { - return ResponseEntityBuilder.okWithCommonError("Table [" + tblName + "] " + "does not exists"); - } - // just only support OlapTable, ignore others such as ESTable - if (!(table instanceof OlapTable)) { - return ResponseEntityBuilder.okWithCommonError("Table [" + tblName + "] " - + "is not a OlapTable, only support OlapTable currently"); - } try { List columns = table.getBaseSchema(); List> propList = new ArrayList(columns.size()); @@ -101,7 +98,7 @@ protected Object schema( return ResponseEntityBuilder.okWithCommonError(e.getMessage()); } } finally { - db.readUnlock(); + table.readUnlock(); } } catch (DorisHttpException e) { // status code should conforms to HTTP semantic From b4b680790f1548548ef350b2b8efb091b9d13faf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Wed, 2 Dec 2020 15:36:01 +0800 Subject: [PATCH 47/50] Fix broker load failed bug and add unit test for db --- .../org/apache/doris/catalog/Database.java | 7 +-- .../apache/doris/catalog/DatabaseTest.java | 43 +++++++++++++++++++ 2 files changed, 47 insertions(+), 3 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 85da3bb2ff4d61..7682736578458d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -366,8 +366,9 @@ public List
getTablesOnIdOrderOrThrowException(List tableIdList) th for (Long tableId : tableIdList) { Table table = idToTable.get(tableId); if (table == null) { - throw new MetaNotFoundException("unknown table, table=" + tableId); + throw new MetaNotFoundException("unknown table, tableId=" + tableId); } + tableList.add(table); } if (tableList.size() > 1) { return tableList.stream().sorted(Comparator.comparing(Table::getId)).collect(Collectors.toList()); @@ -405,7 +406,7 @@ public Table getTableOrThrowException(String tableName, TableType tableType) thr throw new MetaNotFoundException("unknown table, table=" + tableName); } if (table.getType() != tableType) { - throw new MetaNotFoundException("table type is not " + tableType + ", table=" + tableName + "type=" + table.getClass()); + throw new MetaNotFoundException("table type is not " + tableType + ", table=" + tableName + ", type=" + table.getClass()); } return table; } @@ -432,7 +433,7 @@ public Table getTableOrThrowException(long tableId, TableType tableType) throws throw new MetaNotFoundException("unknown table, tableId=" + tableId); } if (table.getType() != tableType) { - throw new MetaNotFoundException("table type is not " + tableType + ", type=" + table.getClass()); + throw new MetaNotFoundException("table type is not " + tableType + ", tableId=" + tableId +", type=" + table.getClass()); } return table; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DatabaseTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DatabaseTest.java index 89aeec17b404f6..ac491708e6775e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DatabaseTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DatabaseTest.java @@ -18,7 +18,9 @@ package org.apache.doris.catalog; import org.apache.doris.catalog.MaterializedIndex.IndexState; +import org.apache.doris.common.ExceptionChecker; import org.apache.doris.common.FeConstants; +import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.jmockit.Deencapsulation; import org.apache.doris.persist.CreateTableInfo; import org.apache.doris.persist.EditLog; @@ -99,6 +101,47 @@ public void lockTest() { } } + @Test + public void getTablesOnIdOrderOrThrowExceptionTest() throws MetaNotFoundException { + List baseSchema1 = new LinkedList<>(); + OlapTable table1 = new OlapTable(2000L, "baseTable1", baseSchema1, KeysType.AGG_KEYS, + new SinglePartitionInfo(), new RandomDistributionInfo(10)); + List baseSchema2 = new LinkedList<>(); + OlapTable table2 = new OlapTable(2001L, "baseTable2", baseSchema2, KeysType.DUP_KEYS, + new SinglePartitionInfo(), new RandomDistributionInfo(10)); + db.createTable(table1); + db.createTable(table2); + List tableIdList = Lists.newArrayList(2001L, 2000L); + List
tableList = db.getTablesOnIdOrderOrThrowException(tableIdList); + Assert.assertEquals(2, tableList.size()); + Assert.assertEquals(2000L, tableList.get(0).getId()); + Assert.assertEquals(2001L, tableList.get(1).getId()); + ExceptionChecker.expectThrowsWithMsg(MetaNotFoundException.class, "unknown table, tableId=3000", + () -> db.getTablesOnIdOrderOrThrowException(Lists.newArrayList(3000L))); + } + + @Test + public void getTableOrThrowExceptionTest() throws MetaNotFoundException { + List baseSchema = new LinkedList<>(); + OlapTable table = new OlapTable(2000L, "baseTable", baseSchema, KeysType.AGG_KEYS, + new SinglePartitionInfo(), new RandomDistributionInfo(10)); + db.createTable(table); + Table resultTable1 = db.getTableOrThrowException(2000L, Table.TableType.OLAP); + Table resultTable2 = db.getTableOrThrowException("baseTable", Table.TableType.OLAP); + Assert.assertEquals(table, resultTable1); + Assert.assertEquals(table, resultTable2); + ExceptionChecker.expectThrowsWithMsg(MetaNotFoundException.class, "unknown table, tableId=3000", + () -> db.getTableOrThrowException(3000L, Table.TableType.OLAP)); + ExceptionChecker.expectThrowsWithMsg(MetaNotFoundException.class, "unknown table, table=baseTable1", + () -> db.getTableOrThrowException("baseTable1", Table.TableType.OLAP)); + ExceptionChecker.expectThrowsWithMsg(MetaNotFoundException.class, + "table type is not BROKER, tableId=2000, type=class org.apache.doris.catalog.OlapTable", + () -> db.getTableOrThrowException(2000L, Table.TableType.BROKER)); + ExceptionChecker.expectThrowsWithMsg(MetaNotFoundException.class, + "table type is not BROKER, table=baseTable, type=class org.apache.doris.catalog.OlapTable", + () -> db.getTableOrThrowException("baseTable", Table.TableType.BROKER)); + } + @Test public void createAndDropPartitionTest() { Assert.assertEquals("dbTest", db.getFullName()); From b2f55d756fa12b888cf44d69eaa30a58342b3d26 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Wed, 2 Dec 2020 15:44:06 +0800 Subject: [PATCH 48/50] fix --- fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 7682736578458d..d9fe637c603644 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -433,7 +433,7 @@ public Table getTableOrThrowException(long tableId, TableType tableType) throws throw new MetaNotFoundException("unknown table, tableId=" + tableId); } if (table.getType() != tableType) { - throw new MetaNotFoundException("table type is not " + tableType + ", tableId=" + tableId +", type=" + table.getClass()); + throw new MetaNotFoundException("table type is not " + tableType + ", tableId=" + tableId + ", type=" + table.getClass()); } return table; } From faa6eb24a1d5a4bff4aed3202c8c468fd667f07c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Thu, 3 Dec 2020 20:45:25 +0800 Subject: [PATCH 49/50] Add some comment for the usage of MetaLockUtils --- .../java/org/apache/doris/common/util/MetaLockUtils.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java index d8506b30afc56b..ed0e06a4de3c52 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/MetaLockUtils.java @@ -23,6 +23,11 @@ import java.util.List; import java.util.concurrent.TimeUnit; +/** + * MetaLockUtils is a helper class to lock and unlock all meta object in a list. + * In order to escape dead lock, meta object in list should be sorted in ascending + * order by id first, and then MetaLockUtils can lock them. + */ public class MetaLockUtils { public static void readLockDatabases(List databaseList) { From 56b8d3159e8fde9a54a0e6cfca43c970aa0ee51f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?caiconghui=20=5B=E8=94=A1=E8=81=AA=E8=BE=89=5D?= Date: Mon, 11 Jan 2021 14:49:20 +0800 Subject: [PATCH 50/50] apply fix patch --- .../main/java/org/apache/doris/load/Load.java | 53 ++++++++++++------- .../org/apache/doris/load/LoadChecker.java | 41 +++++++++----- .../java/org/apache/doris/load/LoadJob.java | 8 +-- .../doris/system/SystemInfoService.java | 25 +++------ .../apache/doris/task/LoadPendingTask.java | 5 +- 5 files changed, 77 insertions(+), 55 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java index 73ad11d502c663..9f741283e183e1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java @@ -81,6 +81,7 @@ import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.ListComparator; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.load.AsyncDeleteJob.DeleteState; import org.apache.doris.load.FailMsg.CancelType; @@ -108,6 +109,11 @@ import org.apache.doris.transaction.TransactionState.TxnSourceType; import org.apache.doris.transaction.TransactionStatus; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.base.Strings; @@ -116,11 +122,6 @@ import com.google.common.collect.Sets; import com.google.gson.Gson; -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang.StringUtils; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -2501,8 +2502,24 @@ public void unprotectQuorumLoadJob(LoadJob job, Database db) { public void replayQuorumLoadJob(LoadJob job, Catalog catalog) throws DdlException { // TODO: need to call this.writeLock()? Database db = catalog.getDb(job.getDbId()); - Table table = db.getTable(job.getTableId()); - table.writeLock(); + + List tableIds = Lists.newArrayList(); + long tblId = job.getTableId(); + if (tblId > 0) { + tableIds.add(tblId); + } else { + tableIds.addAll(job.getIdToTableLoadInfo().keySet()); + } + + List
tables = null; + try { + tables = db.getTablesOnIdOrderOrThrowException(tableIds); + } catch (MetaNotFoundException e) { + LOG.error("should not happen", e); + return; + } + + MetaLockUtils.writeLockTables(tables); try { writeLock(); try { @@ -2511,7 +2528,7 @@ public void replayQuorumLoadJob(LoadJob job, Catalog catalog) throws DdlExceptio writeUnlock(); } } finally { - table.writeUnlock(); + MetaLockUtils.writeUnlockTables(tables); } } @@ -2570,8 +2587,9 @@ public void unprotectFinishLoadJob(LoadJob job, Database db) { public void replayFinishLoadJob(LoadJob job, Catalog catalog) { // TODO: need to call this.writeLock()? Database db = catalog.getDb(job.getDbId()); - Table table = db.getTable(job.getTableId()); - table.writeLock(); + // After finish, the idToTableLoadInfo in load job will be set to null. + // We lost table info. So we have to use db lock here. + db.writeLock(); try { writeLock(); try { @@ -2580,7 +2598,7 @@ public void replayFinishLoadJob(LoadJob job, Catalog catalog) { writeUnlock(); } } finally { - table.writeUnlock(); + db.writeUnlock(); } } @@ -2842,7 +2860,6 @@ public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType ca long jobId = job.getId(); long dbId = job.getDbId(); Database db = Catalog.getCurrentCatalog().getDb(dbId); - Table table = db.getTable(job.getTableId()); String errMsg = msg; if (db == null) { // if db is null, update job to cancelled @@ -2857,7 +2874,7 @@ public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType ca writeUnlock(); } } else { - table.writeLock(); + db.writeLock(); try { writeLock(); try { @@ -2870,7 +2887,7 @@ public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType ca Set destStates = STATE_CHANGE_MAP.get(srcState); if (!destStates.contains(destState)) { LOG.warn("state change error. src state: {}, dest state: {}", - srcState.name(), destState.name()); + srcState.name(), destState.name()); return false; } @@ -2914,7 +2931,7 @@ public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType ca TableCommitInfo tableCommitInfo = transactionState.getTableCommitInfo(deleteInfo.getTableId()); PartitionCommitInfo partitionCommitInfo = tableCommitInfo.getPartitionCommitInfo(deleteInfo.getPartitionId()); deleteInfo.updatePartitionVersionInfo(partitionCommitInfo.getVersion(), - partitionCommitInfo.getVersionHash()); + partitionCommitInfo.getVersionHash()); } } MetricRepo.COUNTER_LOAD_FINISHED.increase(1L); @@ -2926,8 +2943,8 @@ public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType ca // clear push tasks for (PushTask pushTask : job.getPushTasks()) { AgentTaskQueue.removePushTask(pushTask.getBackendId(), pushTask.getSignature(), - pushTask.getVersion(), pushTask.getVersionHash(), - pushTask.getPushType(), pushTask.getTaskType()); + pushTask.getVersion(), pushTask.getVersionHash(), + pushTask.getPushType(), pushTask.getTaskType()); } // Clear the Map and Set in this job, reduce the memory cost for finished load job. // for delete job, keep the map and set because some of them is used in show proc method @@ -2948,7 +2965,7 @@ public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType ca writeUnlock(); } } finally { - table.writeUnlock(); + db.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java index c9ddaec721575c..b789cebd7b6a8e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java @@ -17,7 +17,6 @@ package org.apache.doris.load; -import avro.shaded.com.google.common.collect.Lists; import org.apache.doris.alter.RollupJob; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; @@ -32,6 +31,7 @@ import org.apache.doris.common.Config; import org.apache.doris.common.UserException; import org.apache.doris.common.util.MasterDaemon; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.load.AsyncDeleteJob.DeleteState; import org.apache.doris.load.FailMsg.CancelType; import org.apache.doris.load.LoadJob.JobState; @@ -55,12 +55,12 @@ import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TransactionStatus; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; - import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -68,6 +68,8 @@ import java.util.Map.Entry; import java.util.Set; +import avro.shaded.com.google.common.collect.Lists; + public class LoadChecker extends MasterDaemon { private static final Logger LOG = LogManager.getLogger(LoadChecker.class); @@ -242,12 +244,23 @@ private void runOneLoadingJob(LoadJob job) { load.cancelLoadJob(job, CancelType.LOAD_RUN_FAIL, "db does not exist. id: " + dbId); return; } + + List tableIds = Lists.newArrayList(); + long tableId = job.getTableId(); - Table table = null; + if (tableId > 0) { + tableIds.add(tableId); + } else { + // For hadoop load job, the tableId in job is 0(which is unused). So we need to get + // table ids somewhere else. + tableIds.addAll(job.getIdToTableLoadInfo().keySet()); + } + + List
tables = null; try { - table = db.getTableOrThrowException(tableId, Table.TableType.OLAP); + tables = db.getTablesOnIdOrderOrThrowException(tableIds); } catch (UserException e) { - load.cancelLoadJob(job, CancelType.LOAD_RUN_FAIL, "table does not exist. dbId: " + dbId + ", tableId: " + tableId); + load.cancelLoadJob(job, CancelType.LOAD_RUN_FAIL, "table does not exist. dbId: " + dbId + ", err: " + e.getMessage()); return; } @@ -315,12 +328,12 @@ private void runOneLoadingJob(LoadJob job) { // if all tablets are finished or stay in quorum finished for long time, try to commit it. if (System.currentTimeMillis() - job.getQuorumFinishTimeMs() > stragglerTimeout || job.getFullTablets().containsAll(jobTotalTablets)) { - tryCommitJob(job, table); + tryCommitJob(job, tables); } } } - - private void tryCommitJob(LoadJob job, Table table) { + + private void tryCommitJob(LoadJob job, List
tables) { // check transaction state Load load = Catalog.getCurrentCatalog().getLoadInstance(); GlobalTransactionMgr globalTransactionMgr = Catalog.getCurrentGlobalTransactionMgr(); @@ -328,7 +341,9 @@ private void tryCommitJob(LoadJob job, Table table) { List tabletCommitInfos = new ArrayList(); // when be finish load task, fe will update job's finish task info, use lock here to prevent // concurrent problems - table.writeLock(); + + // table in tables are ordered. + MetaLockUtils.writeLockTables(tables); try { TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); for (Replica replica : job.getFinishedReplicas()) { @@ -340,7 +355,7 @@ private void tryCommitJob(LoadJob job, Table table) { } tabletCommitInfos.add(new TabletCommitInfo(tabletId, replica.getBackendId())); } - globalTransactionMgr.commitTransaction(job.getDbId(), Lists.newArrayList(table), job.getTransactionId(), tabletCommitInfos); + globalTransactionMgr.commitTransaction(job.getDbId(), tables, job.getTransactionId(), tabletCommitInfos); } catch (TabletQuorumFailedException e) { // wait the upper application retry } catch (UserException e) { @@ -348,7 +363,7 @@ private void tryCommitJob(LoadJob job, Table table) { transactionState.getTransactionId(), job, e); load.cancelLoadJob(job, CancelType.UNKNOWN, transactionState.getReason()); } finally { - table.writeUnlock(); + MetaLockUtils.writeUnlockTables(tables); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/LoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/LoadJob.java index c8cc31f810d547..b2939c968ff52b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/LoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/LoadJob.java @@ -39,14 +39,14 @@ import org.apache.doris.thrift.TPriority; import org.apache.doris.thrift.TResourceInfo; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + 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; - import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; @@ -246,6 +246,8 @@ public long getDbId() { } public long getTableId() { + // table id may be 0 for some load job, eg, hadoop load job. + // use it carefully. return tableId; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java index b3048ac6512a1d..470dfb525b1f37 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java @@ -20,7 +20,6 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.DiskInfo; -import org.apache.doris.catalog.Table; import org.apache.doris.cluster.Cluster; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; @@ -30,6 +29,11 @@ import org.apache.doris.metric.MetricRepo; import org.apache.doris.system.Backend.BackendState; import org.apache.doris.thrift.TStatusCode; +import org.apache.doris.thrift.TStorageMedium; + +import org.apache.commons.validator.routines.InetAddressValidator; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import com.google.common.base.Preconditions; import com.google.common.base.Strings; @@ -40,11 +44,6 @@ import com.google.common.collect.Multimap; import com.google.common.collect.Sets; -import org.apache.commons.validator.routines.InetAddressValidator; -import org.apache.doris.thrift.TStorageMedium; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; @@ -900,18 +899,8 @@ public void updateBackendReportVersion(long backendId, long newReportVersion, lo LOG.warn("failed to update backend report version, db {} does not exist", dbId); return; } - Table table = db.getTable(tableId); - if (table == null) { - LOG.warn("failed to update backend report version, table {} in db {} does not exist", tableId, dbId); - return; - } - table.readLock(); - try { - atomicLong.set(newReportVersion); - LOG.debug("update backend {} report version: {}, db: {}, table: {}", backendId, newReportVersion, dbId, tableId); - } finally { - table.readUnlock(); - } + atomicLong.set(newReportVersion); + LOG.debug("update backend {} report version: {}, db: {}, table: {}", backendId, newReportVersion, dbId, tableId); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/LoadPendingTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/LoadPendingTask.java index 3dca5c3144b0f0..5242ca0137f5ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/LoadPendingTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/LoadPendingTask.java @@ -32,11 +32,11 @@ import org.apache.doris.transaction.TransactionState.TxnCoordinator; import org.apache.doris.transaction.TransactionState.TxnSourceType; -import com.google.common.base.Joiner; - import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import com.google.common.base.Joiner; + import java.util.List; import java.util.UUID; @@ -69,7 +69,6 @@ protected void exec() { // get db long dbId = job.getDbId(); - long tableId = job.getTableId(); db = Catalog.getCurrentCatalog().getDb(dbId); if (db == null) { load.cancelLoadJob(job, CancelType.ETL_SUBMIT_FAIL, "db does not exist. id: " + dbId);