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/catalog/OlapTable.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/AnalyzerUtils.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 6e5310e commit 97c7ae0
Show file tree
Hide file tree
Showing 10 changed files with 691 additions and 124 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 @@ -213,6 +213,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
329 changes: 208 additions & 121 deletions fe/fe-core/src/main/java/com/starrocks/catalog/ExternalOlapTable.java

Large diffs are not rendered by default.

69 changes: 69 additions & 0 deletions fe/fe-core/src/main/java/com/starrocks/catalog/OlapTable.java
Original file line number Diff line number Diff line change
Expand Up @@ -248,6 +248,75 @@ public OlapTable(long id, String tableName, List<Column> baseSchema, KeysType ke
this.tableProperty = null;
}

<<<<<<< HEAD
=======
// Only Copy necessary metadata for query.
// We don't do deep copy, because which is very expensive;
public void copyOnlyForQuery(OlapTable olapTable) {
olapTable.id = this.id;
olapTable.name = this.name;
olapTable.fullSchema = Lists.newArrayList(this.fullSchema);
olapTable.nameToColumn = Maps.newHashMap(this.nameToColumn);
olapTable.relatedMaterializedViews = Sets.newHashSet(this.relatedMaterializedViews);
olapTable.state = this.state;
olapTable.indexNameToId = Maps.newHashMap(this.indexNameToId);
olapTable.indexIdToMeta = Maps.newHashMap(this.indexIdToMeta);
olapTable.keysType = this.keysType;
olapTable.partitionInfo = new PartitionInfo();
if (this.partitionInfo instanceof RangePartitionInfo) {
olapTable.partitionInfo = new RangePartitionInfo((RangePartitionInfo) this.partitionInfo);
} else if (this.partitionInfo instanceof SinglePartitionInfo) {
olapTable.partitionInfo = this.partitionInfo;
}
olapTable.defaultDistributionInfo = this.defaultDistributionInfo;
Map<Long, Partition> idToPartitions = new HashMap<>(this.idToPartition.size());
Map<String, Partition> nameToPartitions = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
for (Map.Entry<Long, Partition> kv : this.idToPartition.entrySet()) {
Partition copiedPartition = kv.getValue().shallowCopy();
idToPartitions.put(kv.getKey(), copiedPartition);
nameToPartitions.put(kv.getValue().getName(), copiedPartition);
}
olapTable.idToPartition = idToPartitions;
olapTable.nameToPartition = nameToPartitions;
olapTable.baseIndexId = this.baseIndexId;
if (this.tableProperty != null) {
olapTable.tableProperty = this.tableProperty.copy();
}
}

public BinlogConfig getCurBinlogConfig() {
if (tableProperty != null) {
return tableProperty.getBinlogConfig();
}
return null;
}

public void setCurBinlogConfig(BinlogConfig curBinlogConfig) {
if (tableProperty == null) {
tableProperty = new TableProperty(Maps.newHashMap());
}
tableProperty.modifyTableProperties(curBinlogConfig.toProperties());
tableProperty.setBinlogConfig(curBinlogConfig);
}

public boolean containsBinlogConfig() {
if (tableProperty == null ||
tableProperty.getBinlogConfig() == null ||
tableProperty.getBinlogConfig().getVersion() == BinlogConfig.INVALID) {
return false;
}
return true;
}

public long getBinlogTxnId() {
return binlogTxnId;
}

public void setBinlogTxnId(long binlogTxnId) {
this.binlogTxnId = binlogTxnId;
}

>>>>>>> ef1fc6a40 ([Refactor] Synchronize OLAP external table metadata when loading data (#24739))
public void setTableProperty(TableProperty tableProperty) {
this.tableProperty = tableProperty;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,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 @@ -17,7 +18,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 @@ -37,11 +38,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 @@ -1266,8 +1266,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
123 changes: 122 additions & 1 deletion fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java
Original file line number Diff line number Diff line change
Expand Up @@ -123,8 +123,11 @@
import com.starrocks.connector.iceberg.IcebergRepository;
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 @@ -338,8 +341,11 @@ public class GlobalStateMgr {
private Daemon replayer;
private Daemon timePrinter;
private EsRepository esRepository; // it is a daemon, so add it here
<<<<<<< HEAD
private StarRocksRepository starRocksRepository;
private IcebergRepository icebergRepository;
=======
>>>>>>> ef1fc6a40 ([Refactor] Synchronize OLAP external table metadata when loading data (#24739))
private MetastoreEventsProcessor metastoreEventsProcessor;
private ConnectorTableMetadataProcessor connectorTableMetadataProcessor;

Expand Down Expand Up @@ -592,8 +598,11 @@ private GlobalStateMgr(boolean isCheckpointCatalog) {
this.resourceGroupMgr = new ResourceGroupMgr(this);

this.esRepository = new EsRepository();
<<<<<<< HEAD
this.starRocksRepository = new StarRocksRepository();
this.icebergRepository = new IcebergRepository();
=======
>>>>>>> ef1fc6a40 ([Refactor] Synchronize OLAP external table metadata when loading data (#24739))
this.metastoreEventsProcessor = new MetastoreEventsProcessor();
this.connectorTableMetadataProcessor = new ConnectorTableMetadataProcessor();

Expand Down Expand Up @@ -1217,7 +1226,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 @@ -1293,6 +1301,7 @@ public void loadImage(String imageDir) throws IOException, DdlException {
esRepository.loadTableFromCatalog();
starRocksRepository.loadTableFromCatalog();

<<<<<<< HEAD
checksum = load.loadLoadJob(dis, checksum);
checksum = loadAlterJob(dis, checksum);
checksum = recycleBin.loadRecycleBin(dis, checksum);
Expand Down Expand Up @@ -1331,6 +1340,115 @@ public void loadImage(String imageDir) throws IOException, DdlException {
checksum = loadStreamLoadManager(dis, checksum);
remoteChecksum = dis.readLong();
loadRBACPrivilege(dis);
=======
Iterator<Map.Entry<SRMetaBlockID, SRMetaBlockLoader>> iterator = loadImages.entrySet().iterator();
Map.Entry<SRMetaBlockID, SRMetaBlockLoader> entry = iterator.next();
while (true) {
SRMetaBlockID srMetaBlockID = entry.getKey();
SRMetaBlockReader reader = new SRMetaBlockReader(dis, srMetaBlockID);
if (reader.getHeader().getId() != srMetaBlockID) {
/*
The expected read module does not match the module stored in the image,
and the json chunk is skipped directly. This usually occurs in several situations.
1. When the obsolete image code is deleted.
2. When the new version rolls back to the old version,
the old version ignores the functions of the new version
*/
LOG.warn(String.format("Ignore this invalid meta block, sr meta block id mismatch" +
"(expect %s actual %s)", srMetaBlockID.name(), reader.getHeader().getId().name()));
reader.close();
continue;
}

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 @@ -2821,6 +2939,7 @@ public EsRepository getEsRepository() {
return this.esRepository;
}

<<<<<<< HEAD
public StarRocksRepository getStarRocksRepository() {
return this.starRocksRepository;
}
Expand All @@ -2830,6 +2949,8 @@ public IcebergRepository getIcebergRepository() {
}


=======
>>>>>>> ef1fc6a40 ([Refactor] Synchronize OLAP external table metadata when loading data (#24739))
public MetastoreEventsProcessor getMetastoreEventsProcessor() {
return this.metastoreEventsProcessor;
}
Expand Down

0 comments on commit 97c7ae0

Please sign in to comment.