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 93056de commit 92b5cc7
Show file tree
Hide file tree
Showing 10 changed files with 318 additions and 209 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 @@ -221,6 +221,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
397 changes: 199 additions & 198 deletions fe/fe-core/src/main/java/com/starrocks/catalog/ExternalOlapTable.java

Large diffs are not rendered by default.

34 changes: 34 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 @@ -225,6 +225,40 @@ public OlapTable(long id, String tableName, List<Column> baseSchema, KeysType ke
this.relatedMaterializedViews = Sets.newConcurrentHashSet();
}

// 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 void setTableProperty(TableProperty tableProperty) {
this.tableProperty = tableProperty;
}
Expand Down
15 changes: 15 additions & 0 deletions fe/fe-core/src/main/java/com/starrocks/catalog/Partition.java
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,21 @@ public Partition(long id, String name,
this.distributionInfo = distributionInfo;
}

public Partition shallowCopy() {
Partition partition = new Partition();
partition.id = this.id;
partition.name = this.name;
partition.state = this.state;
partition.baseIndex = this.baseIndex;
partition.idToVisibleRollupIndex = Maps.newHashMap(this.idToVisibleRollupIndex);
partition.idToShadowIndex = Maps.newHashMap(this.idToShadowIndex);
partition.visibleVersion = this.visibleVersion;
partition.visibleVersionTime = this.visibleVersionTime;
partition.nextVersion = this.nextVersion;
partition.distributionInfo = this.distributionInfo;
return partition;
}

public void setIdForRestore(long id) {
this.id = id;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,14 @@ public RangePartitionInfo(List<Column> partitionColumns) {
this.isMultiColumnPartition = partitionColumns.size() > 1;
}

public RangePartitionInfo(RangePartitionInfo other) {
super(other.type);
this.partitionColumns = Lists.newArrayList(other.partitionColumns);
this.idToRange = Maps.newHashMap(other.idToRange);
this.idToTempRange = Maps.newHashMap(other.idToTempRange);
this.isMultiColumnPartition = partitionColumns.size() > 1;
}

@Override
public List<Column> getPartitionColumns() {
return partitionColumns;
Expand Down
13 changes: 13 additions & 0 deletions fe/fe-core/src/main/java/com/starrocks/catalog/TableProperty.java
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

package com.starrocks.catalog;

import com.google.common.base.Preconditions;
import com.google.common.collect.Maps;
import com.google.gson.annotations.SerializedName;
import com.starrocks.common.FeConstants;
Expand Down Expand Up @@ -80,6 +81,18 @@ public TableProperty(Map<String, String> properties) {
this.properties = properties;
}

public TableProperty copy() {
TableProperty newTableProperty = new TableProperty(Maps.newHashMap(this.properties));
try {
newTableProperty.gsonPostProcess();
} catch (IOException e) {
Preconditions.checkState(false, "gsonPostProcess shouldn't fail");
}
newTableProperty.hasDelete = this.hasDelete;
newTableProperty.hasForbitGlobalDict = this.hasDelete;
return newTableProperty;
}

public static boolean isSamePrefixProperties(Map<String, String> properties, String prefix) {
for (String value : properties.keySet()) {
if (!value.startsWith(prefix)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@

import com.starrocks.catalog.ExternalOlapTable;
import com.starrocks.common.Config;
import com.starrocks.common.MetaNotFoundException;
import com.starrocks.rpc.FrontendServiceProxy;
import com.starrocks.thrift.TGetTableMetaRequest;
import com.starrocks.thrift.TGetTableMetaResponse;
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());
}
}
};
}
7 changes: 6 additions & 1 deletion fe/fe-core/src/main/java/com/starrocks/qe/StmtExecutor.java
Original file line number Diff line number Diff line change
Expand Up @@ -1101,7 +1101,12 @@ public void handleDMLStmt(ExecPlan execPlan, DmlStmt stmt) throws Exception {

MetaUtils.normalizationTableName(context, stmt.getTableName());
Database database = MetaUtils.getStarRocks(context, stmt.getTableName());
Table targetTable = MetaUtils.getStarRocksTable(context, stmt.getTableName());
Table targetTable;
if (stmt instanceof InsertStmt && ((InsertStmt) stmt).getTargetTable() != null) {
targetTable = ((InsertStmt) stmt).getTargetTable();
} else {
targetTable = MetaUtils.getStarRocksTable(context, stmt.getTableName());
}

String label = DebugUtil.printId(context.getExecutionId());
if (stmt instanceof InsertStmt) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -298,8 +298,8 @@ public class GlobalStateMgr {
private Daemon timePrinter;
private Daemon listener;
private EsRepository esRepository; // it is a daemon, so add it here
private StarRocksRepository starRocksRepository;
private HiveRepository hiveRepository;
private IcebergRepository icebergRepository;
private MetastoreEventsProcessor metastoreEventsProcessor;
private IcebergRepository icebergRepository;

Expand Down Expand Up @@ -510,7 +510,6 @@ private GlobalStateMgr(boolean isCheckpointCatalog) {
this.workGroupMgr = new WorkGroupMgr(this);

this.esRepository = new EsRepository();
this.starRocksRepository = new StarRocksRepository();
this.hiveRepository = new HiveRepository();
this.icebergRepository = new IcebergRepository();
this.metastoreEventsProcessor = new MetastoreEventsProcessor(hiveRepository);
Expand Down Expand Up @@ -1044,7 +1043,6 @@ private void startNonMasterDaemonThreads() {
labelCleaner.start();
// ES state store
esRepository.start();
starRocksRepository.start();

if (Config.enable_hms_events_incremental_sync) {
// load hive table to event processor and start to process hms events.
Expand Down Expand Up @@ -1122,7 +1120,6 @@ public void loadImage(String imageDir) throws IOException, DdlException {
localMetastore.recreateTabletInvertIndex();
// rebuild es state state
esRepository.loadTableFromCatalog();
starRocksRepository.loadTableFromCatalog();

checksum = load.loadLoadJob(dis, checksum);
checksum = loadAlterJob(dis, checksum);
Expand Down Expand Up @@ -2476,14 +2473,14 @@ public EsRepository getEsRepository() {
return this.esRepository;
}

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

public HiveRepository getHiveRepository() {
return this.hiveRepository;
}

public IcebergRepository getIcebergRepository() {
return this.icebergRepository;
}

public MetastoreEventsProcessor getMetastoreEventsProcessor() {
return this.metastoreEventsProcessor;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,11 +10,15 @@
import com.starrocks.analysis.PartitionNames;
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.PartitionType;
import com.starrocks.catalog.Table;
import com.starrocks.common.MetaNotFoundException;
import com.starrocks.external.starrocks.TableMetaSyncer;
import com.starrocks.qe.ConnectContext;
import com.starrocks.sql.ast.QueryRelation;
import com.starrocks.sql.ast.ValuesRelation;
Expand All @@ -39,6 +43,10 @@ public static void analyze(InsertStmt insertStmt, ConnectContext session) {
Database database = MetaUtils.getStarRocks(session, insertStmt.getTableName());
Table table = MetaUtils.getStarRocksTable(session, insertStmt.getTableName());

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

if (!(table instanceof OlapTable) && !(table instanceof MysqlTable)) {
throw unsupportedException("Only support insert into olap table or mysql table");
}
Expand Down Expand Up @@ -137,4 +145,25 @@ public static void analyze(InsertStmt insertStmt, ConnectContext session) {
insertStmt.setTargetColumns(targetColumns);
session.getDumpInfo().addTable(database.getFullName().split(":")[1], table);
}

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);
} catch (MetaNotFoundException e) {
throw new SemanticException(e.getMessage());
} finally {
while (lockTimes-- > 0) {
database.readLock();
}
}
return copiedTable;
}
}

0 comments on commit 92b5cc7

Please sign in to comment.