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
…StarRocks#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>
  • Loading branch information
gengjun-git committed Jun 16, 2023
1 parent 8942387 commit b3656af
Show file tree
Hide file tree
Showing 8 changed files with 277 additions and 253 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
408 changes: 198 additions & 210 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());
}
}
};
}
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,6 @@
import com.starrocks.consistency.ConsistencyChecker;
import com.starrocks.credential.CloudCredentialUtil;
import com.starrocks.external.elasticsearch.EsRepository;
import com.starrocks.external.starrocks.StarRocksRepository;
import com.starrocks.ha.FrontendNodeType;
import com.starrocks.ha.HAProtocol;
import com.starrocks.ha.LeaderInfo;
Expand Down Expand Up @@ -359,7 +358,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 @@ -631,7 +629,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 @@ -1284,7 +1281,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 @@ -2935,10 +2931,6 @@ public EsRepository getEsRepository() {
return this.esRepository;
}

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

public MetastoreEventsProcessor getMetastoreEventsProcessor() {
return this.metastoreEventsProcessor;
}
Expand Down
71 changes: 42 additions & 29 deletions fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java
Original file line number Diff line number Diff line change
Expand Up @@ -1298,7 +1298,7 @@ private void addPartitions(Database db, String tableName, List<PartitionDesc> pa
distributionInfo = getDistributionInfo(olapTable, addPartitionClause).copy();
if (distributionInfo.getBucketNum() == 0) {
int numBucket = CatalogUtils.calAvgBucketNumOfRecentPartitions(olapTable, 5,
Config.enable_auto_tablet_distribution);
Config.enable_auto_tablet_distribution);
distributionInfo.setBucketNum(numBucket);
}

Expand Down Expand Up @@ -1360,7 +1360,8 @@ private void addPartitions(Database db, String tableName, List<PartitionDesc> pa
// update partition info
updatePartitionInfo(partitionInfo, partitionMap, existPartitionNameSet, addPartitionClause, olapTable);

colocateTableIndex.updateLakeTableColocationInfo(olapTable, true /* isJoin */, null /* expectGroupId */);
colocateTableIndex
.updateLakeTableColocationInfo(olapTable, true /* isJoin */, null /* expectGroupId */);

// add partition log
addPartitionLog(db, olapTable, partitionDescs, addPartitionClause, partitionInfo, partitionList,
Expand Down Expand Up @@ -1595,7 +1596,8 @@ private Partition createPartitionCommon(Database db, OlapTable table, long parti
}
DistributionInfo distributionInfo = table.getDefaultDistributionInfo().copy();
if (distributionInfo.getBucketNum() == 0) {
int numBucket = CatalogUtils.calAvgBucketNumOfRecentPartitions(table, 5, Config.enable_auto_tablet_distribution);
int numBucket =
CatalogUtils.calAvgBucketNumOfRecentPartitions(table, 5, Config.enable_auto_tablet_distribution);
distributionInfo.setBucketNum(numBucket);
}

Expand Down Expand Up @@ -1764,7 +1766,8 @@ private List<CreateReplicaTask> buildCreateReplicaTasks(long dbId, OlapTable tab
return tasks;
}

private List<CreateReplicaTask> buildCreateReplicaTasks(long dbId, OlapTable table, Partition partition) throws DdlException {
private List<CreateReplicaTask> buildCreateReplicaTasks(long dbId, OlapTable table, Partition partition)
throws DdlException {
ArrayList<CreateReplicaTask> tasks = new ArrayList<>((int) partition.getReplicaCount());
for (MaterializedIndex index : partition.getMaterializedIndices(MaterializedIndex.IndexExtState.VISIBLE)) {
tasks.addAll(buildCreateReplicaTasks(dbId, table, partition, index));
Expand Down Expand Up @@ -1896,15 +1899,15 @@ private void waitForFinished(MarkedCountDownLatch<Long, Long> countDownLatch, lo

String userErrorMsg = String.format(
"Table creation timed out.\n You can increase the timeout by increasing the " +
"config \"tablet_create_timeout_second\" and try again.\n" +
"To increase the config \"tablet_create_timeout_second\" (currently %d), run the following command:\n" +
"```\nadmin set frontend config(\"tablet_create_timeout_second\"=\"%d\")\n```\n" +
"or add the following configuration to the fe.conf file and restart the process:\n" +
"```\ntablet_create_timeout_second=%d\n```",
"config \"tablet_create_timeout_second\" and try again.\n" +
"To increase the config \"tablet_create_timeout_second\" (currently %d), run the following command:\n" +
"```\nadmin set frontend config(\"tablet_create_timeout_second\"=\"%d\")\n```\n" +
"or add the following configuration to the fe.conf file and restart the process:\n" +
"```\ntablet_create_timeout_second=%d\n```",
Config.tablet_create_timeout_second,
Config.tablet_create_timeout_second * 2,
Config.tablet_create_timeout_second * 2
);
);
countDownLatch.countDownToZero(new Status(TStatusCode.TIMEOUT, "timed out"));
throw new DdlException(userErrorMsg);
}
Expand Down Expand Up @@ -1949,7 +1952,8 @@ public ColocateTableIndex getColocateTableIndex() {
return colocateTableIndex;
}

private boolean processColocationProperties(String colocateGroup, Database db, OlapTable olapTable, boolean expectLakeTable)
private boolean processColocationProperties(String colocateGroup, Database db, OlapTable olapTable,
boolean expectLakeTable)
throws DdlException {
if (Strings.isNullOrEmpty(colocateGroup)) {
return false;
Expand Down Expand Up @@ -2125,28 +2129,35 @@ private void createOlapOrLakeTable(Database db, CreateTableStmt stmt) throws Ddl
String volume = (properties != null) ? properties.remove(PropertyAnalyzer.PROPERTIES_STORAGE_VOLUME) : null;

if ("local".equalsIgnoreCase(volume)) {
table = new OlapTable(tableId, tableName, baseSchema, keysType, partitionInfo, distributionInfo, indexes);
table = new OlapTable(tableId, tableName, baseSchema, keysType, partitionInfo, distributionInfo,
indexes);
} else if ("default".equalsIgnoreCase(volume)) {
table = new LakeTable(tableId, tableName, baseSchema, keysType, partitionInfo, distributionInfo, indexes);
table = new LakeTable(tableId, tableName, baseSchema, keysType, partitionInfo, distributionInfo,
indexes);
setLakeStorageInfo(table, properties);
} else if (!Strings.isNullOrEmpty(volume)) {
throw new DdlException("Unknown storage volume \"" + volume + "\"");
} else if (runMode == RunMode.SHARED_DATA) {
table = new LakeTable(tableId, tableName, baseSchema, keysType, partitionInfo, distributionInfo, indexes);
table = new LakeTable(tableId, tableName, baseSchema, keysType, partitionInfo, distributionInfo,
indexes);
setLakeStorageInfo(table, properties);
} else {
table = new OlapTable(tableId, tableName, baseSchema, keysType, partitionInfo, distributionInfo, indexes);
table = new OlapTable(tableId, tableName, baseSchema, keysType, partitionInfo, distributionInfo,
indexes);
}

if (table.isCloudNativeTable() && !runMode.isAllowCreateLakeTable()) {
throw new DdlException("Cannot create table with persistent volume in current run mode \"" + runMode + "\"");
if (table.isCloudNativeTable() && !runMode.isAllowCreateLakeTable()) {
throw new DdlException(
"Cannot create table with persistent volume in current run mode \"" + runMode + "\"");
}
if (table.isOlapTable() && !runMode.isAllowCreateOlapTable()) {
throw new DdlException("Cannot create table without persistent volume in current run mode \"" + runMode + "\"");
throw new DdlException(
"Cannot create table without persistent volume in current run mode \"" + runMode + "\"");
}

if (table.isCloudNativeTable() && table.getKeysType() == KeysType.PRIMARY_KEYS) {
throw new DdlException("Does not support primary key in current version with run mode \"" + runMode + "\"");
throw new DdlException(
"Does not support primary key in current version with run mode \"" + runMode + "\"");
}
} else {
throw new DdlException("Unrecognized engine \"" + stmt.getEngineName() + "\"");
Expand Down Expand Up @@ -2291,7 +2302,7 @@ private void createOlapOrLakeTable(Database db, CreateTableStmt stmt) throws Ddl
try {
colocateGroup = PropertyAnalyzer.analyzeColocate(properties);
boolean addedToColocateGroup = processColocationProperties(colocateGroup, db,
table, false /* expectLakeTable */);
table, false /* expectLakeTable */);
if (table instanceof ExternalOlapTable == false && addedToColocateGroup) {
// Colocate table should keep the same bucket number accross the partitions
DistributionInfo defaultDistributionInfo = table.getDefaultDistributionInfo();
Expand Down Expand Up @@ -2800,7 +2811,8 @@ private void createLakeTablets(OlapTable table, long partitionId, long shardGrou
properties.put(LakeTablet.PROPERTY_KEY_INDEX_ID, Long.toString(index.getId()));
int bucketNum = distributionInfo.getBucketNum();
List<Long> shardIds = stateMgr.getStarOSAgent().createShards(bucketNum,
table.getPartitionFilePathInfo(), table.getPartitionFileCacheInfo(partitionId), shardGroupId, properties);
table.getPartitionFilePathInfo(), table.getPartitionFileCacheInfo(partitionId), shardGroupId,
properties);
for (long shardId : shardIds) {
Tablet tablet = new LakeTablet(shardId);
index.addTablet(tablet, tabletMeta);
Expand Down Expand Up @@ -2925,8 +2937,9 @@ private List<Long> chosenBackendIdBySeq(int replicationNum) throws DdlException
if (!CollectionUtils.isEmpty(chosenBackendIds)) {
return chosenBackendIds;
} else if (replicationNum > 1) {
throw new DdlException(String.format("Unable to find %d alive nodes on different hosts to create %d replicas",
replicationNum, replicationNum));
throw new DdlException(
String.format("Unable to find %d alive nodes on different hosts to create %d replicas",
replicationNum, replicationNum));
} else {
throw new DdlException("No alive nodes");
}
Expand Down Expand Up @@ -3895,7 +3908,7 @@ private void disableMaterializedViewForRenameTable(Database db, OlapTable olapTa
MaterializedView mv = (MaterializedView) db.getTable(mvId.getId());
if (mv != null) {
LOG.warn("Setting the materialized view {}({}) to invalid because " +
"the table {} was renamed.", mv.getName(), mv.getId(), olapTable.getName());
"the table {} was renamed.", mv.getName(), mv.getId(), olapTable.getName());
mv.setActive(false);
} else {
LOG.warn("Ignore materialized view {} does not exists", mvId);
Expand Down Expand Up @@ -4063,7 +4076,6 @@ public void modifyTableDynamicPartition(Database db, OlapTable table, Map<String
editLog.logDynamicPartition(info);
}


public void alterTableProperties(Database db, OlapTable table, Map<String, String> properties)
throws DdlException {
Map<String, String> logProperties = new HashMap<>(properties);
Expand Down Expand Up @@ -4245,7 +4257,8 @@ public void modifyTableInMemoryMeta(Database db, OlapTable table, Map<String, St
}

// The caller need to hold the db write lock
public void modifyTableConstraint(Database db, String tableName, Map<String, String> properties) throws DdlException {
public void modifyTableConstraint(Database db, String tableName, Map<String, String> properties)
throws DdlException {
Preconditions.checkArgument(db.isWriteLockHeldByCurrentThread());
Table table = db.getTable(tableName);
if (table == null) {
Expand Down Expand Up @@ -4868,7 +4881,8 @@ public void replayTruncateTable(TruncateTableInfo info) {
long indexId = mIndex.getId();
int schemaHash = olapTable.getSchemaHashByIndexId(indexId);
TabletMeta tabletMeta = new TabletMeta(db.getId(), olapTable.getId(),
partitionId, indexId, schemaHash, medium, olapTable.isCloudNativeTableOrMaterializedView());
partitionId, indexId, schemaHash, medium,
olapTable.isCloudNativeTableOrMaterializedView());
for (Tablet tablet : mIndex.getTablets()) {
long tabletId = tablet.getId();
invertedIndex.addTablet(tabletId, tabletMeta);
Expand Down Expand Up @@ -5308,5 +5322,4 @@ public void addOrReplaceAutoIncrementIdByTableId(Long tableId, Long id) {
tableIdToIncrementId.replace(tableId, id);
}
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -525,7 +525,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
Original file line number Diff line number Diff line change
Expand Up @@ -22,12 +22,14 @@
import com.starrocks.analysis.LiteralExpr;
import com.starrocks.catalog.Column;
import com.starrocks.catalog.Database;
import com.starrocks.catalog.ExternalOlapTable;
import com.starrocks.catalog.MaterializedView;
import com.starrocks.catalog.MysqlTable;
import com.starrocks.catalog.OlapTable;
import com.starrocks.catalog.Partition;
import com.starrocks.catalog.Table;
import com.starrocks.common.AnalysisException;
import com.starrocks.external.starrocks.TableMetaSyncer;
import com.starrocks.qe.ConnectContext;
import com.starrocks.sql.ast.DefaultValueExpr;
import com.starrocks.sql.ast.InsertStmt;
Expand Down Expand Up @@ -59,6 +61,10 @@ public static void analyze(InsertStmt insertStmt, ConnectContext session) {
Database database = MetaUtils.getDatabase(session, insertStmt.getTableName());
Table table = MetaUtils.getTable(session, insertStmt.getTableName());

if (table instanceof ExternalOlapTable) {
table = getOLAPExternalTableMeta(database, (ExternalOlapTable) table);
}

if (table instanceof MaterializedView && !insertStmt.isSystem()) {
throw new SemanticException(
"The data of '%s' cannot be inserted because '%s' is a materialized view," +
Expand Down Expand Up @@ -222,4 +228,23 @@ private static void checkStaticKeyPartitionInsert(InsertStmt insertStmt, Table t
}
}
}

private static ExternalOlapTable getOLAPExternalTableMeta(Database database, ExternalOlapTable externalOlapTable) {
// copy the table, and release database lock when synchronize table meta
ExternalOlapTable copiedTable = new ExternalOlapTable();
externalOlapTable.copyOnlyForQuery(copiedTable);
int lockTimes = 0;
while (database.isReadLockHeldByCurrentThread()) {
database.readUnlock();
lockTimes++;
}
try {
new TableMetaSyncer().syncTable(copiedTable);
} finally {
while (lockTimes-- > 0) {
database.readLock();
}
}
return copiedTable;
}
}

0 comments on commit b3656af

Please sign in to comment.