Skip to content

Commit

Permalink
[Refactor] Synchronize OLAP external table metadata when loading data (
Browse files Browse the repository at this point in the history
…#24739)

Currently, StarRocks use StarRocksRepository to synchronize OLAP
external table metadata every 10 seconds, which is
inefficient, because the meta data will never be used when there is no
load job on that external table. Change the synchronization to be
triggered when there is load job on that table.

Since the synchronization is time costly, so we should release the
database lock when synchronizing meta data.

Signed-off-by: gengjun-git <gengjun@starrocks.com>
(cherry picked from commit ef1fc6a)

# Conflicts:
#	fe/fe-core/src/main/java/com/starrocks/catalog/ExternalOlapTable.java
#	fe/fe-core/src/main/java/com/starrocks/external/starrocks/StarRocksRepository.java
#	fe/fe-core/src/main/java/com/starrocks/qe/StmtExecutor.java
#	fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java
#	fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java
#	fe/fe-core/src/main/java/com/starrocks/sql/analyzer/InsertAnalyzer.java
#	fe/fe-core/src/main/java/com/starrocks/sql/analyzer/QueryAnalyzer.java
  • Loading branch information
gengjun-git authored and mergify[bot] committed Jun 12, 2023
1 parent 15b58c9 commit 078cc6f
Show file tree
Hide file tree
Showing 10 changed files with 488 additions and 213 deletions.
4 changes: 4 additions & 0 deletions fe/fe-core/src/main/java/com/starrocks/catalog/Database.java
Original file line number Diff line number Diff line change
Expand Up @@ -232,6 +232,10 @@ public void readUnlock() {
this.rwLock.readLock().unlock();
}

public boolean isReadLockHeldByCurrentThread() {
return this.rwLock.getReadHoldCount() > 0;
}

public void writeLock() {
long startMs = TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
String threadDump = getOwnerInfo(rwLock.getOwner());
Expand Down
399 changes: 199 additions & 200 deletions fe/fe-core/src/main/java/com/starrocks/catalog/ExternalOlapTable.java

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -289,8 +289,7 @@ public OlapTable(long id, String tableName, List<Column> baseSchema, KeysType ke

// Only Copy necessary metadata for query.
// We don't do deep copy, because which is very expensive;
public OlapTable copyOnlyForQuery() {
OlapTable olapTable = new OlapTable();
public void copyOnlyForQuery(OlapTable olapTable) {
olapTable.id = this.id;
olapTable.name = this.name;
olapTable.fullSchema = Lists.newArrayList(this.fullSchema);
Expand Down Expand Up @@ -320,7 +319,6 @@ public OlapTable copyOnlyForQuery() {
if (this.tableProperty != null) {
olapTable.tableProperty = this.tableProperty.copy();
}
return olapTable;
}

public BinlogConfig getCurBinlogConfig() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
import com.starrocks.catalog.ExternalOlapTable;
import com.starrocks.common.Config;
import com.starrocks.rpc.FrontendServiceProxy;
import com.starrocks.sql.common.MetaNotFoundException;
import com.starrocks.thrift.TGetTableMetaRequest;
import com.starrocks.thrift.TGetTableMetaResponse;
import com.starrocks.thrift.TNetworkAddress;
Expand All @@ -30,7 +31,7 @@
public class TableMetaSyncer {
private static final Logger LOG = LogManager.getLogger(TableMetaSyncer.class);

public void syncTable(ExternalOlapTable table) {
public void syncTable(ExternalOlapTable table) throws MetaNotFoundException {
String host = table.getSourceTableHost();
int port = table.getSourceTablePort();
TNetworkAddress addr = new TNetworkAddress(host, port);
Expand All @@ -50,11 +51,13 @@ public void syncTable(ExternalOlapTable table) {
errMsg = "";
}
LOG.warn("get TableMeta failed: {}", errMsg);
throw new MetaNotFoundException(errMsg);
} else {
table.updateMeta(request.getDb_name(), response.getTable_meta(), response.getBackends());
}
} catch (Exception e) {
LOG.warn("call fe {} refreshTable rpc method failed", addr, e);
throw new MetaNotFoundException("get TableMeta failed from " + addr + ", error: " + e.getMessage());
}
}
};
}
18 changes: 18 additions & 0 deletions fe/fe-core/src/main/java/com/starrocks/qe/StmtExecutor.java
Original file line number Diff line number Diff line change
Expand Up @@ -1301,8 +1301,26 @@ public void handleDMLStmt(ExecPlan execPlan, DmlStmt stmt) throws Exception {
return;
}

<<<<<<< HEAD
if (parsedStmt instanceof InsertStmt && ((InsertStmt) parsedStmt).isOverwrite()
&& !((InsertStmt) parsedStmt).hasOverwriteJob()) {
=======
MetaUtils.normalizationTableName(context, stmt.getTableName());
String catalogName = stmt.getTableName().getCatalog();
String dbName = stmt.getTableName().getDb();
String tableName = stmt.getTableName().getTbl();
Database database = GlobalStateMgr.getCurrentState().getMetadataMgr().getDb(catalogName, dbName);
Table targetTable;
if (stmt instanceof InsertStmt && ((InsertStmt) stmt).getTargetTable() != null) {
targetTable = ((InsertStmt) stmt).getTargetTable();
} else {
targetTable = GlobalStateMgr.getCurrentState().getMetadataMgr().getTable(catalogName, dbName, tableName);
}

if (parsedStmt instanceof InsertStmt && ((InsertStmt) parsedStmt).isOverwrite() &&
!((InsertStmt) parsedStmt).hasOverwriteJob() &&
!(targetTable instanceof IcebergTable)) {
>>>>>>> ef1fc6a40 ([Refactor] Synchronize OLAP external table metadata when loading data (#24739))
handleInsertOverwrite((InsertStmt) parsedStmt);
return;
}
Expand Down
101 changes: 94 additions & 7 deletions fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java
Original file line number Diff line number Diff line change
Expand Up @@ -138,8 +138,11 @@
import com.starrocks.connector.hive.events.MetastoreEventsProcessor;
import com.starrocks.consistency.ConsistencyChecker;
import com.starrocks.credential.CloudCredentialUtil;
<<<<<<< HEAD
import com.starrocks.external.elasticsearch.EsRepository;
import com.starrocks.external.starrocks.StarRocksRepository;
=======
>>>>>>> ef1fc6a40 ([Refactor] Synchronize OLAP external table metadata when loading data (#24739))
import com.starrocks.ha.FrontendNodeType;
import com.starrocks.ha.HAProtocol;
import com.starrocks.ha.LeaderInfo;
Expand Down Expand Up @@ -356,7 +359,6 @@ public class GlobalStateMgr {
private Daemon replayer;
private Daemon timePrinter;
private EsRepository esRepository; // it is a daemon, so add it here
private StarRocksRepository starRocksRepository;
private MetastoreEventsProcessor metastoreEventsProcessor;
private ConnectorTableMetadataProcessor connectorTableMetadataProcessor;

Expand Down Expand Up @@ -628,7 +630,6 @@ private GlobalStateMgr(boolean isCkptGlobalState) {
this.resourceGroupMgr = new ResourceGroupMgr(this);

this.esRepository = new EsRepository();
this.starRocksRepository = new StarRocksRepository();
this.metastoreEventsProcessor = new MetastoreEventsProcessor();
this.connectorTableMetadataProcessor = new ConnectorTableMetadataProcessor();

Expand Down Expand Up @@ -1281,7 +1282,6 @@ private void startNonLeaderDaemonThreads() {
labelCleaner.start();
// ES state store
esRepository.start();
starRocksRepository.start();

if (Config.enable_hms_events_incremental_sync) {
metastoreEventsProcessor.start();
Expand Down Expand Up @@ -1389,6 +1389,7 @@ public void loadImage(String imageDir) throws IOException, DdlException {
checksum = loadShardManager(dis, checksum);
remoteChecksum = dis.readLong();

<<<<<<< HEAD
checksum = loadCompactionManager(dis, checksum);
remoteChecksum = dis.readLong();
checksum = loadStreamLoadManager(dis, checksum);
Expand All @@ -1402,6 +1403,96 @@ public void loadImage(String imageDir) throws IOException, DdlException {
checksum = localMetastore.loadAutoIncrementId(dis, checksum);
remoteChecksum = dis.readLong();
// ** NOTICE **: always add new code at the end
=======
try {
SRMetaBlockLoader imageLoader = entry.getValue();
imageLoader.apply(reader);
LOG.info("Success load StarRocks meta block " + srMetaBlockID.name() + " from image");
} catch (SRMetaBlockEOFException srMetaBlockEOFException) {
/*
The number of json expected to be read is more than the number of json actually stored
in the image, which usually occurs when the module adds new functions.
*/
LOG.warn("Got EOF exception, ignore, ", srMetaBlockEOFException);
} catch (SRMetaBlockException srMetaBlockException) {
LOG.error("Load meta block failed ", srMetaBlockException);
throw new IOException("Load meta block failed ", srMetaBlockException);
} finally {
reader.close();
}
if (iterator.hasNext()) {
entry = iterator.next();
} else {
break;
}
}
} catch (SRMetaBlockException e) {
LOG.error("load meta block failed ", e);
throw new IOException("load meta block failed ", e);
}
} else {
checksum = loadHeaderV1(dis, checksum);
checksum = nodeMgr.loadLeaderInfo(dis, checksum);
checksum = nodeMgr.loadFrontends(dis, checksum);
checksum = nodeMgr.loadBackends(dis, checksum);
checksum = localMetastore.loadDb(dis, checksum);
// ATTN: this should be done after load Db, and before loadAlterJob
localMetastore.recreateTabletInvertIndex();
// rebuild es state state
esRepository.loadTableFromCatalog();

checksum = load.loadLoadJob(dis, checksum);
checksum = loadAlterJob(dis, checksum);
checksum = recycleBin.loadRecycleBin(dis, checksum);
checksum = VariableMgr.loadGlobalVariable(dis, checksum);
checksum = localMetastore.loadCluster(dis, checksum);
checksum = nodeMgr.loadBrokers(dis, checksum);
checksum = loadResources(dis, checksum);
checksum = exportMgr.loadExportJob(dis, checksum);
checksum = backupHandler.loadBackupHandler(dis, checksum, this);
checksum = auth.loadAuth(dis, checksum);
// global transaction must be replayed before load jobs v2
checksum = globalTransactionMgr.loadTransactionState(dis, checksum);
checksum = colocateTableIndex.loadColocateTableIndex(dis, checksum);
checksum = routineLoadMgr.loadRoutineLoadJobs(dis, checksum);
checksum = loadMgr.loadLoadJobsV2(dis, checksum);
checksum = smallFileMgr.loadSmallFiles(dis, checksum);
checksum = pluginMgr.loadPlugins(dis, checksum);
checksum = loadDeleteHandler(dis, checksum);
remoteChecksum = dis.readLong();
checksum = analyzeMgr.loadAnalyze(dis, checksum);
remoteChecksum = dis.readLong();
checksum = resourceGroupMgr.loadResourceGroups(dis, checksum);
checksum = auth.readAsGson(dis, checksum);
remoteChecksum = dis.readLong();
checksum = taskManager.loadTasks(dis, checksum);
remoteChecksum = dis.readLong();
checksum = catalogMgr.loadCatalogs(dis, checksum);
remoteChecksum = dis.readLong();
checksum = loadInsertOverwriteJobs(dis, checksum);
checksum = nodeMgr.loadComputeNodes(dis, checksum);
remoteChecksum = dis.readLong();
// ShardManager DEPRECATED, keep it for backward compatible
checksum = loadShardManager(dis, checksum);
remoteChecksum = dis.readLong();

checksum = loadCompactionManager(dis, checksum);
remoteChecksum = dis.readLong();
checksum = loadStreamLoadManager(dis, checksum);
remoteChecksum = dis.readLong();
checksum = MaterializedViewMgr.getInstance().reload(dis, checksum);
remoteChecksum = dis.readLong();
globalFunctionMgr.loadGlobalFunctions(dis, checksum);
loadRBACPrivilege(dis);
checksum = warehouseMgr.loadWarehouses(dis, checksum);
remoteChecksum = dis.readLong();
checksum = localMetastore.loadAutoIncrementId(dis, checksum);
remoteChecksum = dis.readLong();
// ** NOTICE **: always add new code at the end

Preconditions.checkState(remoteChecksum == checksum, remoteChecksum + " vs. " + checksum);
}
>>>>>>> ef1fc6a40 ([Refactor] Synchronize OLAP external table metadata when loading data (#24739))
} catch (EOFException exception) {
LOG.warn("load image eof.", exception);
} finally {
Expand Down Expand Up @@ -2929,10 +3020,6 @@ public EsRepository getEsRepository() {
return this.esRepository;
}

public StarRocksRepository getStarRocksRepository() {
return this.starRocksRepository;
}

public MetastoreEventsProcessor getMetastoreEventsProcessor() {
return this.metastoreEventsProcessor;
}
Expand Down
102 changes: 102 additions & 0 deletions fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java
Original file line number Diff line number Diff line change
Expand Up @@ -5309,5 +5309,107 @@ public void addOrReplaceAutoIncrementIdByTableId(Long tableId, Long id) {
tableIdToIncrementId.replace(tableId, id);
}
}
<<<<<<< HEAD
=======

private void setDbStorageVolumeInfo(Database db, String volume) throws DdlException {
StorageVolumeMgr svm = GlobalStateMgr.getCurrentState().getStorageVolumeMgr();
StorageVolume sv = null;
try {
if (volume.equals(StorageVolumeMgr.DEFAULT)) {
sv = svm.getDefaultStorageVolume();
} else {
sv = svm.getStorageVolumeByName(volume);
}
} catch (AnalysisException e) {
throw new DdlException(e.getMessage());
}
if (sv == null) {
throw new DdlException("Unknown storage volume \"" + volume + "\"");
}
db.setStorageVolumeId(sv.getId());
}

public void save(DataOutputStream dos) throws IOException, SRMetaBlockException {
// Don't write system db meta
Map<Long, Database> idToDbNormal = idToDb.entrySet().stream().filter(entry -> entry.getKey() > NEXT_ID_INIT_VALUE)
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
int totalTableNum = 0;
for (Database database : idToDbNormal.values()) {
totalTableNum += database.getTableNumber();
}
int cnt = 1 + idToDbNormal.size() + idToDbNormal.size() /* record database table size */ + totalTableNum + 1;

SRMetaBlockWriter writer = new SRMetaBlockWriter(dos, SRMetaBlockID.LOCAL_META_STORE, cnt);

writer.writeJson(idToDbNormal.size());
for (Database database : idToDbNormal.values()) {
writer.writeJson(database);
writer.writeJson(database.getTables().size());
List<Table> tables = database.getTables();
for (Table table : tables) {
writer.writeJson(table);
}
}

AutoIncrementInfo info = new AutoIncrementInfo(tableIdToIncrementId);
writer.writeJson(info);

writer.close();
}

public void load(SRMetaBlockReader reader) throws IOException, SRMetaBlockException, SRMetaBlockEOFException {
int dbSize = reader.readJson(int.class);
for (int i = 0; i < dbSize; ++i) {
Database db = reader.readJson(Database.class);
int tableSize = reader.readInt();
for (int j = 0; j < tableSize; ++j) {
Table table = reader.readJson(Table.class);
db.createTableWithLock(table, true);
}

idToDb.put(db.getId(), db);
fullNameToDb.put(db.getFullName(), db);
stateMgr.getGlobalTransactionMgr().addDatabaseTransactionMgr(db.getId());
db.getMaterializedViews().forEach(Table::onCreate);
db.getHiveTables().forEach(Table::onCreate);
}

// put built-in database into local metastore
InfoSchemaDb infoSchemaDb = new InfoSchemaDb();
Preconditions.checkState(infoSchemaDb.getId() < NEXT_ID_INIT_VALUE,
"InfoSchemaDb id shouldn't larger than " + NEXT_ID_INIT_VALUE);
idToDb.put(infoSchemaDb.getId(), infoSchemaDb);
fullNameToDb.put(infoSchemaDb.getFullName(), infoSchemaDb);

if (getFullNameToDb().containsKey(StarRocksDb.DATABASE_NAME)) {
LOG.warn("Since the the database of starrocks already exists, " +
"the system will not automatically create the database of starrocks for system.");
} else {
StarRocksDb starRocksDb = new StarRocksDb();
Preconditions.checkState(infoSchemaDb.getId() < NEXT_ID_INIT_VALUE,
"starocks id shouldn't larger than " + NEXT_ID_INIT_VALUE);
idToDb.put(starRocksDb.getId(), starRocksDb);
fullNameToDb.put(starRocksDb.getFullName(), starRocksDb);
}

AutoIncrementInfo autoIncrementInfo = reader.readJson(AutoIncrementInfo.class);
for (Map.Entry<Long, Long> entry : autoIncrementInfo.tableIdToIncrementId().entrySet()) {
Long tableId = entry.getKey();
Long id = entry.getValue();

tableIdToIncrementId.put(tableId, id);
}

recreateTabletInvertIndex();
GlobalStateMgr.getCurrentState().getEsRepository().loadTableFromCatalog();

/*
* defaultCluster has no meaning, it is only for compatibility with
* old versions of the code (defaultCluster is required for 3.0 fallback)
*/
defaultCluster = new Cluster(SystemInfoService.DEFAULT_CLUSTER, NEXT_ID_INIT_VALUE);
}
>>>>>>> ef1fc6a40 ([Refactor] Synchronize OLAP external table metadata when loading data (#24739))
}

Original file line number Diff line number Diff line change
Expand Up @@ -524,7 +524,8 @@ public Void visitTable(TableRelation node, Void context) {
OlapTable table = (OlapTable) node.getTable();
olapTables.add(table);
// Only copy the necessary olap table meta to avoid the lock when plan query
OlapTable copied = table.copyOnlyForQuery();
OlapTable copied = new OlapTable();
table.copyOnlyForQuery(copied);
node.setTable(copied);
}
return null;
Expand Down

0 comments on commit 078cc6f

Please sign in to comment.