From ecf8cc66aba52a4771b60106812d88536ff2710b Mon Sep 17 00:00:00 2001 From: maochongxin Date: Thu, 19 Sep 2024 19:20:42 +0800 Subject: [PATCH 01/15] Optimize SQL for refreshing table location information --- .../oceanbase/rpc/location/LocationUtil.java | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java index 20d657e0..8ed8165f 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java @@ -154,13 +154,19 @@ public class LocationUtil { + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " + "WHERE tenant_name = ? and database_name=? and table_name = ? and tablet_id = 0"; - private static final String PROXY_LOCATION_SQL_PARTITION_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.tablet_id as tablet_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " - + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " - + ", A.spare1 as replica_type, D.ls_id as ls_id " - + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " - + "inner join oceanbase.DBA_OB_TENANTS C on C.tenant_name = A.tenant_name " - + "left join oceanbase.CDB_OB_TABLET_TO_LS D on D.tenant_id = C.tenant_id and D.tablet_id = A.tablet_id " - + "WHERE C.tenant_name = ? and database_name= ? and table_name = ? and A.tablet_id in ({0}) "; + private static final String PROXY_LOCATION_SQL_PARTITION_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ * FROM ( " + + " SELECT A.tablet_id as tablet__id, A.svr_ip as svr_ip, A.sql_port as sql_port, A.table_id as table_id, " + + " A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, " + + " B.stop_time as stop_time, A.spare1 as replica_type " + + " FROM oceanbase.__all_virtual_proxy_schema A " + + " INNER JOIN oceanbase.__all_server B ON A.svr_ip = B.svr_ip AND A.sql_port = B.inner_port " + + " WHERE A.tablet_id IN ({0}) AND A.tenant_name = ? AND A.database_name = ? AND A.table_name = ?) AS left_table " + + "LEFT JOIN (" + + " SELECT D.ls_id, D.tablet_id " + + " FROM oceanbase.__all_virtual_tablet_to_ls D " + + " INNER JOIN oceanbase.DBA_OB_TENANTS C ON D.tenant_id = C.tenant_id " + + " WHERE C.tenant_name = ? " + + ") AS right_table ON left_table.tablet__id = right_table.tablet_id;"; private static final String PROXY_FIRST_PARTITION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_id, part_name, tablet_id, high_bound_val, sub_part_num " + "FROM oceanbase.__all_virtual_proxy_partition " @@ -797,6 +803,7 @@ public static TableEntry getTableEntryLocationFromRemote(Connection connection, ps.setString(1, key.getTenantName()); ps.setString(2, key.getDatabaseName()); ps.setString(3, key.getTableName()); + ps.setString(4, key.getTenantName()); rs = ps.executeQuery(); partitionEntry = getPartitionLocationFromResultSet(tableEntry, rs, partitionEntry); } catch (Exception e) { From a82d5d0fa962b77dab1322cae1262b59a9ecf29d Mon Sep 17 00:00:00 2001 From: maochongxin Date: Tue, 24 Sep 2024 21:03:04 +0800 Subject: [PATCH 02/15] partical refresh --- .../alipay/oceanbase/rpc/ObTableClient.java | 235 ++++++---- .../oceanbase/rpc/location/LocationUtil.java | 420 +++++++++++------- .../rpc/location/model/TableEntry.java | 26 +- .../model/partition/ObPartitionEntry.java | 39 +- .../partition/ObPartitionLocationInfo.java | 76 ++++ .../ObTableClientQueryAsyncStreamResult.java | 12 +- .../rpc/table/ObTableClientBatchOpsImpl.java | 14 +- 7 files changed, 571 insertions(+), 251 deletions(-) create mode 100644 src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionLocationInfo.java diff --git a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java index efd49dc5..2677dcbf 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java +++ b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java @@ -1241,8 +1241,7 @@ public TableEntry getOrRefreshTableEntry(final String tableName, final boolean r // the server roster is ordered by priority long punishInterval = (long) (tableEntryRefreshIntervalBase * Math.pow(2, -serverRoster.getMaxPriority())); - punishInterval = punishInterval <= tableEntryRefreshIntervalCeiling ? punishInterval - : tableEntryRefreshIntervalCeiling; + punishInterval = Math.min(punishInterval, tableEntryRefreshIntervalCeiling); // control refresh frequency less than 100 milli second // just in case of connecting to OB Server failed or change master long interval = System.currentTimeMillis() - tableEntry.getRefreshTimeMills(); @@ -1267,8 +1266,7 @@ public TableEntry getOrRefreshTableEntry(final String tableName, final boolean r } int serverSize = serverRoster.getMembers().size(); - int refreshTryTimes = tableEntryRefreshTryTimes > serverSize ? serverSize - : tableEntryRefreshTryTimes; + int refreshTryTimes = Math.min(tableEntryRefreshTryTimes, serverSize); for (int i = 0; i < refreshTryTimes; i++) { try { @@ -1330,6 +1328,69 @@ private TableEntry refreshTableEntry(TableEntry tableEntry, String tableName) return refreshTableEntry(tableEntry, tableName, false); } + public TableEntry refreshTableLocationByTabletId(TableEntry tableEntry, String tableName, Long tabletId) throws ObTableAuthException { + TableEntryKey tableEntryKey = new TableEntryKey(clusterName, tenantName, database, tableName); + Lock lock = null; + try { + // if table entry is null, throw exception + if (tableEntry == null) { + throw new ObTableEntryRefreshException("table entry is null, tableName=" + tableName); + } + + long lastRefreshTime = tableEntry.getTabletLocationLastRefreshTimeMills(tabletId); + long currentTime = System.currentTimeMillis(); + // 100ms + if (currentTime - lastRefreshTime < 100) { + return tableEntry; + } + Lock tempLock = new ReentrantLock(); + lock = tableEntry.refreshLockMap.putIfAbsent(tabletId, tempLock); + lock = (lock == null) ? tempLock : lock; // check the first lock + + boolean acquired = lock.tryLock(tableEntryRefreshLockTimeout, TimeUnit.MILLISECONDS); + tableEntry.setTableLocationLastRefreshTimeMills(tabletId, currentTime); + + if (!acquired) { + String errMsg = "try to lock table-entry refreshing timeout " + "dataSource:" + + dataSourceName + " ,tableName:" + tableName + ", refresh:" + + " , timeout:" + tableEntryRefreshLockTimeout + "."; + RUNTIME.error(errMsg); + throw new ObTableEntryRefreshException(errMsg); + } + tableEntry = loadTableEntryLocationWithPriority(serverRoster, // + tableEntryKey,// + tableEntry,// + tabletId, + tableEntryAcquireConnectTimeout,// + tableEntryAcquireSocketTimeout,// + serverAddressPriorityTimeout, // + serverAddressCachingTimeout, sysUA); + // prepare the table entry for weak read. + tableEntry.prepareForWeakRead(serverRoster.getServerLdcLocation()); + } catch (ObTableNotExistException | ObTableServerCacheExpiredException e) { + RUNTIME.error("refreshTableEntry meet exception", e); + throw e; + } catch (Exception e) { + RUNTIME.error(LCD.convert("01-00020"), tableEntryKey, tableEntry, e); + throw new ObTableEntryRefreshException(String.format( + "failed to get table entry key=%s original tableEntry=%s ", tableEntryKey, + tableEntry), e); + } finally { + if (lock != null) { + lock.unlock(); + } + } + tableLocations.put(tableName, tableEntry); + + tableEntryRefreshContinuousFailureCount.set(0); + if (logger.isInfoEnabled()) { + logger.info( + "refresh table entry, dataSource: {}, tableName: {}, refresh: {} key:{} entry:{} ", + dataSourceName, tableName, true, tableEntryKey, JSON.toJSON(tableEntry)); + } + return tableEntry; + } + /** * 刷新 table entry 元数据 * @param tableEntry @@ -1345,13 +1406,7 @@ private TableEntry refreshTableEntry(TableEntry tableEntry, String tableName, bo try { // if table entry is exist we just need to refresh table locations if (tableEntry != null && !fetchAll) { - tableEntry = loadTableEntryLocationWithPriority(serverRoster, // - tableEntryKey,// - tableEntry,// - tableEntryAcquireConnectTimeout,// - tableEntryAcquireSocketTimeout,// - serverAddressPriorityTimeout, // - serverAddressCachingTimeout, sysUA); + // do nothing } else { // if table entry is not exist we should fetch partition info and table locations tableEntry = loadTableEntryWithPriority(serverRoster, // @@ -1360,7 +1415,6 @@ private TableEntry refreshTableEntry(TableEntry tableEntry, String tableName, bo tableEntryAcquireSocketTimeout,// serverAddressPriorityTimeout,// serverAddressCachingTimeout, sysUA); - if (tableEntry.isPartitionTable()) { switch (runningMode) { case HBASE: @@ -1544,17 +1598,22 @@ private ObPair getPartitionReplica(TableEntry tableEntry, */ private ReplicaLocation getPartitionLocation(TableEntry tableEntry, long partId, ObServerRoute route) { - if (ObGlobal.obVsnMajor() >= 4 && tableEntry.isPartitionTable()) { - ObPartitionInfo partInfo = tableEntry.getPartitionInfo(); - Map tabletIdMap = partInfo.getPartTabletIdMap(); - long partIdx = tableEntry.getPartIdx(partId); - long TabletId = tabletIdMap.get(partIdx); - return tableEntry.getPartitionEntry().getPartitionLocationWithTabletId(TabletId) - .getReplica(route); - } else { - return tableEntry.getPartitionEntry().getPartitionLocationWithPartId(partId) - .getReplica(route); + long tabletId = getTabletIdByPartId(tableEntry, partId); + return tableEntry.getPartitionEntry().getPartitionLocationWithTabletId(tabletId) + .getReplica(route); + + } + + private ReplicaLocation getPartitionLocation(ObPartitionLocationInfo obPartitionLocationInfo, + ObServerRoute route) { + while (obPartitionLocationInfo.getPartitionLocation() == null) { + try { + Thread.sleep(10); + } catch (InterruptedException e) { + e.printStackTrace(); + } } + return obPartitionLocationInfo.getPartitionLocation().getReplica(route); } /** @@ -1784,11 +1843,14 @@ public ObTable addTable(ObServerAddr addr){ public ObPair getTableInternal(String tableName, TableEntry tableEntry, long partId, boolean waitForRefresh, ObServerRoute route) throws Exception { - ObPair partitionReplica = getPartitionReplica(tableEntry, partId, - route); - - ReplicaLocation replica = partitionReplica.getRight(); - + long tabletId = getTabletIdByPartId(tableEntry, partId); + ObPartitionLocationInfo obPartitionLocationInfo = tableEntry.getPartitionEntry().getPartitionInfo(tabletId); + if (!obPartitionLocationInfo.initialized.get()) { + tableEntry = refreshTableLocationByTabletId(tableEntry, tableName, tabletId); + obPartitionLocationInfo.initialized.compareAndSet(false, true); + } + + ReplicaLocation replica = getPartitionLocation(obPartitionLocationInfo, route); ObServerAddr addr = replica.getAddr(); ObTable obTable = tableRoster.get(addr); boolean addrExpired = addr.isExpired(serverAddressCachingTimeout); @@ -1801,8 +1863,9 @@ public ObPair getTableInternal(String tableName, TableEntry logger.info("server addr {} is expired, refresh tableEntry.", addr); } - tableEntry = getOrRefreshTableEntry(tableName, true, waitForRefresh, false); - replica = getPartitionReplica(tableEntry, partId, route).getRight(); + tableEntry = refreshTableLocationByTabletId(tableEntry, tableName, tabletId); + obPartitionLocationInfo = tableEntry.getPartitionEntry().getPartitionInfo(tabletId); + replica = getPartitionLocation(obPartitionLocationInfo, route); addr = replica.getAddr(); obTable = tableRoster.get(addr); } @@ -1815,17 +1878,14 @@ public ObPair getTableInternal(String tableName, TableEntry ObTableParam param = new ObTableParam(obTable); param.setPartId(partId); // used in getTable(), 4.x may change the origin partId if (ObGlobal.obVsnMajor() >= 4 && tableEntry != null) { - long partIdx = tableEntry.getPartIdx(partId); - partId = tableEntry.isPartitionTable() ? tableEntry.getPartitionInfo() - .getPartTabletIdMap().get(partIdx) : partId; - param.setLsId(tableEntry.getPartitionEntry().getLsId(partId)); + param.setLsId(obPartitionLocationInfo.getTabletLsId()); } param.setTableId(tableEntry.getTableId()); - param.setPartitionId(partId); + param.setPartitionId(tabletId); addr.recordAccess(); - return new ObPair(partitionReplica.getLeft(), param); + return new ObPair<>(tabletId, param); } /** @@ -1840,39 +1900,66 @@ public ObPair getTableInternal(String tableName, TableEntry * @throws Exception */ private List> getPartitionReplica(TableEntry tableEntry, + String tableName, Row startRow, boolean startIncluded, Row endRow, boolean endIncluded, - ObServerRoute route) - throws Exception { - // non partition - List> replicas = new ArrayList>(); - if (!tableEntry.isPartitionTable() - || tableEntry.getPartitionInfo().getLevel() == ObPartitionLevel.LEVEL_ZERO) { - replicas.add(new ObPair(0L, getPartitionLocation(tableEntry, 0L, - route))); - return replicas; - } else if (tableEntry.getPartitionInfo().getLevel() == ObPartitionLevel.LEVEL_ONE) { - List partIds = tableEntry.getPartitionInfo().getFirstPartDesc() - .getPartIds(startRow, startIncluded, endRow, endIncluded); - for (Long partId : partIds) { - replicas.add(new ObPair(partId, getPartitionLocation( - tableEntry, partId, route))); + ObServerRoute route) throws Exception { + List> replicas = new ArrayList<>(); + + if (!tableEntry.isPartitionTable() || tableEntry.getPartitionInfo().getLevel() == ObPartitionLevel.LEVEL_ZERO) { + long tabletId = getTabletIdByPartId(tableEntry, 0L); + ObPartitionLocationInfo locationInfo = tableEntry.getPartitionEntry().getPartitionInfo(tabletId); + if (!locationInfo.initialized.get()) { + refreshTableLocationByTabletId(tableEntry, tableName, tabletId); + locationInfo.initialized.compareAndSet(false, true); } - } else if (tableEntry.getPartitionInfo().getLevel() == ObPartitionLevel.LEVEL_TWO) { - List partIds = getPartitionsForLevelTwo(tableEntry, startRow, startIncluded, - endRow, endIncluded); - for (Long partId : partIds) { - replicas.add(new ObPair(partId, getPartitionLocation( - tableEntry, partId, route))); + replicas.add(new ObPair<>(tabletId, getPartitionLocation(locationInfo, route))); + return replicas; + } + + ObPartitionLevel partitionLevel = tableEntry.getPartitionInfo().getLevel(); + List partIds = getPartitionTablePartitionIds(tableEntry, startRow, startIncluded, endRow, endIncluded, partitionLevel); + + for (Long partId : partIds) { + long tabletId = getTabletIdByPartId(tableEntry, partId); + ObPartitionLocationInfo locationInfo = tableEntry.getPartitionEntry().getPartitionInfo(tabletId); + if (!locationInfo.initialized.get()) { + refreshTableLocationByTabletId(tableEntry, tableName, tabletId); + locationInfo.initialized.compareAndSet(false, true); } + replicas.add(new ObPair<>(tabletId, getPartitionLocation(locationInfo, route))); + } + + return replicas; + } + + private List getPartitionTablePartitionIds(TableEntry tableEntry, + Row startRow, boolean startIncluded, + Row endRow, boolean endIncluded, + ObPartitionLevel level) + throws Exception { + if (level == ObPartitionLevel.LEVEL_ONE) { + return tableEntry.getPartitionInfo().getFirstPartDesc() + .getPartIds(startRow, startIncluded, endRow, endIncluded); + } else if (level == ObPartitionLevel.LEVEL_TWO) { + return getPartitionsForLevelTwo(tableEntry, startRow, startIncluded, + endRow, endIncluded); } else { RUNTIME.error("not allowed bigger than level two"); throw new ObTableGetException("not allowed bigger than level two"); } + } - return replicas; + private long getTabletIdByPartId(TableEntry tableEntry, Long partId) { + if (ObGlobal.obVsnMajor() >= 4 && tableEntry.isPartitionTable()) { + ObPartitionInfo partInfo = tableEntry.getPartitionInfo(); + Map tabletIdMap = partInfo.getPartTabletIdMap(); + long partIdx = tableEntry.getPartIdx(partId); + return tabletIdMap.getOrDefault(partIdx, partId); + } + return partId; } /** @@ -1947,28 +2034,28 @@ public List> getTables(String tableName, ObTableQuery } } - List> partIdWithReplicaList = getPartitionReplica(tableEntry, + List> partIdWithReplicaList = getPartitionReplica(tableEntry, tableName, startRow, startInclusive, endRow, endInclusive, route); // obTableParams -> List> List> obTableParams = new ArrayList>(); for (ObPair partIdWithReplica : partIdWithReplicaList) { - long partId = partIdWithReplica.getLeft(); + long tabletId = partIdWithReplica.getLeft(); ReplicaLocation replica = partIdWithReplica.getRight(); ObServerAddr addr = replica.getAddr(); ObTable obTable = tableRoster.get(addr); boolean addrExpired = addr.isExpired(serverAddressCachingTimeout); - if (addrExpired || obTable == null) { - logger - .warn( - "server address {} is expired={} or can not get ob table. So that will sync refresh metadata", - addr, addrExpired); - syncRefreshMetadata(); - tableEntry = getOrRefreshTableEntry(tableName, true, waitForRefresh, false); - replica = getPartitionLocation(tableEntry, partId, route); - addr = replica.getAddr(); - obTable = tableRoster.get(addr); - } +// if (addrExpired || obTable == null) { +// logger +// .warn( +// "server address {} is expired={} or can not get ob table. So that will sync refresh metadata", +// addr, addrExpired); +// syncRefreshMetadata(); +// tableEntry = getOrRefreshTableEntry(tableName, true, waitForRefresh, false); +// replica = getPartitionLocation(tableEntry, partId, route); +// addr = replica.getAddr(); +// obTable = tableRoster.get(addr); +// } if (obTable == null) { RUNTIME.error("cannot get table by addr: " + addr); @@ -1976,15 +2063,9 @@ public List> getTables(String tableName, ObTableQuery } ObTableParam param = new ObTableParam(obTable); - if (ObGlobal.obVsnMajor() >= 4) { - long partIdx = tableEntry.getPartIdx(partId); - partId = tableEntry.isPartitionTable() ? tableEntry.getPartitionInfo() - .getPartTabletIdMap().get(partIdx) : partId; - } - param.setTableId(tableEntry.getTableId()); // real partition(tablet) id - param.setPartitionId(partId); + param.setPartitionId(tabletId); addr.recordAccess(); obTableParams.add(new ObPair(partIdWithReplica.getLeft(), param)); diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java index 8ed8165f..2fa12ab0 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java @@ -56,141 +56,156 @@ public class LocationUtil { - private static final Logger logger = TableClientLoggerFactory - .getLogger(LocationUtil.class); + private static final Logger logger = TableClientLoggerFactory + .getLogger(LocationUtil.class); static { ParserConfig.getGlobalInstance().setSafeMode(true); } - private static final String OB_VERSION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ OB_VERSION() AS CLUSTER_VERSION;"; + private static final String OB_VERSION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ OB_VERSION() AS CLUSTER_VERSION;"; - private static final String PROXY_INDEX_INFO_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ data_table_id, table_id, index_type FROM oceanbase.__all_virtual_table " - + "where table_name = ?"; + private static final String PROXY_INDEX_INFO_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ data_table_id, table_id, index_type FROM oceanbase.__all_virtual_table " + + "where table_name = ?"; - private static final String PROXY_TABLE_ID_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ table_id from oceanbase.__all_virtual_proxy_schema " - + "where tenant_name = ? and database_name = ? and table_name = ? limit 1"; + private static final String PROXY_TABLE_ID_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ table_id from oceanbase.__all_virtual_proxy_schema " + + "where tenant_name = ? and database_name = ? and table_name = ? limit 1"; - private static final String OB_TENANT_EXIST_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ tenant_id from __all_tenant where tenant_name = ?;"; + private static final String OB_TENANT_EXIST_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ tenant_id from __all_tenant where tenant_name = ?;"; @Deprecated @SuppressWarnings("unused") - private static final String PROXY_PLAIN_SCHEMA_SQL_FORMAT = "SELECT /*+READ_CONSISTENCY(WEAK)*/ partition_id, svr_ip, sql_port, table_id, role, part_num, replica_num, schema_version, spare1 " - + "FROM oceanbase.__all_virtual_proxy_schema " - + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND partition_id in ({0}) AND sql_port > 0 " - + "ORDER BY role ASC LIMIT ?"; - - private static final String PROXY_PART_INFO_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_level, part_num, part_type, part_space, part_expr, " - + "part_range_type, part_interval_bin, interval_start_bin, " - + "sub_part_num, sub_part_type, sub_part_space, " - + "sub_part_range_type, def_sub_part_interval_bin, def_sub_interval_start_bin, sub_part_expr, " - + "part_key_name, part_key_type, part_key_idx, part_key_extra, spare1 " - + "FROM oceanbase.__all_virtual_proxy_partition_info " - + "WHERE table_id = ? group by part_key_name order by part_key_name LIMIT ?;"; + private static final String PROXY_PLAIN_SCHEMA_SQL_FORMAT = "SELECT /*+READ_CONSISTENCY(WEAK)*/ partition_id, svr_ip, sql_port, table_id, role, part_num, replica_num, schema_version, spare1 " + + "FROM oceanbase.__all_virtual_proxy_schema " + + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND partition_id in ({0}) AND sql_port > 0 " + + "ORDER BY role ASC LIMIT ?"; + + private static final String PROXY_PART_INFO_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_level, part_num, part_type, part_space, part_expr, " + + "part_range_type, part_interval_bin, interval_start_bin, " + + "sub_part_num, sub_part_type, sub_part_space, " + + "sub_part_range_type, def_sub_part_interval_bin, def_sub_interval_start_bin, sub_part_expr, " + + "part_key_name, part_key_type, part_key_idx, part_key_extra, spare1 " + + "FROM oceanbase.__all_virtual_proxy_partition_info " + + "WHERE table_id = ? group by part_key_name order by part_key_name LIMIT ?;"; @Deprecated @SuppressWarnings("unused") - private static final String PROXY_TENANT_SCHEMA_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ svr_ip, sql_port, table_id, role, part_num, replica_num, spare1 " - + "FROM oceanbase.__all_virtual_proxy_schema " - + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND sql_port > 0 " - + "ORDER BY partition_id ASC, role ASC LIMIT ?"; - - private static final String PROXY_DUMMY_LOCATION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.partition_id as partition_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " - + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " - + ", A.spare1 as replica_type " - + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " - + "WHERE tenant_name = ? and database_name=? and table_name = ?"; - - private static final String PROXY_LOCATION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.partition_id as partition_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " - + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " - + ", A.spare1 as replica_type " - + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " - + "WHERE tenant_name = ? and database_name=? and table_name = ? and partition_id = 0"; - - private static final String PROXY_LOCATION_SQL_PARTITION = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.partition_id as partition_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " - + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " - + ", A.spare1 as replica_type " - + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " - + "WHERE tenant_name = ? and database_name=? and table_name = ? and partition_id in ({0})"; - - private static final String PROXY_FIRST_PARTITION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_id, part_name, high_bound_val " - + "FROM oceanbase.__all_virtual_proxy_partition " - + "WHERE table_id = ? LIMIT ?;"; - - private static final String PROXY_SUB_PARTITION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ sub_part_id, part_name, high_bound_val " - + "FROM oceanbase.__all_virtual_proxy_sub_partition " - + "WHERE table_id = ? LIMIT ?;"; - - private static final String PROXY_SERVER_STATUS_INFO = "SELECT ss.svr_ip, ss.zone, zs.region, zs.spare4 as idc " - + "FROM oceanbase.__all_virtual_proxy_server_stat ss, oceanbase.__all_virtual_zone_stat zs " - + "WHERE zs.zone = ss.zone ;"; + private static final String PROXY_TENANT_SCHEMA_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ svr_ip, sql_port, table_id, role, part_num, replica_num, spare1 " + + "FROM oceanbase.__all_virtual_proxy_schema " + + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND sql_port > 0 " + + "ORDER BY partition_id ASC, role ASC LIMIT ?"; + + private static final String PROXY_DUMMY_LOCATION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.partition_id as partition_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " + + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " + + ", A.spare1 as replica_type " + + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " + + "WHERE tenant_name = ? and database_name=? and table_name = ?"; + + private static final String PROXY_LOCATION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.partition_id as partition_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " + + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " + + ", A.spare1 as replica_type " + + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " + + "WHERE tenant_name = ? and database_name=? and table_name = ? and partition_id = 0"; + + private static final String PROXY_LOCATION_SQL_PARTITION = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.partition_id as partition_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " + + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " + + ", A.spare1 as replica_type " + + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " + + "WHERE tenant_name = ? and database_name=? and table_name = ? and partition_id in ({0})"; + + private static final String PROXY_FIRST_PARTITION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_id, part_name, high_bound_val " + + "FROM oceanbase.__all_virtual_proxy_partition " + + "WHERE table_id = ? LIMIT ?;"; + + private static final String PROXY_SUB_PARTITION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ sub_part_id, part_name, high_bound_val " + + "FROM oceanbase.__all_virtual_proxy_sub_partition " + + "WHERE table_id = ? LIMIT ?;"; + + private static final String PROXY_SERVER_STATUS_INFO = "SELECT ss.svr_ip, ss.zone, zs.region, zs.spare4 as idc " + + "FROM oceanbase.__all_virtual_proxy_server_stat ss, oceanbase.__all_virtual_zone_stat zs " + + "WHERE zs.zone = ss.zone ;"; @Deprecated @SuppressWarnings("unused") - private static final String PROXY_PLAIN_SCHEMA_SQL_FORMAT_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ tablet_id, svr_ip, sql_port, table_id, role, part_num, replica_num, schema_version, spare1 " - + "FROM oceanbase.__all_virtual_proxy_schema " - + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND tablet_id in ({0}) AND sql_port > 0 " - + "ORDER BY role ASC LIMIT ?"; - - private static final String PROXY_PART_INFO_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_level, part_num, part_type, part_space, part_expr, " - + "part_range_type, sub_part_num, sub_part_type, sub_part_space, sub_part_range_type, sub_part_expr, " - + "part_key_name, part_key_type, part_key_idx, part_key_extra, part_key_collation_type " - + "FROM oceanbase.__all_virtual_proxy_partition_info " - + "WHERE tenant_name = ? and table_id = ? group by part_key_name order by part_key_name LIMIT ?;"; + private static final String PROXY_PLAIN_SCHEMA_SQL_FORMAT_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ tablet_id, svr_ip, sql_port, table_id, role, part_num, replica_num, schema_version, spare1 " + + "FROM oceanbase.__all_virtual_proxy_schema " + + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND tablet_id in ({0}) AND sql_port > 0 " + + "ORDER BY role ASC LIMIT ?"; + + private static final String PROXY_PART_INFO_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_level, part_num, part_type, part_space, part_expr, " + + "part_range_type, sub_part_num, sub_part_type, sub_part_space, sub_part_range_type, sub_part_expr, " + + "part_key_name, part_key_type, part_key_idx, part_key_extra, part_key_collation_type " + + "FROM oceanbase.__all_virtual_proxy_partition_info " + + "WHERE tenant_name = ? and table_id = ? group by part_key_name order by part_key_name LIMIT ?;"; @Deprecated @SuppressWarnings("unused") - private static final String PROXY_TENANT_SCHEMA_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ svr_ip, sql_port, table_id, role, part_num, replica_num, spare1 " - + "FROM oceanbase.__all_virtual_proxy_schema " - + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND sql_port > 0 " - + "ORDER BY tablet_id ASC, role ASC LIMIT ?"; - - private static final String PROXY_DUMMY_LOCATION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.tablet_id as tablet_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " - + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " - + ", A.spare1 as replica_type " - + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " - + "WHERE tenant_name = ? and database_name=? and table_name = ?"; - - private static final String PROXY_LOCATION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.tablet_id as tablet_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " - + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " - + ", A.spare1 as replica_type " - + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " - + "WHERE tenant_name = ? and database_name=? and table_name = ? and tablet_id = 0"; - - private static final String PROXY_LOCATION_SQL_PARTITION_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ * FROM ( " - + " SELECT A.tablet_id as tablet__id, A.svr_ip as svr_ip, A.sql_port as sql_port, A.table_id as table_id, " - + " A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, " - + " B.stop_time as stop_time, A.spare1 as replica_type " - + " FROM oceanbase.__all_virtual_proxy_schema A " - + " INNER JOIN oceanbase.__all_server B ON A.svr_ip = B.svr_ip AND A.sql_port = B.inner_port " - + " WHERE A.tablet_id IN ({0}) AND A.tenant_name = ? AND A.database_name = ? AND A.table_name = ?) AS left_table " - + "LEFT JOIN (" - + " SELECT D.ls_id, D.tablet_id " - + " FROM oceanbase.__all_virtual_tablet_to_ls D " - + " INNER JOIN oceanbase.DBA_OB_TENANTS C ON D.tenant_id = C.tenant_id " - + " WHERE C.tenant_name = ? " - + ") AS right_table ON left_table.tablet__id = right_table.tablet_id;"; - - private static final String PROXY_FIRST_PARTITION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_id, part_name, tablet_id, high_bound_val, sub_part_num " - + "FROM oceanbase.__all_virtual_proxy_partition " - + "WHERE tenant_name = ? and table_id = ? LIMIT ?;"; - - private static final String PROXY_SUB_PARTITION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ sub_part_id, part_name, tablet_id, high_bound_val " - + "FROM oceanbase.__all_virtual_proxy_sub_partition " - + "WHERE tenant_name = ? and table_id = ? LIMIT ?;"; - - private static final String PROXY_SERVER_STATUS_INFO_V4 = "SELECT ss.svr_ip, ss.zone, zs.region, zs.idc as idc " - + "FROM DBA_OB_SERVERS ss, DBA_OB_ZONES zs " - + "WHERE zs.zone = ss.zone ;"; - - private static final String home = System.getProperty("user.home", - "/home/admin"); - - private static final String TABLE_GROUP_GET_TABLE_NAME_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ table_name " - + "FROM oceanbase.CDB_OB_TABLEGROUP_TABLES " - + "WHERE tablegroup_name = ? and tenant_id = ? limit 1;"; - - private static final int TEMPLATE_PART_ID = -1; + private static final String PROXY_TENANT_SCHEMA_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ svr_ip, sql_port, table_id, role, part_num, replica_num, spare1 " + + "FROM oceanbase.__all_virtual_proxy_schema " + + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND sql_port > 0 " + + "ORDER BY tablet_id ASC, role ASC LIMIT ?"; + + private static final String PROXY_DUMMY_LOCATION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.tablet_id as tablet_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " + + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " + + ", A.spare1 as replica_type " + + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " + + "WHERE tenant_name = ? and database_name=? and table_name = ?"; + + private static final String PROXY_LOCATION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.tablet_id as tablet_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " + + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " + + ", A.spare1 as replica_type " + + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " + + "WHERE tenant_name = ? and database_name=? and table_name = ? and tablet_id = 0"; + + private static final String PROXY_LOCATION_SQL_PARTITION_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ * FROM ( " + + " SELECT A.tablet_id as tablet__id, A.svr_ip as svr_ip, A.sql_port as sql_port, A.table_id as table_id, " + + " A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, " + + " B.stop_time as stop_time, A.spare1 as replica_type " + + " FROM oceanbase.__all_virtual_proxy_schema A " + + " INNER JOIN oceanbase.__all_server B ON A.svr_ip = B.svr_ip AND A.sql_port = B.inner_port " + + " WHERE A.tablet_id IN ({0}) AND A.tenant_name = ? AND A.database_name = ? AND A.table_name = ?) AS left_table " + + "LEFT JOIN (" + + " SELECT D.ls_id, D.tablet_id " + + " FROM oceanbase.__all_virtual_tablet_to_ls D " + + " INNER JOIN oceanbase.DBA_OB_TENANTS C ON D.tenant_id = C.tenant_id " + + " WHERE C.tenant_name = ? " + + ") AS right_table ON left_table.tablet__id = right_table.tablet_id;"; + + private static final String PROXY_LOCATION_SQL_PARTITION_BY_TABLETID_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ * FROM ( " + + " SELECT A.tablet_id as tablet__id, A.svr_ip as svr_ip, A.sql_port as sql_port, A.table_id as table_id, " + + " A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, " + + " B.stop_time as stop_time, A.spare1 as replica_type " + + " FROM oceanbase.__all_virtual_proxy_schema A " + + " INNER JOIN oceanbase.__all_server B ON A.svr_ip = B.svr_ip AND A.sql_port = B.inner_port " + + " WHERE A.tablet_id = ? AND A.tenant_name = ? AND A.database_name = ? AND A.table_name = ?) AS left_table " + + "LEFT JOIN (" + + " SELECT D.ls_id, D.tablet_id " + + " FROM oceanbase.__all_virtual_tablet_to_ls D " + + " INNER JOIN oceanbase.DBA_OB_TENANTS C ON D.tenant_id = C.tenant_id " + + " WHERE C.tenant_name = ? " + + ") AS right_table ON left_table.tablet__id = right_table.tablet_id;"; + + private static final String PROXY_FIRST_PARTITION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_id, part_name, tablet_id, high_bound_val, sub_part_num " + + "FROM oceanbase.__all_virtual_proxy_partition " + + "WHERE tenant_name = ? and table_id = ? LIMIT ?;"; + + private static final String PROXY_SUB_PARTITION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ sub_part_id, part_name, tablet_id, high_bound_val " + + "FROM oceanbase.__all_virtual_proxy_sub_partition " + + "WHERE tenant_name = ? and table_id = ? LIMIT ?;"; + + private static final String PROXY_SERVER_STATUS_INFO_V4 = "SELECT ss.svr_ip, ss.zone, zs.region, zs.idc as idc " + + "FROM DBA_OB_SERVERS ss, DBA_OB_ZONES zs " + + "WHERE zs.zone = ss.zone ;"; + + private static final String home = System.getProperty( + "user.home", + "/home/admin"); + + private static final String TABLE_GROUP_GET_TABLE_NAME_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ table_name " + + "FROM oceanbase.CDB_OB_TABLEGROUP_TABLES " + + "WHERE tablegroup_name = ? and tenant_id = ? limit 1;"; + + private static final int TEMPLATE_PART_ID = -1; // limit the size of get tableEntry location from remote each time - private static final int MAX_TABLET_NUMS_EPOCH = 300; + private static final int MAX_TABLET_NUMS_EPOCH = 300; private abstract static class TableEntryRefreshWithPriorityCallback { abstract T execute(ObServerAddr obServerAddr) throws ObTableEntryRefreshException; @@ -463,6 +478,37 @@ TableEntry execute(Connection connection) }); } + /* + * Load table entry location with priority by tablet id. + */ + public static TableEntry loadTableEntryLocationWithPriority(final ServerRoster serverRoster, + final TableEntryKey key, + final TableEntry tableEntry, + final Long tabletId, + final long connectTimeout, + final long socketTimeout, + final long priorityTimeout, + final long cachingTimeout, + final ObUserAuth sysUA) + throws ObTableEntryRefreshException { + + return callTableEntryRefreshWithPriority(serverRoster, priorityTimeout, cachingTimeout, + new TableEntryRefreshWithPriorityCallback() { + @Override + TableEntry execute(ObServerAddr obServerAddr) throws ObTableEntryRefreshException { + return callTableEntryRefresh(obServerAddr, key, connectTimeout, socketTimeout, + sysUA, true, new TableEntryRefreshCallback() { + @Override + TableEntry execute(Connection connection) + throws ObTablePartitionLocationRefreshException { + return getTableEntryLocationFromRemote(connection, key, tableEntry, + tabletId); + } + }); + } + }); + } + /* * load Table Name With table Group */ @@ -689,8 +735,10 @@ private static TableEntry getTableEntryFromRemote(Connection connection, TableEn } } - // get location info - getTableEntryLocationFromRemote(connection, key, tableEntry); + // only set empty partitionEntry + ObPartitionEntry partitionEntry = new ObPartitionEntry(); + tableEntry.setPartitionEntry(partitionEntry); + tableEntry.setRefreshTimeMills(System.currentTimeMillis()); if (!initialized) { if (BOOT.isInfoEnabled()) { @@ -725,6 +773,16 @@ private static TableEntry getTableEntryFromRemote(Connection connection, TableEn return tableEntry; } + private static String genLocationSQLByTabletId() { + String sql = null; + if (ObGlobal.obVsnMajor() >= 4) { + sql = PROXY_LOCATION_SQL_PARTITION_BY_TABLETID_V4; + } else { + throw new FeatureNotSupportedException("not support ob version less than 4"); + } + return sql; + } + private static String genLocationSQLByOffset(TableEntry tableEntry, int offset, int size) { StringBuilder sb = new StringBuilder(); String sql = null; @@ -782,6 +840,45 @@ private static String genLocationSQLByOffset(TableEntry tableEntry, int offset, return sql; } + public static TableEntry getTableEntryLocationFromRemote(Connection connection, + TableEntryKey key, + TableEntry tableEntry, Long tabletId) + throws ObTablePartitionLocationRefreshException { + PreparedStatement ps = null; + ResultSet rs = null; + ObPartitionEntry partitionEntry = tableEntry.getPartitionEntry(); + String sql = genLocationSQLByTabletId(); + try { + ps = connection.prepareStatement(sql); + ps.setLong(1, tabletId); + ps.setString(2, key.getTenantName()); + ps.setString(3, key.getDatabaseName()); + ps.setString(4, key.getTableName()); + ps.setString(5, key.getTenantName()); + rs = ps.executeQuery(); + getPartitionLocationFromResultSetByTablet(tableEntry, rs, partitionEntry, tabletId); + } catch (Exception e) { + RUNTIME.error(LCD.convert("01-00010"), key, tableEntry, e); + throw new ObTablePartitionLocationRefreshException(format( + "fail to get partition location entry from remote entryKey = %s tableEntry =%s ", + key, tableEntry), e); + } finally { + try { + if (null != rs) { + rs.close(); + } + if (null != ps) { + ps.close(); + } + } catch (SQLException e) { + // ignore + } + } + // 可能不需要了 + tableEntry.setRefreshTimeMills(System.currentTimeMillis()); + return tableEntry; + } + /* * Get table entry location from remote. */ @@ -793,37 +890,6 @@ public static TableEntry getTableEntryLocationFromRemote(Connection connection, PreparedStatement ps = null; ResultSet rs = null; ObPartitionEntry partitionEntry = new ObPartitionEntry(); - long partitionNum = tableEntry.getPartitionNum(); - int epoch = (int) ((partitionNum / MAX_TABLET_NUMS_EPOCH) + 1); - for (int i = 0; i < epoch; i++) { - try { - int offset = i * MAX_TABLET_NUMS_EPOCH; - String sql = genLocationSQLByOffset(tableEntry, offset, MAX_TABLET_NUMS_EPOCH); - ps = connection.prepareStatement(sql); - ps.setString(1, key.getTenantName()); - ps.setString(2, key.getDatabaseName()); - ps.setString(3, key.getTableName()); - ps.setString(4, key.getTenantName()); - rs = ps.executeQuery(); - partitionEntry = getPartitionLocationFromResultSet(tableEntry, rs, partitionEntry); - } catch (Exception e) { - RUNTIME.error(LCD.convert("01-00010"), key, partitionNum, tableEntry, e); - throw new ObTablePartitionLocationRefreshException(format( - "fail to get partition location entry from remote entryKey = %s partNum = %d tableEntry =%s " - + "offset =%d epoch =%d", key, partitionNum, tableEntry, i, epoch), e); - } finally { - try { - if (null != rs) { - rs.close(); - } - if (null != ps) { - ps.close(); - } - } catch (SQLException e) { - // ignore - } - } - } // end for tableEntry.setPartitionEntry(partitionEntry); tableEntry.setRefreshTimeMills(System.currentTimeMillis()); return tableEntry; @@ -1088,6 +1154,62 @@ private static TableEntry getTableEntryFromResultSet(TableEntryKey key, ResultSe return entry; } + private static ObPartitionEntry getPartitionLocationFromResultSetByTablet(TableEntry tableEntry, + ResultSet rs, + ObPartitionEntry partitionEntry, + long tabletId) + throws SQLException, + ObTablePartitionLocationRefreshException { + if (partitionEntry == null || tableEntry == null) { + throw new IllegalArgumentException("partitionEntry: " + partitionEntry + + " tableEntry: " + tableEntry); + } + ObPartitionLocationInfo partitionLocationInfo = partitionEntry.getPartitionInfo(tabletId); + try { + partitionLocationInfo.rwLock.writeLock().lock(); + while (rs.next()) { + ReplicaLocation replica = buildReplicaLocation(rs); + long partitionId; + long lsId; + if (ObGlobal.obVsnMajor() >= 4) { + partitionId = rs.getLong("tablet_id"); + lsId = rs.getLong("ls_id"); + if (rs.wasNull()) { + lsId = INVALID_LS_ID; // non-partitioned table + } + partitionLocationInfo.setTabletLsId(lsId); + } else { + partitionId = rs.getLong("partition_id"); + if (tableEntry.isPartitionTable() + && null != tableEntry.getPartitionInfo().getSubPartDesc()) { + partitionId = ObPartIdCalculator.getPartIdx(partitionId, tableEntry + .getPartitionInfo().getSubPartDesc().getPartNum()); + } + } + if (!replica.isValid()) { + RUNTIME + .warn(format( + "replica is invalid, continue, replica=%s, partitionId/tabletId=%d, tableId=%d", + replica, partitionId, tableEntry.getTableId())); + continue; + } + ObPartitionLocation location = partitionLocationInfo.getPartitionLocation(); + + if (location == null) { + location = new ObPartitionLocation(); + partitionLocationInfo.setPartitionLocation(location); + } + location.addReplicaLocation(replica); + } + } finally { + partitionLocationInfo.rwLock.writeLock().unlock(); + } + // TODO: v3 + if (ObGlobal.obVsnMajor() < 4) { + } + return partitionEntry; + } + private static ObPartitionEntry getPartitionLocationFromResultSet(TableEntry tableEntry, ResultSet rs, ObPartitionEntry partitionEntry) diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/model/TableEntry.java b/src/main/java/com/alipay/oceanbase/rpc/location/model/TableEntry.java index 559c9e04..fce7a2c1 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/model/TableEntry.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/model/TableEntry.java @@ -23,9 +23,10 @@ import com.alipay.oceanbase.rpc.location.model.partition.ObPartitionLevel; import com.alipay.oceanbase.rpc.protocol.payload.Constants; -import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.Lock; import static com.google.common.base.Preconditions.checkArgument; @@ -53,7 +54,26 @@ public class TableEntry { // partition location private TableEntryKey tableEntryKey = null; private volatile ObPartitionEntry partitionEntry = null; - + + // tablet id ==> refresh time + private final ConcurrentHashMap refreshTimeMap = new ConcurrentHashMap<>(); + public ConcurrentHashMap refreshLockMap = new ConcurrentHashMap<>(); + + public long getTabletLocationLastRefreshTimeMills(Long tabletId) { + return refreshTimeMap.getOrDefault(tabletId, 0L); + } + public void setTableLocationLastRefreshTimeMills(Long tabletId, Long lastRefreshTime) { + refreshTimeMap.put(tabletId, lastRefreshTime); + } + + public Lock getRefreshLock(Long tabletId) { + return refreshLockMap.get(tabletId); + } + public void setRefreshLock(Long tabletId, Lock refreshLock) { + refreshLockMap.put(tabletId, refreshLock); + } + + /* * Is valid. */ @@ -218,8 +238,6 @@ public void prepare() throws IllegalArgumentException { checkArgument(partitionInfo != null, "partition table partition info is not ready. key" + tableEntryKey); partitionInfo.prepare(); - checkArgument(partitionEntry != null, - "partition table partition entry is not ready. key" + tableEntryKey); } } diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionEntry.java b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionEntry.java index d3e87e71..709ecd99 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionEntry.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionEntry.java @@ -21,13 +21,30 @@ import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + public class ObPartitionEntry { private Map partitionLocation = new HashMap(); // mapping from tablet id to ls id, and the part id to tablet id mapping is in ObPartitionInfo private Map tabletLsIdMap = new HashMap<>(); - + + // tabelt id -> (PartitionLocation, LsId) + // tablet id 作为索引管理PartitionInfo 其中包含了 PartitionLocation 和LSID + // 外部会通过tablet id并发的读写ObPartitionLocationInfo + // 写的场景就是更新,读的场景是正常的请求执行,需要保证读写的安全性,更新的时候一方面是保证线程安全,另一方面还需要保证不能频繁更新 + private ConcurrentHashMap partitionInfos = new ConcurrentHashMap<>(); + + + public ObPartitionLocationInfo getPartitionInfo(long tabletId) { + if (!partitionInfos.containsKey(tabletId)) { + ObPartitionLocationInfo partitionInfo = new ObPartitionLocationInfo(); + partitionInfos.put(tabletId, partitionInfo); + } + return partitionInfos.get(tabletId); + } + public Map getPartitionLocation() { return partitionLocation; } @@ -39,6 +56,16 @@ public void setPartitionLocation(Map partitionLocatio this.partitionLocation = partitionLocation; } + public Map getTabletLsIdMap() { + return tabletLsIdMap; + } + + public void setTabletLsIdMap(Map tabletLsIdMap) { + this.tabletLsIdMap = tabletLsIdMap; + } + + public long getLsId(long tabletId) { return tabletLsIdMap.get(tabletId); } + /* * Get partition location with part id. */ @@ -86,14 +113,4 @@ public void prepareForWeakRead(ObServerLdcLocation ldcLocation) { public String toString() { return "ObPartitionEntry{" + "partitionLocation=" + partitionLocation + '}'; } - - public Map getTabletLsIdMap() { - return tabletLsIdMap; - } - - public void setTabletLsIdMap(Map tabletLsIdMap) { - this.tabletLsIdMap = tabletLsIdMap; - } - - public long getLsId(long tabletId) { return tabletLsIdMap.get(tabletId); } } diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionLocationInfo.java b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionLocationInfo.java new file mode 100644 index 00000000..a226e1d4 --- /dev/null +++ b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionLocationInfo.java @@ -0,0 +1,76 @@ +/*- + * #%L + * com.oceanbase:obkv-table-client + * %% + * Copyright (C) 2021 - 2024 OceanBase + * %% + * OBKV Table Client Framework is licensed under Mulan PSL v2. + * You can use this software according to the terms and conditions of the Mulan PSL v2. + * You may obtain a copy of Mulan PSL v2 at: + * http://license.coscl.org.cn/MulanPSL2 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PSL v2 for more details. + * #L% + */ + +package com.alipay.oceanbase.rpc.location.model.partition; + +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import static com.alipay.oceanbase.rpc.protocol.payload.Constants.OB_INVALID_ID; + +// 这个类不做线程安全之类的处理 +public class ObPartitionLocationInfo { + private ObPartitionLocation partitionLocation = null; + private Long tabletLsId = OB_INVALID_ID; + private Long lastUpdateTime; // 最后更新时间 + public ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock(); // 读写锁 + public AtomicBoolean initialized = new AtomicBoolean(false); + + public ObPartitionLocationInfo() { + this.lastUpdateTime = System.currentTimeMillis(); // 初始化为当前时间 + } + + public ObPartitionLocation getPartitionLocation() { + rwLock.readLock().lock(); + try { + return partitionLocation; + } finally { + rwLock.readLock().unlock(); + } + } + + public void setPartitionLocation(ObPartitionLocation partitionLocation) { + this.partitionLocation = partitionLocation; + } + + public void updateLocation(ObPartitionLocation newLocation) { + rwLock.writeLock().lock(); + try { + this.partitionLocation = newLocation; + this.lastUpdateTime = System.currentTimeMillis(); + } finally { + rwLock.writeLock().unlock(); + } + } + + public Long getTabletLsId() { + return tabletLsId; + } + + public void setTabletLsId(Long tabletLsId) { + this.tabletLsId = tabletLsId; + } + + public Long getLastUpdateTime() { + rwLock.readLock().lock(); + try { + return lastUpdateTime; + } finally { + rwLock.readLock().unlock(); + } + } +} diff --git a/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryAsyncStreamResult.java b/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryAsyncStreamResult.java index b640fa71..d54c65a1 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryAsyncStreamResult.java +++ b/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryAsyncStreamResult.java @@ -84,8 +84,10 @@ public void init() throws Exception { it = expectant.entrySet().iterator(); retryTimes++; if (retryTimes > maxRetries) { - RUNTIME.error("Fail to get refresh table entry response after {}", retryTimes); - throw new ObTableRetryExhaustedException("Fail to get refresh table entry response after " + retryTimes); + RUNTIME.error("Fail to get refresh table entry response after {}", + retryTimes); + throw new ObTableRetryExhaustedException( + "Fail to get refresh table entry response after " + retryTimes); } } else { @@ -237,8 +239,10 @@ public boolean next() throws Exception { it = expectant.entrySet().iterator(); retryTimes++; if (retryTimes > client.getTableEntryRefreshTryTimes()) { - RUNTIME.error("Fail to get refresh table entry response after {}", retryTimes); - throw new ObTableRetryExhaustedException("Fail to get refresh table entry response after " + retryTimes); + RUNTIME.error("Fail to get refresh table entry response after {}", + retryTimes); + throw new ObTableRetryExhaustedException( + "Fail to get refresh table entry response after " + retryTimes); } continue; } else { diff --git a/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientBatchOpsImpl.java b/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientBatchOpsImpl.java index 5334eece..5672731a 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientBatchOpsImpl.java +++ b/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientBatchOpsImpl.java @@ -20,6 +20,7 @@ import com.alipay.oceanbase.rpc.ObTableClient; import com.alipay.oceanbase.rpc.exception.*; import com.alipay.oceanbase.rpc.location.model.ObServerRoute; +import com.alipay.oceanbase.rpc.location.model.TableEntry; import com.alipay.oceanbase.rpc.location.model.partition.ObPair; import com.alipay.oceanbase.rpc.mutation.result.*; import com.alipay.oceanbase.rpc.protocol.payload.ObPayload; @@ -352,11 +353,12 @@ public void partitionExecute(ObTableOperationResult[] results, if (failedServerList != null) { route.setBlackList(failedServerList); } - ObTableParam newParam = obTableClient.getTableWithPartId(tableName, - originPartId, needRefreshTableEntry, - obTableClient.isTableEntryRefreshIntervalWait(), needFetchAllRouteInfo, - route).getRight(); - + TableEntry entry = obTableClient.getOrRefreshTableEntry(tableName, false, + false, false); + obTableClient.refreshTableLocationByTabletId(entry, tableName, partId); + ObTableParam newParam = obTableClient.getTableWithPartId(tableName, partId, + false, obTableClient.isTableEntryRefreshIntervalWait(), needFetchAllRouteInfo, route) + .getRight(); subObTable = newParam.getObTable(); subRequest.setPartitionId(newParam.getPartitionId()); } @@ -418,6 +420,7 @@ public void partitionExecute(ObTableOperationResult[] results, tableName, partId, ((ObTableException) ex).getErrorCode(), tryTimes, ex); if (ex instanceof ObTableNeedFetchAllException) { + // refresh table info obTableClient.getOrRefreshTableEntry(tableName, needRefreshTableEntry, obTableClient.isTableEntryRefreshIntervalWait(), true); throw ex; @@ -444,7 +447,6 @@ public void partitionExecute(ObTableOperationResult[] results, throw new ObTableUnexpectedException( "check batch operation result error: client get unexpected NULL result"); } - List subObTableOperationResults = subObTableBatchOperationResult .getResults(); From 6874323a463de563da6efa71acc208d7ec9c579b Mon Sep 17 00:00:00 2001 From: maochongxin Date: Thu, 10 Oct 2024 19:56:21 +0800 Subject: [PATCH 03/15] Fix infinite loop caused by removed reference --- .../alipay/oceanbase/rpc/ObTableClient.java | 149 ++++++++---------- .../oceanbase/rpc/location/LocationUtil.java | 47 +++--- .../model/partition/ObPartitionEntry.java | 8 +- .../partition/ObPartitionLocationInfo.java | 30 ++-- 4 files changed, 103 insertions(+), 131 deletions(-) diff --git a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java index 2677dcbf..0bd6813e 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java +++ b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java @@ -1330,64 +1330,53 @@ private TableEntry refreshTableEntry(TableEntry tableEntry, String tableName) public TableEntry refreshTableLocationByTabletId(TableEntry tableEntry, String tableName, Long tabletId) throws ObTableAuthException { TableEntryKey tableEntryKey = new TableEntryKey(clusterName, tenantName, database, tableName); - Lock lock = null; try { - // if table entry is null, throw exception if (tableEntry == null) { - throw new ObTableEntryRefreshException("table entry is null, tableName=" + tableName); + throw new ObTableEntryRefreshException("Table entry is null, tableName=" + tableName); } - - long lastRefreshTime = tableEntry.getTabletLocationLastRefreshTimeMills(tabletId); + + long lastRefreshTime = tableEntry.getPartitionEntry().getPartitionInfo(tabletId).getLastUpdateTime(); long currentTime = System.currentTimeMillis(); - // 100ms - if (currentTime - lastRefreshTime < 100) { + if (currentTime - lastRefreshTime < 200) { return tableEntry; } - Lock tempLock = new ReentrantLock(); - lock = tableEntry.refreshLockMap.putIfAbsent(tabletId, tempLock); - lock = (lock == null) ? tempLock : lock; // check the first lock - - boolean acquired = lock.tryLock(tableEntryRefreshLockTimeout, TimeUnit.MILLISECONDS); - tableEntry.setTableLocationLastRefreshTimeMills(tabletId, currentTime); - if (!acquired) { - String errMsg = "try to lock table-entry refreshing timeout " + "dataSource:" - + dataSourceName + " ,tableName:" + tableName + ", refresh:" + - " , timeout:" + tableEntryRefreshLockTimeout + "."; + Lock lock = tableEntry.refreshLockMap.computeIfAbsent(tabletId, k -> new ReentrantLock()); + + if (!lock.tryLock(tableEntryRefreshLockTimeout, TimeUnit.MILLISECONDS)) { + String errMsg = String.format("Try to lock table-entry refreshing timeout. DataSource: %s, TableName: %s, Timeout: %d.", + dataSourceName, tableName, tableEntryRefreshLockTimeout); RUNTIME.error(errMsg); throw new ObTableEntryRefreshException(errMsg); } - tableEntry = loadTableEntryLocationWithPriority(serverRoster, // - tableEntryKey,// - tableEntry,// - tabletId, - tableEntryAcquireConnectTimeout,// - tableEntryAcquireSocketTimeout,// - serverAddressPriorityTimeout, // - serverAddressCachingTimeout, sysUA); - // prepare the table entry for weak read. - tableEntry.prepareForWeakRead(serverRoster.getServerLdcLocation()); + + try { + tableEntry = loadTableEntryLocationWithPriority(serverRoster, tableEntryKey, tableEntry, tabletId, + tableEntryAcquireConnectTimeout, tableEntryAcquireSocketTimeout, + serverAddressPriorityTimeout, serverAddressCachingTimeout, sysUA); + + tableEntry.prepareForWeakRead(serverRoster.getServerLdcLocation()); + } finally { + lock.unlock(); + } + } catch (ObTableNotExistException | ObTableServerCacheExpiredException e) { - RUNTIME.error("refreshTableEntry meet exception", e); + RUNTIME.error("RefreshTableEntry encountered an exception", e); throw e; } catch (Exception e) { + String errorMsg = String.format("Failed to get table entry. Key=%s, Original TableEntry=%s", tableEntryKey, tableEntry); RUNTIME.error(LCD.convert("01-00020"), tableEntryKey, tableEntry, e); - throw new ObTableEntryRefreshException(String.format( - "failed to get table entry key=%s original tableEntry=%s ", tableEntryKey, - tableEntry), e); - } finally { - if (lock != null) { - lock.unlock(); - } + throw new ObTableEntryRefreshException(errorMsg, e); } - tableLocations.put(tableName, tableEntry); + tableLocations.put(tableName, tableEntry); tableEntryRefreshContinuousFailureCount.set(0); + if (logger.isInfoEnabled()) { - logger.info( - "refresh table entry, dataSource: {}, tableName: {}, refresh: {} key:{} entry:{} ", - dataSourceName, tableName, true, tableEntryKey, JSON.toJSON(tableEntry)); + logger.info("Refreshed table entry. DataSource: {}, TableName: {}, Key: {}, Entry: {}", + dataSourceName, tableName, tableEntryKey, JSON.toJSON(tableEntry)); } + return tableEntry; } @@ -1606,13 +1595,6 @@ private ReplicaLocation getPartitionLocation(TableEntry tableEntry, long partId, private ReplicaLocation getPartitionLocation(ObPartitionLocationInfo obPartitionLocationInfo, ObServerRoute route) { - while (obPartitionLocationInfo.getPartitionLocation() == null) { - try { - Thread.sleep(10); - } catch (InterruptedException e) { - e.printStackTrace(); - } - } return obPartitionLocationInfo.getPartitionLocation().getReplica(route); } @@ -1844,48 +1826,61 @@ public ObPair getTableInternal(String tableName, TableEntry long partId, boolean waitForRefresh, ObServerRoute route) throws Exception { long tabletId = getTabletIdByPartId(tableEntry, partId); - ObPartitionLocationInfo obPartitionLocationInfo = tableEntry.getPartitionEntry().getPartitionInfo(tabletId); - if (!obPartitionLocationInfo.initialized.get()) { - tableEntry = refreshTableLocationByTabletId(tableEntry, tableName, tabletId); - obPartitionLocationInfo.initialized.compareAndSet(false, true); - } - + ObPartitionLocationInfo obPartitionLocationInfo = getOrRefreshPartitionInfo(tableEntry, tableName, tabletId); + ReplicaLocation replica = getPartitionLocation(obPartitionLocationInfo, route); ObServerAddr addr = replica.getAddr(); ObTable obTable = tableRoster.get(addr); - boolean addrExpired = addr.isExpired(serverAddressCachingTimeout); - if (obTable == null) { - logger.warn("can not get ObTable by addr {}, refresh metadata.", addr); - syncRefreshMetadata(); - } - if (addrExpired || obTable == null) { - if (logger.isInfoEnabled() && addrExpired) { - logger.info("server addr {} is expired, refresh tableEntry.", addr); - } - tableEntry = refreshTableLocationByTabletId(tableEntry, tableName, tabletId); - obPartitionLocationInfo = tableEntry.getPartitionEntry().getPartitionInfo(tabletId); + if (obTable == null || addr.isExpired(serverAddressCachingTimeout)) { + if (obTable == null) { + logger.warn("Cannot get ObTable by addr {}, refreshing metadata.", addr); + syncRefreshMetadata(); + } + if (addr.isExpired(serverAddressCachingTimeout)) { + logger.info("Server addr {} is expired, refreshing tableEntry.", addr); + } + + obPartitionLocationInfo = getOrRefreshPartitionInfo(tableEntry, tableName, tabletId); replica = getPartitionLocation(obPartitionLocationInfo, route); addr = replica.getAddr(); obTable = tableRoster.get(addr); + + if (obTable == null) { + RUNTIME.error("Cannot get table by addr: " + addr); + throw new ObTableGetException("Cannot get table by addr: " + addr); + } } - if (obTable == null) { - RUNTIME.error("cannot get table by addr: " + addr); - throw new ObTableGetException("cannot get table by addr: " + addr); + ObTableParam param = createTableParam(obTable, tableEntry, obPartitionLocationInfo, partId, tabletId); + addr.recordAccess(); + return new ObPair<>(tabletId, param); + } + + private ObPartitionLocationInfo getOrRefreshPartitionInfo(TableEntry tableEntry, + String tableName, long tabletId) + throws Exception { + ObPartitionLocationInfo obPartitionLocationInfo = tableEntry.getPartitionEntry() + .getPartitionInfo(tabletId); + if (!obPartitionLocationInfo.initialized.get()) { + tableEntry = refreshTableLocationByTabletId(tableEntry, tableName, tabletId); + obPartitionLocationInfo = tableEntry.getPartitionEntry().getPartitionInfo(tabletId); + obPartitionLocationInfo.initializationLatch.await(); } + return obPartitionLocationInfo; + } + private ObTableParam createTableParam(ObTable obTable, TableEntry tableEntry, + ObPartitionLocationInfo obPartitionLocationInfo, + long partId, long tabletId) { ObTableParam param = new ObTableParam(obTable); - param.setPartId(partId); // used in getTable(), 4.x may change the origin partId + param.setPartId(partId); if (ObGlobal.obVsnMajor() >= 4 && tableEntry != null) { param.setLsId(obPartitionLocationInfo.getTabletLsId()); } - param.setTableId(tableEntry.getTableId()); param.setPartitionId(tabletId); - - addr.recordAccess(); - return new ObPair<>(tabletId, param); + return param; } /** @@ -1910,11 +1905,7 @@ private List> getPartitionReplica(TableEntry table if (!tableEntry.isPartitionTable() || tableEntry.getPartitionInfo().getLevel() == ObPartitionLevel.LEVEL_ZERO) { long tabletId = getTabletIdByPartId(tableEntry, 0L); - ObPartitionLocationInfo locationInfo = tableEntry.getPartitionEntry().getPartitionInfo(tabletId); - if (!locationInfo.initialized.get()) { - refreshTableLocationByTabletId(tableEntry, tableName, tabletId); - locationInfo.initialized.compareAndSet(false, true); - } + ObPartitionLocationInfo locationInfo = getOrRefreshPartitionInfo(tableEntry, tableName, tabletId); replicas.add(new ObPair<>(tabletId, getPartitionLocation(locationInfo, route))); return replicas; } @@ -1924,11 +1915,7 @@ private List> getPartitionReplica(TableEntry table for (Long partId : partIds) { long tabletId = getTabletIdByPartId(tableEntry, partId); - ObPartitionLocationInfo locationInfo = tableEntry.getPartitionEntry().getPartitionInfo(tabletId); - if (!locationInfo.initialized.get()) { - refreshTableLocationByTabletId(tableEntry, tableName, tabletId); - locationInfo.initialized.compareAndSet(false, true); - } + ObPartitionLocationInfo locationInfo = getOrRefreshPartitionInfo(tableEntry, tableName, tabletId); replicas.add(new ObPair<>(tabletId, getPartitionLocation(locationInfo, route))); } diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java index 2fa12ab0..6f1e3ee7 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java @@ -1160,53 +1160,54 @@ private static ObPartitionEntry getPartitionLocationFromResultSetByTablet(TableE long tabletId) throws SQLException, ObTablePartitionLocationRefreshException { + if (partitionEntry == null || tableEntry == null) { throw new IllegalArgumentException("partitionEntry: " + partitionEntry + " tableEntry: " + tableEntry); } + ObPartitionLocationInfo partitionLocationInfo = partitionEntry.getPartitionInfo(tabletId); + + partitionLocationInfo.rwLock.writeLock().lock(); try { - partitionLocationInfo.rwLock.writeLock().lock(); while (rs.next()) { ReplicaLocation replica = buildReplicaLocation(rs); - long partitionId; - long lsId; - if (ObGlobal.obVsnMajor() >= 4) { - partitionId = rs.getLong("tablet_id"); - lsId = rs.getLong("ls_id"); - if (rs.wasNull()) { - lsId = INVALID_LS_ID; // non-partitioned table - } - partitionLocationInfo.setTabletLsId(lsId); - } else { - partitionId = rs.getLong("partition_id"); - if (tableEntry.isPartitionTable() - && null != tableEntry.getPartitionInfo().getSubPartDesc()) { - partitionId = ObPartIdCalculator.getPartIdx(partitionId, tableEntry - .getPartitionInfo().getSubPartDesc().getPartNum()); - } + + long partitionId = (ObGlobal.obVsnMajor() >= 4) ? rs.getLong("tablet_id") : rs + .getLong("partition_id"); + long lsId = ObGlobal.obVsnMajor() >= 4 ? rs.getLong("ls_id") : INVALID_LS_ID; + if (rs.wasNull() && ObGlobal.obVsnMajor() >= 4) { + lsId = INVALID_LS_ID; // For non-partitioned table } + partitionLocationInfo.setTabletLsId(lsId); + + if (ObGlobal.obVsnMajor() < 4 && tableEntry.isPartitionTable() + && tableEntry.getPartitionInfo().getSubPartDesc() != null) { + partitionId = ObPartIdCalculator.getPartIdx(partitionId, tableEntry + .getPartitionInfo().getSubPartDesc().getPartNum()); + } + if (!replica.isValid()) { RUNTIME .warn(format( - "replica is invalid, continue, replica=%s, partitionId/tabletId=%d, tableId=%d", + "Replica is invalid; continuing. Replica=%s, PartitionId/TabletId=%d, TableId=%d", replica, partitionId, tableEntry.getTableId())); continue; } ObPartitionLocation location = partitionLocationInfo.getPartitionLocation(); - if (location == null) { location = new ObPartitionLocation(); - partitionLocationInfo.setPartitionLocation(location); + partitionLocationInfo.updateLocation(location); } location.addReplicaLocation(replica); + + if (partitionLocationInfo.initialized.compareAndSet(false, true)) { + partitionLocationInfo.initializationLatch.countDown(); + } } } finally { partitionLocationInfo.rwLock.writeLock().unlock(); } - // TODO: v3 - if (ObGlobal.obVsnMajor() < 4) { - } return partitionEntry; } diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionEntry.java b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionEntry.java index 709ecd99..20af3b89 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionEntry.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionEntry.java @@ -36,13 +36,9 @@ public class ObPartitionEntry { // 写的场景就是更新,读的场景是正常的请求执行,需要保证读写的安全性,更新的时候一方面是保证线程安全,另一方面还需要保证不能频繁更新 private ConcurrentHashMap partitionInfos = new ConcurrentHashMap<>(); - + public ObPartitionLocationInfo getPartitionInfo(long tabletId) { - if (!partitionInfos.containsKey(tabletId)) { - ObPartitionLocationInfo partitionInfo = new ObPartitionLocationInfo(); - partitionInfos.put(tabletId, partitionInfo); - } - return partitionInfos.get(tabletId); + return partitionInfos.computeIfAbsent(tabletId, id -> new ObPartitionLocationInfo()); } public Map getPartitionLocation() { diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionLocationInfo.java b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionLocationInfo.java index a226e1d4..8b9181b9 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionLocationInfo.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionLocationInfo.java @@ -17,22 +17,19 @@ package com.alipay.oceanbase.rpc.location.model.partition; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantReadWriteLock; import static com.alipay.oceanbase.rpc.protocol.payload.Constants.OB_INVALID_ID; -// 这个类不做线程安全之类的处理 public class ObPartitionLocationInfo { - private ObPartitionLocation partitionLocation = null; - private Long tabletLsId = OB_INVALID_ID; - private Long lastUpdateTime; // 最后更新时间 - public ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock(); // 读写锁 - public AtomicBoolean initialized = new AtomicBoolean(false); - - public ObPartitionLocationInfo() { - this.lastUpdateTime = System.currentTimeMillis(); // 初始化为当前时间 - } + private ObPartitionLocation partitionLocation = null; + private Long tabletLsId = OB_INVALID_ID; + private Long lastUpdateTime = 0L; + public ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock(); + public AtomicBoolean initialized = new AtomicBoolean(false); + public final CountDownLatch initializationLatch = new CountDownLatch(1); public ObPartitionLocation getPartitionLocation() { rwLock.readLock().lock(); @@ -43,18 +40,9 @@ public ObPartitionLocation getPartitionLocation() { } } - public void setPartitionLocation(ObPartitionLocation partitionLocation) { - this.partitionLocation = partitionLocation; - } - public void updateLocation(ObPartitionLocation newLocation) { - rwLock.writeLock().lock(); - try { - this.partitionLocation = newLocation; - this.lastUpdateTime = System.currentTimeMillis(); - } finally { - rwLock.writeLock().unlock(); - } + this.partitionLocation = newLocation; + this.lastUpdateTime = System.currentTimeMillis(); } public Long getTabletLsId() { From 1d46a86a7e936325f07cb245c7043a95daa0ef2b Mon Sep 17 00:00:00 2001 From: maochongxin Date: Mon, 14 Oct 2024 16:48:14 +0800 Subject: [PATCH 04/15] fix lsop retry fail --- .../alipay/oceanbase/rpc/ObTableClient.java | 26 +++++++++---------- .../oceanbase/rpc/location/LocationUtil.java | 1 - .../table/ObTableClientLSBatchOpsImpl.java | 2 +- 3 files changed, 14 insertions(+), 15 deletions(-) diff --git a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java index 0bd6813e..3557b133 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java +++ b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java @@ -1364,7 +1364,7 @@ public TableEntry refreshTableLocationByTabletId(TableEntry tableEntry, String t RUNTIME.error("RefreshTableEntry encountered an exception", e); throw e; } catch (Exception e) { - String errorMsg = String.format("Failed to get table entry. Key=%s, Original TableEntry=%s", tableEntryKey, tableEntry); + String errorMsg = String.format("Failed to get table entry. Key=%s, Original TableEntry=%s, TabletId=%d", tableEntryKey, tableEntry, tabletId); RUNTIME.error(LCD.convert("01-00020"), tableEntryKey, tableEntry, e); throw new ObTableEntryRefreshException(errorMsg, e); } @@ -1939,7 +1939,7 @@ private List getPartitionTablePartitionIds(TableEntry tableEntry, } } - private long getTabletIdByPartId(TableEntry tableEntry, Long partId) { + public long getTabletIdByPartId(TableEntry tableEntry, Long partId) { if (ObGlobal.obVsnMajor() >= 4 && tableEntry.isPartitionTable()) { ObPartitionInfo partInfo = tableEntry.getPartitionInfo(); Map tabletIdMap = partInfo.getPartTabletIdMap(); @@ -2032,17 +2032,17 @@ public List> getTables(String tableName, ObTableQuery ObServerAddr addr = replica.getAddr(); ObTable obTable = tableRoster.get(addr); boolean addrExpired = addr.isExpired(serverAddressCachingTimeout); -// if (addrExpired || obTable == null) { -// logger -// .warn( -// "server address {} is expired={} or can not get ob table. So that will sync refresh metadata", -// addr, addrExpired); -// syncRefreshMetadata(); -// tableEntry = getOrRefreshTableEntry(tableName, true, waitForRefresh, false); -// replica = getPartitionLocation(tableEntry, partId, route); -// addr = replica.getAddr(); -// obTable = tableRoster.get(addr); -// } + if (addrExpired || obTable == null) { + logger + .warn( + "server address {} is expired={} or can not get ob table. So that will sync refresh metadata", + addr, addrExpired); + syncRefreshMetadata(); + tableEntry = getOrRefreshTableEntry(tableName, true, waitForRefresh, false); + replica = getPartitionLocation(tableEntry, tabletId, route); + addr = replica.getAddr(); + obTable = tableRoster.get(addr); + } if (obTable == null) { RUNTIME.error("cannot get table by addr: " + addr); diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java index 6f1e3ee7..9f737f83 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java @@ -874,7 +874,6 @@ public static TableEntry getTableEntryLocationFromRemote(Connection connection, // ignore } } - // 可能不需要了 tableEntry.setRefreshTimeMills(System.currentTimeMillis()); return tableEntry; } diff --git a/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java b/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java index b08bee14..9f6ad172 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java +++ b/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java @@ -740,7 +740,7 @@ public void doTask() { // Execute sub-batch operation one by one for (final Map.Entry>>>> entry : lsOperations .entrySet()) { - partitionExecute(obTableOperationResults, entry); + executeWithRetries(obTableOperationResults, entry, maxRetries); } } From 8d60c55b0079f0cdb9e3f0bb4b258eb318f14fd2 Mon Sep 17 00:00:00 2001 From: maochongxin Date: Tue, 15 Oct 2024 14:20:27 +0800 Subject: [PATCH 05/15] Remove unnecessary comments and format code --- .../com/alipay/oceanbase/rpc/ObTableClient.java | 13 +++++++++---- .../rpc/bolt/transport/ObTableRemoting.java | 2 +- .../rpc/location/model/TableEntry.java | 17 ----------------- .../model/partition/ObPartitionEntry.java | 3 --- .../query/AbstractQueryStreamResult.java | 1 - 5 files changed, 10 insertions(+), 26 deletions(-) diff --git a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java index 3557b133..ccb43355 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java +++ b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java @@ -1337,7 +1337,7 @@ public TableEntry refreshTableLocationByTabletId(TableEntry tableEntry, String t long lastRefreshTime = tableEntry.getPartitionEntry().getPartitionInfo(tabletId).getLastUpdateTime(); long currentTime = System.currentTimeMillis(); - if (currentTime - lastRefreshTime < 200) { + if (currentTime - lastRefreshTime < tableEntryRefreshIntervalBase) { return tableEntry; } @@ -2027,7 +2027,7 @@ public List> getTables(String tableName, ObTableQuery // obTableParams -> List> List> obTableParams = new ArrayList>(); for (ObPair partIdWithReplica : partIdWithReplicaList) { - long tabletId = partIdWithReplica.getLeft(); + long partId = partIdWithReplica.getLeft(); ReplicaLocation replica = partIdWithReplica.getRight(); ObServerAddr addr = replica.getAddr(); ObTable obTable = tableRoster.get(addr); @@ -2039,7 +2039,7 @@ public List> getTables(String tableName, ObTableQuery addr, addrExpired); syncRefreshMetadata(); tableEntry = getOrRefreshTableEntry(tableName, true, waitForRefresh, false); - replica = getPartitionLocation(tableEntry, tabletId, route); + replica = getPartitionLocation(tableEntry, partId, route); addr = replica.getAddr(); obTable = tableRoster.get(addr); } @@ -2050,9 +2050,14 @@ public List> getTables(String tableName, ObTableQuery } ObTableParam param = new ObTableParam(obTable); + if (ObGlobal.obVsnMajor() >= 4) { + long partIdx = tableEntry.getPartIdx(partId); + partId = tableEntry.isPartitionTable() ? tableEntry.getPartitionInfo() + .getPartTabletIdMap().get(partIdx) : partId; + } param.setTableId(tableEntry.getTableId()); // real partition(tablet) id - param.setPartitionId(tabletId); + param.setPartitionId(partId); addr.recordAccess(); obTableParams.add(new ObPair(partIdWithReplica.getLeft(), param)); diff --git a/src/main/java/com/alipay/oceanbase/rpc/bolt/transport/ObTableRemoting.java b/src/main/java/com/alipay/oceanbase/rpc/bolt/transport/ObTableRemoting.java index 4da53fb5..f7d9a200 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/bolt/transport/ObTableRemoting.java +++ b/src/main/java/com/alipay/oceanbase/rpc/bolt/transport/ObTableRemoting.java @@ -122,7 +122,7 @@ public ObPayload invokeSync(final ObTableConnection conn, final ObPayload reques ObRpcResultCode resultCode = new ObRpcResultCode(); resultCode.decode(buf); // If response indicates the request is routed to wrong server, we should refresh the routing meta. - if (!conn.getObTable().getReRouting() &&response.getHeader().isRoutingWrong()) { + if (!conn.getObTable().getReRouting() && response.getHeader().isRoutingWrong()) { String errMessage = TraceUtil.formatTraceMessage(conn, request, "routed to the wrong server: " + response.getMessage()); logger.warn(errMessage); diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/model/TableEntry.java b/src/main/java/com/alipay/oceanbase/rpc/location/model/TableEntry.java index fce7a2c1..260a067c 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/model/TableEntry.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/model/TableEntry.java @@ -55,25 +55,8 @@ public class TableEntry { private TableEntryKey tableEntryKey = null; private volatile ObPartitionEntry partitionEntry = null; - // tablet id ==> refresh time - private final ConcurrentHashMap refreshTimeMap = new ConcurrentHashMap<>(); public ConcurrentHashMap refreshLockMap = new ConcurrentHashMap<>(); - public long getTabletLocationLastRefreshTimeMills(Long tabletId) { - return refreshTimeMap.getOrDefault(tabletId, 0L); - } - public void setTableLocationLastRefreshTimeMills(Long tabletId, Long lastRefreshTime) { - refreshTimeMap.put(tabletId, lastRefreshTime); - } - - public Lock getRefreshLock(Long tabletId) { - return refreshLockMap.get(tabletId); - } - public void setRefreshLock(Long tabletId, Lock refreshLock) { - refreshLockMap.put(tabletId, refreshLock); - } - - /* * Is valid. */ diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionEntry.java b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionEntry.java index 20af3b89..8b514d78 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionEntry.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionEntry.java @@ -31,9 +31,6 @@ public class ObPartitionEntry { private Map tabletLsIdMap = new HashMap<>(); // tabelt id -> (PartitionLocation, LsId) - // tablet id 作为索引管理PartitionInfo 其中包含了 PartitionLocation 和LSID - // 外部会通过tablet id并发的读写ObPartitionLocationInfo - // 写的场景就是更新,读的场景是正常的请求执行,需要保证读写的安全性,更新的时候一方面是保证线程安全,另一方面还需要保证不能频繁更新 private ConcurrentHashMap partitionInfos = new ConcurrentHashMap<>(); diff --git a/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java b/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java index a11a8a67..0b846a31 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java +++ b/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java @@ -51,7 +51,6 @@ public abstract class AbstractQueryStreamResult extends AbstractPayload implemen protected volatile boolean closed = false; protected volatile List row = null; protected volatile int rowIndex = -1; - // 调整它的startKey protected ObTableQuery tableQuery; protected long operationTimeout = -1; protected String tableName; From 22eeb36773d836ae8e5197bf16c8d4dca63334d4 Mon Sep 17 00:00:00 2001 From: maochongxin Date: Fri, 18 Oct 2024 14:42:00 +0800 Subject: [PATCH 06/15] Fix frequent refresh lock failures due to short refresh interval --- .../java/com/alipay/oceanbase/rpc/ObTableClient.java | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java index ccb43355..bead0ef1 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java +++ b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java @@ -1334,10 +1334,9 @@ public TableEntry refreshTableLocationByTabletId(TableEntry tableEntry, String t if (tableEntry == null) { throw new ObTableEntryRefreshException("Table entry is null, tableName=" + tableName); } - long lastRefreshTime = tableEntry.getPartitionEntry().getPartitionInfo(tabletId).getLastUpdateTime(); long currentTime = System.currentTimeMillis(); - if (currentTime - lastRefreshTime < tableEntryRefreshIntervalBase) { + if (currentTime - lastRefreshTime < tableEntryRefreshLockTimeout) { return tableEntry; } @@ -1364,7 +1363,7 @@ public TableEntry refreshTableLocationByTabletId(TableEntry tableEntry, String t RUNTIME.error("RefreshTableEntry encountered an exception", e); throw e; } catch (Exception e) { - String errorMsg = String.format("Failed to get table entry. Key=%s, Original TableEntry=%s, TabletId=%d", tableEntryKey, tableEntry, tabletId); + String errorMsg = String.format("Failed to get table entry. Key=%s, TabletId=%d, message=%s", tableEntryKey, tabletId, e.getMessage()); RUNTIME.error(LCD.convert("01-00020"), tableEntryKey, tableEntry, e); throw new ObTableEntryRefreshException(errorMsg, e); } @@ -2050,11 +2049,6 @@ public List> getTables(String tableName, ObTableQuery } ObTableParam param = new ObTableParam(obTable); - if (ObGlobal.obVsnMajor() >= 4) { - long partIdx = tableEntry.getPartIdx(partId); - partId = tableEntry.isPartitionTable() ? tableEntry.getPartitionInfo() - .getPartTabletIdMap().get(partIdx) : partId; - } param.setTableId(tableEntry.getTableId()); // real partition(tablet) id param.setPartitionId(partId); From 73ef9aa439bfdfd491748045d7e58b7f24cdb665 Mon Sep 17 00:00:00 2001 From: maochongxin Date: Fri, 18 Oct 2024 15:22:23 +0800 Subject: [PATCH 07/15] Fix frequent refresh lock failures due to short refresh interval --- src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java | 5 +++++ .../oceanbase/rpc/bolt/transport/ObTableRemoting.java | 6 +++++- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java index bead0ef1..b6d7318c 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java +++ b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java @@ -1350,6 +1350,11 @@ public TableEntry refreshTableLocationByTabletId(TableEntry tableEntry, String t } try { + lastRefreshTime = tableEntry.getPartitionEntry().getPartitionInfo(tabletId).getLastUpdateTime(); + currentTime = System.currentTimeMillis(); + if (currentTime - lastRefreshTime < tableEntryRefreshLockTimeout) { + return tableEntry; + } tableEntry = loadTableEntryLocationWithPriority(serverRoster, tableEntryKey, tableEntry, tabletId, tableEntryAcquireConnectTimeout, tableEntryAcquireSocketTimeout, serverAddressPriorityTimeout, serverAddressCachingTimeout, sysUA); diff --git a/src/main/java/com/alipay/oceanbase/rpc/bolt/transport/ObTableRemoting.java b/src/main/java/com/alipay/oceanbase/rpc/bolt/transport/ObTableRemoting.java index f7d9a200..69f90bc9 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/bolt/transport/ObTableRemoting.java +++ b/src/main/java/com/alipay/oceanbase/rpc/bolt/transport/ObTableRemoting.java @@ -31,6 +31,8 @@ import io.netty.buffer.ByteBuf; import org.slf4j.Logger; +import javax.xml.transform.Result; + import static com.alipay.oceanbase.rpc.protocol.packet.ObCompressType.INVALID_COMPRESSOR; import static com.alipay.oceanbase.rpc.protocol.packet.ObCompressType.NONE_COMPRESSOR; @@ -139,7 +141,8 @@ public ObPayload invokeSync(final ObTableConnection conn, final ObPayload reques throw new ObTableNeedFetchAllException(errMessage, resultCode.getRcode()); } } - if (resultCode.getRcode() != 0 && response.getHeader().getPcode() != Pcodes.OB_TABLE_API_MOVE) { + if (resultCode.getRcode() != 0 + && response.getHeader().getPcode() != Pcodes.OB_TABLE_API_MOVE) { String errMessage = TraceUtil.formatTraceMessage(conn, request, "routed to the wrong server: " + response.getMessage()); logger.warn(errMessage); @@ -193,6 +196,7 @@ private boolean needFetchAll(int errorCode, int pcode) { || errorCode == ResultCodes.OB_TABLE_NOT_EXIST.errorCode || errorCode == ResultCodes.OB_TABLET_NOT_EXIST.errorCode || errorCode == ResultCodes.OB_LS_NOT_EXIST.errorCode + || errorCode == ResultCodes.OB_MAPPING_BETWEEN_TABLET_AND_LS_NOT_EXIST.errorCode || (pcode == Pcodes.OB_TABLE_API_LS_EXECUTE && errorCode == ResultCodes.OB_NOT_MASTER.errorCode); } From a46e2b0a7f72d98aa4028dc2326fa7a7afe9704e Mon Sep 17 00:00:00 2001 From: maochongxin Date: Fri, 18 Oct 2024 15:48:32 +0800 Subject: [PATCH 08/15] add result code -4723 --- .../java/com/alipay/oceanbase/rpc/location/LocationUtil.java | 3 ++- .../alipay/oceanbase/rpc/protocol/payload/ResultCodes.java | 4 +++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java index 9f737f83..4d344f11 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java @@ -395,7 +395,8 @@ private static TableEntry callTableEntryRefresh(ObServerAddr obServerAddr, Table RUNTIME.error(LCD.convert("01-00007"), url, key, e); } throw new ObTableEntryRefreshException(format( - "fail to refresh table entry from remote url=%s, key=%s", url, key), e); + "fail to refresh table entry from remote url=%s, key=%s, message=%s", url, key, + e.getMessage()), e); } finally { try { if (null != connection) { diff --git a/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/ResultCodes.java b/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/ResultCodes.java index fbb79d79..349912f1 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/ResultCodes.java +++ b/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/ResultCodes.java @@ -358,7 +358,9 @@ public enum ResultCodes { OB_CLUSTER_NO_MATCH(-4666), // OB_CHECK_ZONE_MERGE_ORDER(-4667), // OB_ERR_ZONE_NOT_EMPTY(-4668), // - OB_USE_DUP_FOLLOW_AFTER_DML(-4686), OB_LS_NOT_EXIST(-4719), // + OB_USE_DUP_FOLLOW_AFTER_DML(-4686), // + OB_LS_NOT_EXIST(-4719), // + OB_MAPPING_BETWEEN_TABLET_AND_LS_NOT_EXIST(-4723), // OB_TABLET_NOT_EXIST(-4725), // OB_ERR_PARSER_INIT(-5000), // OB_ERR_PARSE_SQL(-5001), // From fb03d26c10464904c7a98ab89c6078e9072e68ee Mon Sep 17 00:00:00 2001 From: maochongxin Date: Wed, 23 Oct 2024 19:26:17 +0800 Subject: [PATCH 09/15] fix review --- .../alipay/oceanbase/rpc/ObTableClient.java | 50 ++++++++++++++++--- .../oceanbase/rpc/location/LocationUtil.java | 47 +++++++++++++++-- .../query/AbstractQueryStreamResult.java | 5 +- .../ObTableClientQueryAsyncStreamResult.java | 35 +++++++++---- 4 files changed, 113 insertions(+), 24 deletions(-) diff --git a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java index b6d7318c..203e1679 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java +++ b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java @@ -1399,7 +1399,18 @@ private TableEntry refreshTableEntry(TableEntry tableEntry, String tableName, bo try { // if table entry is exist we just need to refresh table locations if (tableEntry != null && !fetchAll) { - // do nothing + if (ObGlobal.obVsnMajor() >= 4) { + // do nothing + } else { + // 3.x still proactively refreshes all locations + tableEntry = loadTableEntryLocationWithPriority(serverRoster, // + tableEntryKey,// + tableEntry,// + tableEntryAcquireConnectTimeout,// + tableEntryAcquireSocketTimeout,// + serverAddressPriorityTimeout, // + serverAddressCachingTimeout, sysUA); + } } else { // if table entry is not exist we should fetch partition info and table locations tableEntry = loadTableEntryWithPriority(serverRoster, // @@ -1829,10 +1840,23 @@ public ObTable addTable(ObServerAddr addr){ public ObPair getTableInternal(String tableName, TableEntry tableEntry, long partId, boolean waitForRefresh, ObServerRoute route) throws Exception { + ReplicaLocation replica = null; long tabletId = getTabletIdByPartId(tableEntry, partId); - ObPartitionLocationInfo obPartitionLocationInfo = getOrRefreshPartitionInfo(tableEntry, tableName, tabletId); + ObPartitionLocationInfo obPartitionLocationInfo = null; + if (ObGlobal.obVsnMajor() >= 4) { - ReplicaLocation replica = getPartitionLocation(obPartitionLocationInfo, route); + obPartitionLocationInfo = getOrRefreshPartitionInfo(tableEntry, tableName, tabletId); + + replica = getPartitionLocation(obPartitionLocationInfo, route); + } else { + ObPair partitionReplica = getPartitionReplica(tableEntry, partId, + route); + replica = partitionReplica.getRight(); + } + if (replica == null) { + RUNTIME.error("Cannot get replica by partId: " + partId); + throw new ObTableGetException("Cannot get replica by partId: " + partId); + } ObServerAddr addr = replica.getAddr(); ObTable obTable = tableRoster.get(addr); @@ -1845,8 +1869,14 @@ public ObPair getTableInternal(String tableName, TableEntry logger.info("Server addr {} is expired, refreshing tableEntry.", addr); } - obPartitionLocationInfo = getOrRefreshPartitionInfo(tableEntry, tableName, tabletId); - replica = getPartitionLocation(obPartitionLocationInfo, route); + if (ObGlobal.obVsnMajor() >= 4) { + obPartitionLocationInfo = getOrRefreshPartitionInfo(tableEntry, tableName, tabletId); + replica = getPartitionLocation(obPartitionLocationInfo, route); + } else { + tableEntry = getOrRefreshTableEntry(tableName, true, waitForRefresh, false); + replica = getPartitionReplica(tableEntry, partId, route).getRight(); + } + addr = replica.getAddr(); obTable = tableRoster.get(addr); @@ -1855,8 +1885,14 @@ public ObPair getTableInternal(String tableName, TableEntry throw new ObTableGetException("Cannot get table by addr: " + addr); } } - - ObTableParam param = createTableParam(obTable, tableEntry, obPartitionLocationInfo, partId, tabletId); + ObTableParam param = null; + if (ObGlobal.obVsnMajor() >= 4) { + param = createTableParam(obTable, tableEntry, obPartitionLocationInfo, partId, tabletId); + } else { + param.setPartId(partId); + param.setTableId(tableEntry.getTableId()); + param.setPartitionId(partId); + } addr.recordAccess(); return new ObPair<>(tabletId, param); } diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java index 4d344f11..ecc37f25 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java @@ -735,11 +735,16 @@ private static TableEntry getTableEntryFromRemote(Connection connection, TableEn } } } - - // only set empty partitionEntry - ObPartitionEntry partitionEntry = new ObPartitionEntry(); - tableEntry.setPartitionEntry(partitionEntry); - tableEntry.setRefreshTimeMills(System.currentTimeMillis()); + + if (ObGlobal.obVsnMajor() >= 4) { + // only set empty partitionEntry + ObPartitionEntry partitionEntry = new ObPartitionEntry(); + tableEntry.setPartitionEntry(partitionEntry); + tableEntry.setRefreshTimeMills(System.currentTimeMillis()); + } else { + // get location info + getTableEntryLocationFromRemote(connection, key, tableEntry); + } if (!initialized) { if (BOOT.isInfoEnabled()) { @@ -774,6 +779,7 @@ private static TableEntry getTableEntryFromRemote(Connection connection, TableEn return tableEntry; } + // Note: This code is applicable only for refreshing locations based on tablet ID in version 4.x private static String genLocationSQLByTabletId() { String sql = null; if (ObGlobal.obVsnMajor() >= 4) { @@ -890,6 +896,37 @@ public static TableEntry getTableEntryLocationFromRemote(Connection connection, PreparedStatement ps = null; ResultSet rs = null; ObPartitionEntry partitionEntry = new ObPartitionEntry(); + long partitionNum = tableEntry.getPartitionNum(); + int epoch = (int) ((partitionNum / MAX_TABLET_NUMS_EPOCH) + 1); + for (int i = 0; i < epoch; i++) { + try { + int offset = i * MAX_TABLET_NUMS_EPOCH; + // // This code is executed only in version 3.x + String sql = genLocationSQLByOffset(tableEntry, offset, MAX_TABLET_NUMS_EPOCH); + ps = connection.prepareStatement(sql); + ps.setString(1, key.getTenantName()); + ps.setString(2, key.getDatabaseName()); + ps.setString(3, key.getTableName()); + rs = ps.executeQuery(); + partitionEntry = getPartitionLocationFromResultSet(tableEntry, rs, partitionEntry); + } catch (Exception e) { + RUNTIME.error(LCD.convert("01-00010"), key, partitionNum, tableEntry, e); + throw new ObTablePartitionLocationRefreshException(format( + "fail to get partition location entry from remote entryKey = %s partNum = %d tableEntry =%s " + + "offset =%d epoch =%d", key, partitionNum, tableEntry, i, epoch), e); + } finally { + try { + if (null != rs) { + rs.close(); + } + if (null != ps) { + ps.close(); + } + } catch (SQLException e) { + // ignore + } + } + } // end for tableEntry.setPartitionEntry(partitionEntry); tableEntry.setRefreshTimeMills(System.currentTimeMillis()); return tableEntry; diff --git a/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java b/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java index 0b846a31..d6b1711f 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java +++ b/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java @@ -17,11 +17,13 @@ package com.alipay.oceanbase.rpc.protocol.payload.impl.execute.query; +import com.alipay.oceanbase.rpc.ObGlobal; import com.alipay.oceanbase.rpc.ObTableClient; import com.alipay.oceanbase.rpc.bolt.transport.ObTableConnection; import com.alipay.oceanbase.rpc.exception.*; import com.alipay.oceanbase.rpc.location.model.ObReadConsistency; import com.alipay.oceanbase.rpc.location.model.ObServerRoute; +import com.alipay.oceanbase.rpc.location.model.TableEntry; import com.alipay.oceanbase.rpc.location.model.partition.ObPair; import com.alipay.oceanbase.rpc.protocol.payload.AbstractPayload; import com.alipay.oceanbase.rpc.protocol.payload.ObPayload; @@ -323,10 +325,7 @@ public boolean next() throws Exception { } catch (Exception e) { if (e instanceof ObTableNeedFetchAllException) { - // Adjust the start key and refresh the expectant - this.tableQuery.adjustStartKey(currentStartKey); setExpectant(refreshPartition(tableQuery, tableName)); - // Reset the iterator to start over it = expectant.entrySet().iterator(); referPartition.clear(); // Clear the referPartition if needed diff --git a/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryAsyncStreamResult.java b/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryAsyncStreamResult.java index d54c65a1..cae7f39c 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryAsyncStreamResult.java +++ b/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryAsyncStreamResult.java @@ -17,11 +17,13 @@ package com.alipay.oceanbase.rpc.stream; +import com.alipay.oceanbase.rpc.ObGlobal; import com.alipay.oceanbase.rpc.ObTableClient; import com.alipay.oceanbase.rpc.bolt.transport.ObTableConnection; import com.alipay.oceanbase.rpc.exception.ObTableException; import com.alipay.oceanbase.rpc.exception.ObTableNeedFetchAllException; import com.alipay.oceanbase.rpc.exception.ObTableRetryExhaustedException; +import com.alipay.oceanbase.rpc.location.model.TableEntry; import com.alipay.oceanbase.rpc.location.model.partition.ObPair; import com.alipay.oceanbase.rpc.protocol.payload.Constants; import com.alipay.oceanbase.rpc.protocol.payload.ObPayload; @@ -201,11 +203,21 @@ public boolean next() throws Exception { referToLastStreamResult(lastEntry.getValue()); } catch (Exception e) { if (e instanceof ObTableNeedFetchAllException) { - this.asyncRequest.getObTableQueryRequest().getTableQuery() - .adjustStartKey(currentStartKey); - setExpectant(refreshPartition(this.asyncRequest.getObTableQueryRequest() - .getTableQuery(), tableName)); - setEnd(true); + + TableEntry entry = client.getOrRefreshTableEntry(tableName, false, false, false); + // Calculate the next partition only when the range partition is affected by a split, based on the keys already scanned. + if (ObGlobal.obVsnMajor() >= 4 + && entry.isPartitionTable() + && entry.getPartitionInfo().getFirstPartDesc().getPartFuncType().isRangePart()) { + this.asyncRequest.getObTableQueryRequest().getTableQuery() + .adjustStartKey(currentStartKey); + setExpectant(refreshPartition(this.asyncRequest.getObTableQueryRequest() + .getTableQuery(), tableName)); + setEnd(true); + } else { + setExpectant(refreshPartition(this.asyncRequest.getObTableQueryRequest() + .getTableQuery(), tableName)); + } } else { throw e; } @@ -232,10 +244,15 @@ public boolean next() throws Exception { referToNewPartition(entry.getValue()); } catch (Exception e) { if (e instanceof ObTableNeedFetchAllException) { - this.asyncRequest.getObTableQueryRequest().getTableQuery() - .adjustStartKey(currentStartKey); - setExpectant(refreshPartition(this.asyncRequest.getObTableQueryRequest() - .getTableQuery(), tableName)); + TableEntry tableEntry = client.getOrRefreshTableEntry(tableName, false, false, false); + if (ObGlobal.obVsnMajor() >= 4 + && tableEntry.isPartitionTable() + && tableEntry.getPartitionInfo().getFirstPartDesc().getPartFuncType().isRangePart()) { + this.asyncRequest.getObTableQueryRequest().getTableQuery() + .adjustStartKey(currentStartKey); + setExpectant(refreshPartition(this.asyncRequest.getObTableQueryRequest() + .getTableQuery(), tableName)); + } it = expectant.entrySet().iterator(); retryTimes++; if (retryTimes > client.getTableEntryRefreshTryTimes()) { From e5762f4d2b9e385231b5b2449ce1a4c5e7d8fcc4 Mon Sep 17 00:00:00 2001 From: maochongxin Date: Wed, 23 Oct 2024 20:09:40 +0800 Subject: [PATCH 10/15] add -4138 --- .../oceanbase/rpc/bolt/transport/ObTableRemoting.java | 5 ++--- .../com/alipay/oceanbase/rpc/property/Property.java | 2 +- .../java/com/alipay/oceanbase/rpc/table/ObTable.java | 10 +++++----- 3 files changed, 8 insertions(+), 9 deletions(-) diff --git a/src/main/java/com/alipay/oceanbase/rpc/bolt/transport/ObTableRemoting.java b/src/main/java/com/alipay/oceanbase/rpc/bolt/transport/ObTableRemoting.java index 69f90bc9..37a708bf 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/bolt/transport/ObTableRemoting.java +++ b/src/main/java/com/alipay/oceanbase/rpc/bolt/transport/ObTableRemoting.java @@ -31,8 +31,6 @@ import io.netty.buffer.ByteBuf; import org.slf4j.Logger; -import javax.xml.transform.Result; - import static com.alipay.oceanbase.rpc.protocol.packet.ObCompressType.INVALID_COMPRESSOR; import static com.alipay.oceanbase.rpc.protocol.packet.ObCompressType.NONE_COMPRESSOR; @@ -124,7 +122,7 @@ public ObPayload invokeSync(final ObTableConnection conn, final ObPayload reques ObRpcResultCode resultCode = new ObRpcResultCode(); resultCode.decode(buf); // If response indicates the request is routed to wrong server, we should refresh the routing meta. - if (!conn.getObTable().getReRouting() && response.getHeader().isRoutingWrong()) { + if (!conn.getObTable().isEnableRerouting() && response.getHeader().isRoutingWrong()) { String errMessage = TraceUtil.formatTraceMessage(conn, request, "routed to the wrong server: " + response.getMessage()); logger.warn(errMessage); @@ -197,6 +195,7 @@ private boolean needFetchAll(int errorCode, int pcode) { || errorCode == ResultCodes.OB_TABLET_NOT_EXIST.errorCode || errorCode == ResultCodes.OB_LS_NOT_EXIST.errorCode || errorCode == ResultCodes.OB_MAPPING_BETWEEN_TABLET_AND_LS_NOT_EXIST.errorCode + || errorCode == ResultCodes.OB_SNAPSHOT_DISCARDED.errorCode || (pcode == Pcodes.OB_TABLE_API_LS_EXECUTE && errorCode == ResultCodes.OB_NOT_MASTER.errorCode); } diff --git a/src/main/java/com/alipay/oceanbase/rpc/property/Property.java b/src/main/java/com/alipay/oceanbase/rpc/property/Property.java index abdfbfde..8ee42ebb 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/property/Property.java +++ b/src/main/java/com/alipay/oceanbase/rpc/property/Property.java @@ -134,7 +134,7 @@ public enum Property { NETTY_BLOCKING_WAIT_INTERVAL("bolt.netty.blocking.wait.interval", 1, "netty写缓存满后等待时间"), // [ObTable][OTHERS] - SERVER_ENABLE_REROUTING("server.enable.rerouting", true, "开启server端的重定向回复功能"), + SERVER_ENABLE_REROUTING("server.enable.rerouting", false, "开启server端的重定向回复功能"), /* * other config diff --git a/src/main/java/com/alipay/oceanbase/rpc/table/ObTable.java b/src/main/java/com/alipay/oceanbase/rpc/table/ObTable.java index fdbce217..551782dd 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/table/ObTable.java +++ b/src/main/java/com/alipay/oceanbase/rpc/table/ObTable.java @@ -66,7 +66,7 @@ public class ObTable extends AbstractObTable implements Lifecycle { private volatile boolean initialized = false; private volatile boolean closed = false; - private boolean reRouting = true; // only used for init packet factory + private boolean enableRerouting = true; // only used for init packet factory private ReentrantLock statusLock = new ReentrantLock(); @@ -89,7 +89,7 @@ public void init() throws Exception { .configWriteBufferWaterMark(getNettyBufferLowWatermark(), getNettyBufferHighWatermark()).build(); connectionFactory.init(new ConnectionEventHandler(new GlobalSwitch())); // Only for monitoring connection status - realClient = new ObTableRemoting(new ObPacketFactory(reRouting)); + realClient = new ObTableRemoting(new ObPacketFactory(enableRerouting)); connectionPool = new ObTableConnectionPool(this, obTableConnectionPoolSize); connectionPool.init(); initialized = true; @@ -164,7 +164,7 @@ private void initProperties() { nettyBufferHighWatermark); nettyBlockingWaitInterval = parseToInt(NETTY_BLOCKING_WAIT_INTERVAL.getKey(), nettyBlockingWaitInterval); - reRouting = parseToBoolean(SERVER_ENABLE_REROUTING.getKey(), reRouting); + enableRerouting = parseToBoolean(SERVER_ENABLE_REROUTING.getKey(), enableRerouting); maxConnExpiredTime = parseToLong(MAX_CONN_EXPIRED_TIME.getKey(), maxConnExpiredTime); Object value = this.configs.get("runtime"); @@ -174,8 +174,8 @@ private void initProperties() { } } - public boolean getReRouting(){ - return reRouting; + public boolean isEnableRerouting(){ + return enableRerouting; } /* From 4c630eedc5f318459803c4ebaed2c18046a20c08 Mon Sep 17 00:00:00 2001 From: maochongxin Date: Fri, 25 Oct 2024 11:51:45 +0800 Subject: [PATCH 11/15] fix review: add ut for byteutil --- .../table/ObTableClientLSBatchOpsImpl.java | 15 ++-- .../oceanbase/rpc/util/ByteUtilTest.java | 85 +++++++++++++++++++ 2 files changed, 92 insertions(+), 8 deletions(-) create mode 100644 src/test/java/com/alipay/oceanbase/rpc/util/ByteUtilTest.java diff --git a/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java b/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java index 9f6ad172..2d8e7459 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java +++ b/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java @@ -360,18 +360,17 @@ public Map>>> tabletOperations + Map>>> tabletOperations = lsOperationsMap.computeIfAbsent(lsId, k -> new HashMap<>()); - // if ls id not exists + // if ls id not exists - ObPair>> singleOperations = + ObPair>> singleOperations = tabletOperations.computeIfAbsent(tableObPair.getLeft(), k -> new ObPair<>(tableObPair.getRight(), new ArrayList<>())); - // if tablet id not exists - - singleOperations.getRight().add(new ObPair<>(i, operation)); - } + // if tablet id not exists + singleOperations.getRight().add(new ObPair<>(i, operation)); + } - return lsOperationsMap; + return lsOperationsMap; } public Map>>>> partitionPrepare() diff --git a/src/test/java/com/alipay/oceanbase/rpc/util/ByteUtilTest.java b/src/test/java/com/alipay/oceanbase/rpc/util/ByteUtilTest.java new file mode 100644 index 00000000..92dae7dc --- /dev/null +++ b/src/test/java/com/alipay/oceanbase/rpc/util/ByteUtilTest.java @@ -0,0 +1,85 @@ +package com.alipay.oceanbase.rpc.util; + +import org.junit.Assert; +import org.junit.Test; + +import static com.alipay.oceanbase.rpc.util.ByteUtil.compareByteArrays; +import static org.junit.Assert.*; + + +public class ByteUtilTest { + @Test + public void testcompareByteArrays() { + { + byte[] array1 = {1, 2, 3}; + byte[] array2 = {1, 2, 3}; + Assert.assertEquals(0, compareByteArrays(array1, array2)); + } + { + byte[] array1 = {2, 2, 3}; + byte[] array2 = {1, 2, 3}; + Assert.assertTrue(compareByteArrays(array1, array2) > 0); + } + { + byte[] array1 = {1, 2, 3, 4}; + byte[] array2 = {1, 2, 3}; + assertTrue(compareByteArrays(array1, array2) > 0); + } + { + byte[] array1 = {}; + byte[] array2 = {}; + assertEquals(0, compareByteArrays(array1, array2)); + } + } + @Test + public void testincrementByteArray() { + { + byte[] input = {0x01, 0x02, 0x03}; + byte[] expected = {0x01, 0x02, 0x04}; + assertArrayEquals(expected, ByteUtil.incrementByteArray(input)); + } + { + byte[] input = {(byte) 0xFF, (byte) 0xFF}; + byte[] expected = {0x01, 0x00, 0x00}; + assertArrayEquals(expected, ByteUtil.incrementByteArray(input)); + } + { + byte[] input = {}; + byte[] expected = {0x01}; + assertArrayEquals(expected, ByteUtil.incrementByteArray(input)); + } + { + byte[] expected = {0x01}; + assertArrayEquals(expected, ByteUtil.incrementByteArray(null)); + } + } + + @Test + public void testdecrementByteArray() { + { + byte[] input = {0x01}; + byte[] expected = {0x00}; + assertArrayEquals(expected, ByteUtil.decrementByteArray(input)); + } + { + byte[] input = {0x01, 0x00}; + byte[] expected = {0x00, (byte) 0xFF}; + assertArrayEquals(expected, ByteUtil.decrementByteArray(input)); + } + { + byte[] input = {0x02, 0x00}; + byte[] expected = {0x01, (byte) 0xFF}; + assertArrayEquals(expected, ByteUtil.decrementByteArray(input)); + } + { + byte[] input = {0x01, 0x00, 0x00}; + byte[] expected = {0x00, (byte) 0xFF, (byte) 0xFF}; + assertArrayEquals(expected, ByteUtil.decrementByteArray(input)); + } + { + byte[] input = {(byte) 0xFF, (byte) 0xFF}; + byte[] expected = {(byte) 0xFF, (byte) 0xFE}; + assertArrayEquals(expected, ByteUtil.decrementByteArray(input)); + } + } +} From 64926a44c9b30681fec4aadf8575a393d0b8a1df Mon Sep 17 00:00:00 2001 From: maochongxin Date: Fri, 25 Oct 2024 18:45:03 +0800 Subject: [PATCH 12/15] fix lsop refresh location --- .../oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java b/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java index 2d8e7459..6459398c 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java +++ b/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java @@ -21,6 +21,7 @@ import com.alipay.oceanbase.rpc.checkandmutate.CheckAndInsUp; import com.alipay.oceanbase.rpc.exception.*; import com.alipay.oceanbase.rpc.location.model.ObServerRoute; +import com.alipay.oceanbase.rpc.location.model.TableEntry; import com.alipay.oceanbase.rpc.location.model.partition.ObPair; import com.alipay.oceanbase.rpc.mutation.*; import com.alipay.oceanbase.rpc.mutation.result.MutationResult; @@ -473,6 +474,9 @@ public void partitionExecute(ObTableSingleOpResult[] results, if (failedServerList != null) { route.setBlackList(failedServerList); } + TableEntry entry = obTableClient.getOrRefreshTableEntry(tableName, false, + false, false); + obTableClient.refreshTableLocationByTabletId(entry, tableName, obTableClient.getTabletIdByPartId(entry, originPartId)); subObTable = obTableClient.getTableWithPartId(tableName, originPartId, needRefreshTableEntry, obTableClient.isTableEntryRefreshIntervalWait(), false, route). getRight().getObTable(); From c8e655a71a6f0f358426dae584636edd53163940 Mon Sep 17 00:00:00 2001 From: maochongxin Date: Mon, 28 Oct 2024 17:23:03 +0800 Subject: [PATCH 13/15] fix: correct reverse-scan results affected by retry logic --- .../payload/impl/execute/query/AbstractQueryStreamResult.java | 2 +- .../com/alipay/oceanbase/rpc/table/ObTableClientQueryImpl.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java b/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java index d6b1711f..5da49bbc 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java +++ b/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java @@ -360,7 +360,7 @@ public boolean next() throws Exception { } protected Map> buildPartitions(ObTableClient client, ObTableQuery tableQuery, String tableName) throws Exception { - Map> partitionObTables = new HashMap<>(); + Map> partitionObTables = new LinkedHashMap<>(); String indexName = tableQuery.getIndexName(); String indexTableName = null; diff --git a/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientQueryImpl.java b/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientQueryImpl.java index 46cf35a8..7ff2a8c9 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientQueryImpl.java +++ b/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientQueryImpl.java @@ -242,7 +242,7 @@ ObTableClientQueryAsyncStreamResult execute() throws Exception { } public Map> initPartitions(ObTableQuery tableQuery, String tableName) throws Exception { - Map> partitionObTables = new HashMap<>(); + Map> partitionObTables = new LinkedHashMap<>(); String indexName = tableQuery.getIndexName(); String indexTableName = null; From a53ba60a55913f9e6e1cea1fc41d1c99d0c002d2 Mon Sep 17 00:00:00 2001 From: maochongxin Date: Mon, 4 Nov 2024 17:46:19 +0800 Subject: [PATCH 14/15] feature: Optimize partial refresh and add retry for executeMutation/query (#213) * remove lock in refreshTableLocationByTabletId * fix refresh sql * fix refresh interval * add retry logic for common query * add retry logic for executeMutation * add retry logic for executeMutation * add retry logic for ObTableQueryAndMutateRequest * fix: correct ineffective retry logic * retry logic for ObTableClient execute * fix * fix --- .../alipay/oceanbase/rpc/ObTableClient.java | 163 +++++--- .../oceanbase/rpc/location/LocationUtil.java | 357 +++++++++--------- .../partition/ObPartitionLocationInfo.java | 23 +- .../query/AbstractQueryStreamResult.java | 54 ++- .../ObTableClientQueryAsyncStreamResult.java | 15 +- .../ObTableClientQueryStreamResult.java | 17 - .../table/ObTableClientLSBatchOpsImpl.java | 2 +- 7 files changed, 350 insertions(+), 281 deletions(-) diff --git a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java index 203e1679..b1c7ec09 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java +++ b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java @@ -677,6 +677,7 @@ private T execute(String tableName, TableExecuteCallback callback, ObServ tryTimes); if (ex instanceof ObTableNeedFetchAllException) { needFetchAllRouteInfo = true; + getOrRefreshTableEntry(tableName, true, true, true); // reset failure count while fetch all route info this.resetExecuteContinuousFailureCount(tableName); } @@ -767,7 +768,6 @@ private T executeMutation(String tableName, MutationExecuteCallback callb throw new IllegalArgumentException("table name is null"); } boolean needRefreshTableEntry = false; - boolean needFetchAllRouteInfo = false; int tryTimes = 0; long startExecute = System.currentTimeMillis(); while (true) { @@ -787,10 +787,14 @@ private T executeMutation(String tableName, MutationExecuteCallback callb obPair = new ObPair(0L, new ObTableParam(odpTable)); } else { if (null != callback.getRowKey()) { + // in the case of retry, the location always needs to be refreshed here + if (tryTimes > 1) { + TableEntry entry = getOrRefreshTableEntry(tableName, false, false, false); + Long partId = getPartition(entry, callback.getRowKey()); + refreshTableLocationByTabletId(entry, tableName, getTabletIdByPartId(entry, partId)); + } // using row key - obPair = getTable(tableName, callback.getRowKey(), - needRefreshTableEntry, tableEntryRefreshIntervalWait, - needFetchAllRouteInfo, route); + obPair = getTable(tableName, callback.getRowKey(), needRefreshTableEntry, tableEntryRefreshIntervalWait, false, route); } else if (null != callback.getKeyRanges()) { // using scan range obPair = getTable(tableName, new ObTableQuery(), @@ -852,7 +856,7 @@ private T executeMutation(String tableName, MutationExecuteCallback callb ((ObTableException) ex).getErrorCode(), ex.getMessage(), tryTimes); if (ex instanceof ObTableNeedFetchAllException) { - needFetchAllRouteInfo = true; + getOrRefreshTableEntry(tableName, needRefreshTableEntry, isTableEntryRefreshIntervalWait(), true); // reset failure count while fetch all route info this.resetExecuteContinuousFailureCount(tableName); } @@ -1336,33 +1340,14 @@ public TableEntry refreshTableLocationByTabletId(TableEntry tableEntry, String t } long lastRefreshTime = tableEntry.getPartitionEntry().getPartitionInfo(tabletId).getLastUpdateTime(); long currentTime = System.currentTimeMillis(); - if (currentTime - lastRefreshTime < tableEntryRefreshLockTimeout) { + if (currentTime - lastRefreshTime < tableEntryRefreshIntervalCeiling) { return tableEntry; } - - Lock lock = tableEntry.refreshLockMap.computeIfAbsent(tabletId, k -> new ReentrantLock()); - - if (!lock.tryLock(tableEntryRefreshLockTimeout, TimeUnit.MILLISECONDS)) { - String errMsg = String.format("Try to lock table-entry refreshing timeout. DataSource: %s, TableName: %s, Timeout: %d.", - dataSourceName, tableName, tableEntryRefreshLockTimeout); - RUNTIME.error(errMsg); - throw new ObTableEntryRefreshException(errMsg); - } - - try { - lastRefreshTime = tableEntry.getPartitionEntry().getPartitionInfo(tabletId).getLastUpdateTime(); - currentTime = System.currentTimeMillis(); - if (currentTime - lastRefreshTime < tableEntryRefreshLockTimeout) { - return tableEntry; - } - tableEntry = loadTableEntryLocationWithPriority(serverRoster, tableEntryKey, tableEntry, tabletId, - tableEntryAcquireConnectTimeout, tableEntryAcquireSocketTimeout, - serverAddressPriorityTimeout, serverAddressCachingTimeout, sysUA); + tableEntry = loadTableEntryLocationWithPriority(serverRoster, tableEntryKey, tableEntry, tabletId, + tableEntryAcquireConnectTimeout, tableEntryAcquireSocketTimeout, + serverAddressPriorityTimeout, serverAddressCachingTimeout, sysUA); - tableEntry.prepareForWeakRead(serverRoster.getServerLdcLocation()); - } finally { - lock.unlock(); - } + tableEntry.prepareForWeakRead(serverRoster.getServerLdcLocation()); } catch (ObTableNotExistException | ObTableServerCacheExpiredException e) { RUNTIME.error("RefreshTableEntry encountered an exception", e); @@ -1676,7 +1661,9 @@ private ObPair getTable(String tableName, Object[] rowKey, } long partId = getPartition(tableEntry, row); // partition id in 3.x, origin partId in 4.x, logicId - + if (refresh) { + refreshTableLocationByTabletId(tableEntry, tableName, getTabletIdByPartId(tableEntry, partId)); + } return getTableInternal(tableName, tableEntry, partId, waitForRefresh, route); } @@ -3149,41 +3136,95 @@ public ObPayload execute(final ObTableAbstractOperationRequest request) throws E request.setTimeout(getOdpTable().getObTableOperationTimeout()); return getOdpTable().execute(request); } else { + int maxRetries = getRuntimeRetryTimes(); // Define the maximum number of retries + int tryTimes = 0; + long startExecute = System.currentTimeMillis(); + boolean needRefreshTableEntry = false; Map partIdMapObTable = new HashMap(); - for (ObNewRange rang : tableQuery.getKeyRanges()) { - ObRowKey startKey = rang.getStartKey(); - int startKeySize = startKey.getObjs().size(); - ObRowKey endKey = rang.getEndKey(); - int endKeySize = endKey.getObjs().size(); - Object[] start = new Object[startKeySize]; - Object[] end = new Object[endKeySize]; - for (int i = 0; i < startKeySize; i++) { - start[i] = startKey.getObj(i).getValue(); + while (true) { + long currentExecute = System.currentTimeMillis(); + long costMillis = currentExecute - startExecute; + if (costMillis > getRuntimeMaxWait()) { + logger.error( + "tablename:{} it has tried " + tryTimes + + " times and it has waited " + costMillis + + "/ms which exceeds response timeout " + + getRuntimeMaxWait() + "/ms", request.getTableName()); + throw new ObTableTimeoutExcetion("it has tried " + tryTimes + + " times and it has waited " + costMillis + + "/ms which exceeds response timeout " + + getRuntimeMaxWait() + "/ms"); } + try { + // Recalculate partIdMapObTable + // Clear the map before recalculating + partIdMapObTable.clear(); + for (ObNewRange rang : tableQuery.getKeyRanges()) { + ObRowKey startKey = rang.getStartKey(); + int startKeySize = startKey.getObjs().size(); + ObRowKey endKey = rang.getEndKey(); + int endKeySize = endKey.getObjs().size(); + Object[] start = new Object[startKeySize]; + Object[] end = new Object[endKeySize]; + for (int i = 0; i < startKeySize; i++) { + start[i] = startKey.getObj(i).getValue(); + } - for (int i = 0; i < endKeySize; i++) { - end[i] = endKey.getObj(i).getValue(); - } - ObBorderFlag borderFlag = rang.getBorderFlag(); - List> pairList = getTables(request.getTableName(), - tableQuery, start, borderFlag.isInclusiveStart(), end, - borderFlag.isInclusiveEnd(), false, false); - for (ObPair pair : pairList) { - partIdMapObTable.put(pair.getLeft(), pair.getRight()); - } - } - if (partIdMapObTable.size() > 1) { - throw new ObTablePartitionConsistentException( - "query and mutate must be a atomic operation"); - } + for (int i = 0; i < endKeySize; i++) { + end[i] = endKey.getObj(i).getValue(); + } + ObBorderFlag borderFlag = rang.getBorderFlag(); + List> pairList = getTables(request.getTableName(), + tableQuery, start, borderFlag.isInclusiveStart(), end, + borderFlag.isInclusiveEnd(), needRefreshTableEntry, isTableEntryRefreshIntervalWait()); + for (ObPair pair : pairList) { + partIdMapObTable.put(pair.getLeft(), pair.getRight()); + } + } - for (Long partId : partIdMapObTable.keySet()) { - ObTableParam tableParam = partIdMapObTable.get(partId); - request.setTableId(tableParam.getTableId()); - request.setPartitionId(tableParam.getPartitionId()); - request.setTimeout(tableParam.getObTable().getObTableOperationTimeout()); - ObTable obTable = tableParam.getObTable(); - return executeWithRetry(obTable, request, request.getTableName()); + // Check if partIdMapObTable size is greater than 1 + if (partIdMapObTable.size() > 1) { + throw new ObTablePartitionConsistentException( + "query and mutate must be a atomic operation"); + } + // Proceed with the operation + Map.Entry entry = partIdMapObTable.entrySet().iterator().next(); + ObTableParam tableParam = entry.getValue(); + request.setTableId(tableParam.getTableId()); + request.setPartitionId(tableParam.getPartitionId()); + request.setTimeout(tableParam.getObTable().getObTableOperationTimeout()); + ObTable obTable = tableParam.getObTable(); + + // Attempt to execute the operation + return executeWithRetry(obTable, request, request.getTableName()); + } catch (Exception ex) { + tryTimes++; + if (ex instanceof ObTableException && ((ObTableException) ex).isNeedRefreshTableEntry()) { + needRefreshTableEntry = true; + logger.warn( + "tablename:{} partition id:{} batch ops refresh table while meet ObTableMasterChangeException, errorCode: {}", + request.getTableName(), request.getPartitionId(), ((ObTableException) ex).getErrorCode(), ex); + + if (isRetryOnChangeMasterTimes() && tryTimes <= maxRetries) { + logger.warn( + "tablename:{} partition id:{} batch ops retry while meet ObTableMasterChangeException, errorCode: {} , retry times {}", + request.getTableName(), request.getPartitionId(), ((ObTableException) ex).getErrorCode(), + tryTimes, ex); + + if (ex instanceof ObTableNeedFetchAllException) { + // Refresh table info + getOrRefreshTableEntry(request.getTableName(), needRefreshTableEntry, isTableEntryRefreshIntervalWait(), true); + } + } else { + calculateContinuousFailure(request.getTableName(), ex.getMessage()); + throw ex; + } + } else { + calculateContinuousFailure(request.getTableName(), ex.getMessage()); + // Handle other exceptions or rethrow + throw ex; + } + } } } } diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java index ecc37f25..9e1fde4d 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java @@ -56,156 +56,164 @@ public class LocationUtil { - private static final Logger logger = TableClientLoggerFactory - .getLogger(LocationUtil.class); + private static final Logger logger = TableClientLoggerFactory + .getLogger(LocationUtil.class); static { ParserConfig.getGlobalInstance().setSafeMode(true); } - private static final String OB_VERSION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ OB_VERSION() AS CLUSTER_VERSION;"; + private static final String OB_VERSION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ OB_VERSION() AS CLUSTER_VERSION;"; - private static final String PROXY_INDEX_INFO_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ data_table_id, table_id, index_type FROM oceanbase.__all_virtual_table " - + "where table_name = ?"; + private static final String PROXY_INDEX_INFO_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ data_table_id, table_id, index_type FROM oceanbase.__all_virtual_table " + + "where table_name = ?"; - private static final String PROXY_TABLE_ID_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ table_id from oceanbase.__all_virtual_proxy_schema " - + "where tenant_name = ? and database_name = ? and table_name = ? limit 1"; + private static final String PROXY_TABLE_ID_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ table_id from oceanbase.__all_virtual_proxy_schema " + + "where tenant_name = ? and database_name = ? and table_name = ? limit 1"; - private static final String OB_TENANT_EXIST_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ tenant_id from __all_tenant where tenant_name = ?;"; + private static final String OB_TENANT_EXIST_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ tenant_id from __all_tenant where tenant_name = ?;"; @Deprecated @SuppressWarnings("unused") - private static final String PROXY_PLAIN_SCHEMA_SQL_FORMAT = "SELECT /*+READ_CONSISTENCY(WEAK)*/ partition_id, svr_ip, sql_port, table_id, role, part_num, replica_num, schema_version, spare1 " - + "FROM oceanbase.__all_virtual_proxy_schema " - + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND partition_id in ({0}) AND sql_port > 0 " - + "ORDER BY role ASC LIMIT ?"; - - private static final String PROXY_PART_INFO_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_level, part_num, part_type, part_space, part_expr, " - + "part_range_type, part_interval_bin, interval_start_bin, " - + "sub_part_num, sub_part_type, sub_part_space, " - + "sub_part_range_type, def_sub_part_interval_bin, def_sub_interval_start_bin, sub_part_expr, " - + "part_key_name, part_key_type, part_key_idx, part_key_extra, spare1 " - + "FROM oceanbase.__all_virtual_proxy_partition_info " - + "WHERE table_id = ? group by part_key_name order by part_key_name LIMIT ?;"; + private static final String PROXY_PLAIN_SCHEMA_SQL_FORMAT = "SELECT /*+READ_CONSISTENCY(WEAK)*/ partition_id, svr_ip, sql_port, table_id, role, part_num, replica_num, schema_version, spare1 " + + "FROM oceanbase.__all_virtual_proxy_schema " + + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND partition_id in ({0}) AND sql_port > 0 " + + "ORDER BY role ASC LIMIT ?"; + + private static final String PROXY_PART_INFO_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_level, part_num, part_type, part_space, part_expr, " + + "part_range_type, part_interval_bin, interval_start_bin, " + + "sub_part_num, sub_part_type, sub_part_space, " + + "sub_part_range_type, def_sub_part_interval_bin, def_sub_interval_start_bin, sub_part_expr, " + + "part_key_name, part_key_type, part_key_idx, part_key_extra, spare1 " + + "FROM oceanbase.__all_virtual_proxy_partition_info " + + "WHERE table_id = ? group by part_key_name order by part_key_name LIMIT ?;"; @Deprecated @SuppressWarnings("unused") - private static final String PROXY_TENANT_SCHEMA_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ svr_ip, sql_port, table_id, role, part_num, replica_num, spare1 " - + "FROM oceanbase.__all_virtual_proxy_schema " - + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND sql_port > 0 " - + "ORDER BY partition_id ASC, role ASC LIMIT ?"; - - private static final String PROXY_DUMMY_LOCATION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.partition_id as partition_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " - + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " - + ", A.spare1 as replica_type " - + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " - + "WHERE tenant_name = ? and database_name=? and table_name = ?"; - - private static final String PROXY_LOCATION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.partition_id as partition_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " - + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " - + ", A.spare1 as replica_type " - + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " - + "WHERE tenant_name = ? and database_name=? and table_name = ? and partition_id = 0"; - - private static final String PROXY_LOCATION_SQL_PARTITION = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.partition_id as partition_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " - + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " - + ", A.spare1 as replica_type " - + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " - + "WHERE tenant_name = ? and database_name=? and table_name = ? and partition_id in ({0})"; - - private static final String PROXY_FIRST_PARTITION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_id, part_name, high_bound_val " - + "FROM oceanbase.__all_virtual_proxy_partition " - + "WHERE table_id = ? LIMIT ?;"; - - private static final String PROXY_SUB_PARTITION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ sub_part_id, part_name, high_bound_val " - + "FROM oceanbase.__all_virtual_proxy_sub_partition " - + "WHERE table_id = ? LIMIT ?;"; - - private static final String PROXY_SERVER_STATUS_INFO = "SELECT ss.svr_ip, ss.zone, zs.region, zs.spare4 as idc " - + "FROM oceanbase.__all_virtual_proxy_server_stat ss, oceanbase.__all_virtual_zone_stat zs " - + "WHERE zs.zone = ss.zone ;"; + private static final String PROXY_TENANT_SCHEMA_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ svr_ip, sql_port, table_id, role, part_num, replica_num, spare1 " + + "FROM oceanbase.__all_virtual_proxy_schema " + + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND sql_port > 0 " + + "ORDER BY partition_id ASC, role ASC LIMIT ?"; + + private static final String PROXY_DUMMY_LOCATION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.partition_id as partition_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " + + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " + + ", A.spare1 as replica_type " + + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " + + "WHERE tenant_name = ? and database_name=? and table_name = ?"; + + private static final String PROXY_LOCATION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.partition_id as partition_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " + + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " + + ", A.spare1 as replica_type " + + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " + + "WHERE tenant_name = ? and database_name=? and table_name = ? and partition_id = 0"; + + private static final String PROXY_LOCATION_SQL_PARTITION = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.partition_id as partition_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " + + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " + + ", A.spare1 as replica_type " + + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " + + "WHERE tenant_name = ? and database_name=? and table_name = ? and partition_id in ({0})"; + + private static final String PROXY_FIRST_PARTITION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_id, part_name, high_bound_val " + + "FROM oceanbase.__all_virtual_proxy_partition " + + "WHERE table_id = ? LIMIT ?;"; + + private static final String PROXY_SUB_PARTITION_SQL = "SELECT /*+READ_CONSISTENCY(WEAK)*/ sub_part_id, part_name, high_bound_val " + + "FROM oceanbase.__all_virtual_proxy_sub_partition " + + "WHERE table_id = ? LIMIT ?;"; + + private static final String PROXY_SERVER_STATUS_INFO = "SELECT ss.svr_ip, ss.zone, zs.region, zs.spare4 as idc " + + "FROM oceanbase.__all_virtual_proxy_server_stat ss, oceanbase.__all_virtual_zone_stat zs " + + "WHERE zs.zone = ss.zone ;"; @Deprecated @SuppressWarnings("unused") - private static final String PROXY_PLAIN_SCHEMA_SQL_FORMAT_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ tablet_id, svr_ip, sql_port, table_id, role, part_num, replica_num, schema_version, spare1 " - + "FROM oceanbase.__all_virtual_proxy_schema " - + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND tablet_id in ({0}) AND sql_port > 0 " - + "ORDER BY role ASC LIMIT ?"; - - private static final String PROXY_PART_INFO_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_level, part_num, part_type, part_space, part_expr, " - + "part_range_type, sub_part_num, sub_part_type, sub_part_space, sub_part_range_type, sub_part_expr, " - + "part_key_name, part_key_type, part_key_idx, part_key_extra, part_key_collation_type " - + "FROM oceanbase.__all_virtual_proxy_partition_info " - + "WHERE tenant_name = ? and table_id = ? group by part_key_name order by part_key_name LIMIT ?;"; + private static final String PROXY_PLAIN_SCHEMA_SQL_FORMAT_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ tablet_id, svr_ip, sql_port, table_id, role, part_num, replica_num, schema_version, spare1 " + + "FROM oceanbase.__all_virtual_proxy_schema " + + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND tablet_id in ({0}) AND sql_port > 0 " + + "ORDER BY role ASC LIMIT ?"; + + private static final String PROXY_PART_INFO_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_level, part_num, part_type, part_space, part_expr, " + + "part_range_type, sub_part_num, sub_part_type, sub_part_space, sub_part_range_type, sub_part_expr, " + + "part_key_name, part_key_type, part_key_idx, part_key_extra, part_key_collation_type " + + "FROM oceanbase.__all_virtual_proxy_partition_info " + + "WHERE tenant_name = ? and table_id = ? group by part_key_name order by part_key_name LIMIT ?;"; @Deprecated @SuppressWarnings("unused") - private static final String PROXY_TENANT_SCHEMA_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ svr_ip, sql_port, table_id, role, part_num, replica_num, spare1 " - + "FROM oceanbase.__all_virtual_proxy_schema " - + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND sql_port > 0 " - + "ORDER BY tablet_id ASC, role ASC LIMIT ?"; - - private static final String PROXY_DUMMY_LOCATION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.tablet_id as tablet_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " - + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " - + ", A.spare1 as replica_type " - + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " - + "WHERE tenant_name = ? and database_name=? and table_name = ?"; - - private static final String PROXY_LOCATION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.tablet_id as tablet_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " - + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " - + ", A.spare1 as replica_type " - + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " - + "WHERE tenant_name = ? and database_name=? and table_name = ? and tablet_id = 0"; - - private static final String PROXY_LOCATION_SQL_PARTITION_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ * FROM ( " - + " SELECT A.tablet_id as tablet__id, A.svr_ip as svr_ip, A.sql_port as sql_port, A.table_id as table_id, " - + " A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, " - + " B.stop_time as stop_time, A.spare1 as replica_type " - + " FROM oceanbase.__all_virtual_proxy_schema A " - + " INNER JOIN oceanbase.__all_server B ON A.svr_ip = B.svr_ip AND A.sql_port = B.inner_port " - + " WHERE A.tablet_id IN ({0}) AND A.tenant_name = ? AND A.database_name = ? AND A.table_name = ?) AS left_table " - + "LEFT JOIN (" - + " SELECT D.ls_id, D.tablet_id " - + " FROM oceanbase.__all_virtual_tablet_to_ls D " - + " INNER JOIN oceanbase.DBA_OB_TENANTS C ON D.tenant_id = C.tenant_id " - + " WHERE C.tenant_name = ? " - + ") AS right_table ON left_table.tablet__id = right_table.tablet_id;"; - - private static final String PROXY_LOCATION_SQL_PARTITION_BY_TABLETID_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ * FROM ( " - + " SELECT A.tablet_id as tablet__id, A.svr_ip as svr_ip, A.sql_port as sql_port, A.table_id as table_id, " - + " A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, " - + " B.stop_time as stop_time, A.spare1 as replica_type " - + " FROM oceanbase.__all_virtual_proxy_schema A " - + " INNER JOIN oceanbase.__all_server B ON A.svr_ip = B.svr_ip AND A.sql_port = B.inner_port " - + " WHERE A.tablet_id = ? AND A.tenant_name = ? AND A.database_name = ? AND A.table_name = ?) AS left_table " - + "LEFT JOIN (" - + " SELECT D.ls_id, D.tablet_id " - + " FROM oceanbase.__all_virtual_tablet_to_ls D " - + " INNER JOIN oceanbase.DBA_OB_TENANTS C ON D.tenant_id = C.tenant_id " - + " WHERE C.tenant_name = ? " - + ") AS right_table ON left_table.tablet__id = right_table.tablet_id;"; - - private static final String PROXY_FIRST_PARTITION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_id, part_name, tablet_id, high_bound_val, sub_part_num " - + "FROM oceanbase.__all_virtual_proxy_partition " - + "WHERE tenant_name = ? and table_id = ? LIMIT ?;"; - - private static final String PROXY_SUB_PARTITION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ sub_part_id, part_name, tablet_id, high_bound_val " - + "FROM oceanbase.__all_virtual_proxy_sub_partition " - + "WHERE tenant_name = ? and table_id = ? LIMIT ?;"; - - private static final String PROXY_SERVER_STATUS_INFO_V4 = "SELECT ss.svr_ip, ss.zone, zs.region, zs.idc as idc " - + "FROM DBA_OB_SERVERS ss, DBA_OB_ZONES zs " - + "WHERE zs.zone = ss.zone ;"; - - private static final String home = System.getProperty( - "user.home", - "/home/admin"); - - private static final String TABLE_GROUP_GET_TABLE_NAME_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ table_name " - + "FROM oceanbase.CDB_OB_TABLEGROUP_TABLES " - + "WHERE tablegroup_name = ? and tenant_id = ? limit 1;"; - - private static final int TEMPLATE_PART_ID = -1; + private static final String PROXY_TENANT_SCHEMA_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ svr_ip, sql_port, table_id, role, part_num, replica_num, spare1 " + + "FROM oceanbase.__all_virtual_proxy_schema " + + "WHERE tenant_name = ? AND database_name = ? AND table_name = ? AND sql_port > 0 " + + "ORDER BY tablet_id ASC, role ASC LIMIT ?"; + + private static final String PROXY_DUMMY_LOCATION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.tablet_id as tablet_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " + + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " + + ", A.spare1 as replica_type " + + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " + + "WHERE tenant_name = ? and database_name=? and table_name = ?"; + + private static final String PROXY_LOCATION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ A.tablet_id as tablet_id, A.svr_ip as svr_ip, A.sql_port as sql_port, " + + "A.table_id as table_id, A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, B.stop_time as stop_time " + + ", A.spare1 as replica_type " + + "FROM oceanbase.__all_virtual_proxy_schema A inner join oceanbase.__all_server B on A.svr_ip = B.svr_ip and A.sql_port = B.inner_port " + + "WHERE tenant_name = ? and database_name=? and table_name = ? and tablet_id = 0"; + + private static final String PROXY_LOCATION_SQL_PARTITION_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ * FROM ( " + + " SELECT A.tablet_id as tablet__id, A.svr_ip as svr_ip, A.sql_port as sql_port, A.table_id as table_id, " + + " A.role as role, A.replica_num as replica_num, A.part_num as part_num, B.svr_port as svr_port, B.status as status, " + + " B.stop_time as stop_time, A.spare1 as replica_type " + + " FROM oceanbase.__all_virtual_proxy_schema A " + + " INNER JOIN oceanbase.__all_server B ON A.svr_ip = B.svr_ip AND A.sql_port = B.inner_port " + + " WHERE A.tablet_id IN ({0}) AND A.tenant_name = ? AND A.database_name = ? AND A.table_name = ?) AS left_table " + + "LEFT JOIN (" + + " SELECT D.ls_id, D.tablet_id " + + " FROM oceanbase.__all_virtual_tablet_to_ls D " + + " INNER JOIN oceanbase.DBA_OB_TENANTS C ON D.tenant_id = C.tenant_id " + + " WHERE C.tenant_name = ? " + + ") AS right_table ON left_table.tablet__id = right_table.tablet_id;"; + + private static final String PROXY_LOCATION_SQL_PARTITION_BY_TABLETID_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ " + + " A.tablet_id as tablet_id, " + + " A.svr_ip as svr_ip, " + + " A.sql_port as sql_port, " + + " A.table_id as table_id, " + + " A.role as role, " + + " A.replica_num as replica_num, " + + " A.part_num as part_num, " + + " (SELECT B.svr_port FROM oceanbase.__all_server B WHERE A.svr_ip = B.svr_ip AND A.sql_port = B.inner_port) as svr_port, " + + " (SELECT B.status FROM oceanbase.__all_server B WHERE A.svr_ip = B.svr_ip AND A.sql_port = B.inner_port) as status, " + + " (SELECT B.stop_time FROM oceanbase.__all_server B WHERE A.svr_ip = B.svr_ip AND A.sql_port = B.inner_port) as stop_time, " + + " A.spare1 as replica_type, " + + " (SELECT D.ls_id FROM oceanbase.__all_virtual_tablet_to_ls D WHERE A.tablet_id = D.tablet_id AND D.tenant_id = " + + " (SELECT C.tenant_id FROM oceanbase.DBA_OB_TENANTS C WHERE C.tenant_name = ?)) as ls_id " + + "FROM " + + " oceanbase.__all_virtual_proxy_schema A " + + "WHERE " + + " A.tablet_id = ? " + + " AND A.tenant_name = ? " + + " AND A.database_name = ? " + + " AND A.table_name = ?;"; + + private static final String PROXY_FIRST_PARTITION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ part_id, part_name, tablet_id, high_bound_val, sub_part_num " + + "FROM oceanbase.__all_virtual_proxy_partition " + + "WHERE tenant_name = ? and table_id = ? LIMIT ?;"; + + private static final String PROXY_SUB_PARTITION_SQL_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ sub_part_id, part_name, tablet_id, high_bound_val " + + "FROM oceanbase.__all_virtual_proxy_sub_partition " + + "WHERE tenant_name = ? and table_id = ? LIMIT ?;"; + + private static final String PROXY_SERVER_STATUS_INFO_V4 = "SELECT ss.svr_ip, ss.zone, zs.region, zs.idc as idc " + + "FROM DBA_OB_SERVERS ss, DBA_OB_ZONES zs " + + "WHERE zs.zone = ss.zone ;"; + + private static final String home = System.getProperty( + "user.home", + "/home/admin"); + + private static final String TABLE_GROUP_GET_TABLE_NAME_V4 = "SELECT /*+READ_CONSISTENCY(WEAK)*/ table_name " + + "FROM oceanbase.CDB_OB_TABLEGROUP_TABLES " + + "WHERE tablegroup_name = ? and tenant_id = ? limit 1;"; + + private static final int TEMPLATE_PART_ID = -1; // limit the size of get tableEntry location from remote each time - private static final int MAX_TABLET_NUMS_EPOCH = 300; + private static final int MAX_TABLET_NUMS_EPOCH = 300; private abstract static class TableEntryRefreshWithPriorityCallback { abstract T execute(ObServerAddr obServerAddr) throws ObTableEntryRefreshException; @@ -735,7 +743,7 @@ private static TableEntry getTableEntryFromRemote(Connection connection, TableEn } } } - + if (ObGlobal.obVsnMajor() >= 4) { // only set empty partitionEntry ObPartitionEntry partitionEntry = new ObPartitionEntry(); @@ -857,11 +865,11 @@ public static TableEntry getTableEntryLocationFromRemote(Connection connection, String sql = genLocationSQLByTabletId(); try { ps = connection.prepareStatement(sql); - ps.setLong(1, tabletId); - ps.setString(2, key.getTenantName()); - ps.setString(3, key.getDatabaseName()); - ps.setString(4, key.getTableName()); - ps.setString(5, key.getTenantName()); + ps.setString(1, key.getTenantName()); + ps.setLong(2, tabletId); + ps.setString(3, key.getTenantName()); + ps.setString(4, key.getDatabaseName()); + ps.setString(5, key.getTableName()); rs = ps.executeQuery(); getPartitionLocationFromResultSetByTablet(tableEntry, rs, partitionEntry, tabletId); } catch (Exception e) { @@ -912,8 +920,8 @@ public static TableEntry getTableEntryLocationFromRemote(Connection connection, } catch (Exception e) { RUNTIME.error(LCD.convert("01-00010"), key, partitionNum, tableEntry, e); throw new ObTablePartitionLocationRefreshException(format( - "fail to get partition location entry from remote entryKey = %s partNum = %d tableEntry =%s " - + "offset =%d epoch =%d", key, partitionNum, tableEntry, i, epoch), e); + "fail to get partition location entry from remote entryKey = %s partNum = %d tableEntry =%s " + + "offset =%d epoch =%d", key, partitionNum, tableEntry, i, epoch), e); } finally { try { if (null != rs) { @@ -1205,46 +1213,47 @@ private static ObPartitionEntry getPartitionLocationFromResultSetByTablet(TableE ObPartitionLocationInfo partitionLocationInfo = partitionEntry.getPartitionInfo(tabletId); - partitionLocationInfo.rwLock.writeLock().lock(); - try { - while (rs.next()) { - ReplicaLocation replica = buildReplicaLocation(rs); - - long partitionId = (ObGlobal.obVsnMajor() >= 4) ? rs.getLong("tablet_id") : rs - .getLong("partition_id"); - long lsId = ObGlobal.obVsnMajor() >= 4 ? rs.getLong("ls_id") : INVALID_LS_ID; - if (rs.wasNull() && ObGlobal.obVsnMajor() >= 4) { - lsId = INVALID_LS_ID; // For non-partitioned table - } - partitionLocationInfo.setTabletLsId(lsId); - - if (ObGlobal.obVsnMajor() < 4 && tableEntry.isPartitionTable() - && tableEntry.getPartitionInfo().getSubPartDesc() != null) { - partitionId = ObPartIdCalculator.getPartIdx(partitionId, tableEntry - .getPartitionInfo().getSubPartDesc().getPartNum()); - } + while (rs.next()) { + ReplicaLocation replica = buildReplicaLocation(rs); + long partitionId = (ObGlobal.obVsnMajor() >= 4) ? rs.getLong("tablet_id") : rs + .getLong("partition_id"); + long lsId = ObGlobal.obVsnMajor() >= 4 ? rs.getLong("ls_id") : INVALID_LS_ID; + if (rs.wasNull() && ObGlobal.obVsnMajor() >= 4) { + lsId = INVALID_LS_ID; // For non-partitioned table + } - if (!replica.isValid()) { - RUNTIME - .warn(format( - "Replica is invalid; continuing. Replica=%s, PartitionId/TabletId=%d, TableId=%d", - replica, partitionId, tableEntry.getTableId())); - continue; - } - ObPartitionLocation location = partitionLocationInfo.getPartitionLocation(); - if (location == null) { - location = new ObPartitionLocation(); - partitionLocationInfo.updateLocation(location); + if (ObGlobal.obVsnMajor() < 4 && tableEntry.isPartitionTable() + && tableEntry.getPartitionInfo().getSubPartDesc() != null) { + partitionId = ObPartIdCalculator.getPartIdx(partitionId, tableEntry + .getPartitionInfo().getSubPartDesc().getPartNum()); + } + if (!replica.isValid()) { + RUNTIME + .warn(format( + "Replica is invalid; continuing. Replica=%s, PartitionId/TabletId=%d, TableId=%d", + replica, partitionId, tableEntry.getTableId())); + continue; + } + ObPartitionLocation location = partitionLocationInfo.getPartitionLocation(); + if (location == null) { + partitionLocationInfo.rwLock.writeLock().lock(); + try { + location = partitionLocationInfo.getPartitionLocation(); + if (location == null) { + location = new ObPartitionLocation(); + partitionLocationInfo.updateLocation(location, lsId); + } + } finally { + partitionLocationInfo.rwLock.writeLock().unlock(); } - location.addReplicaLocation(replica); + } + location.addReplicaLocation(replica); - if (partitionLocationInfo.initialized.compareAndSet(false, true)) { - partitionLocationInfo.initializationLatch.countDown(); - } + if (partitionLocationInfo.initialized.compareAndSet(false, true)) { + partitionLocationInfo.initializationLatch.countDown(); } - } finally { - partitionLocationInfo.rwLock.writeLock().unlock(); } + return partitionEntry; } diff --git a/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionLocationInfo.java b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionLocationInfo.java index 8b9181b9..b12b7ba0 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionLocationInfo.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionLocationInfo.java @@ -40,17 +40,24 @@ public ObPartitionLocation getPartitionLocation() { } } - public void updateLocation(ObPartitionLocation newLocation) { - this.partitionLocation = newLocation; - this.lastUpdateTime = System.currentTimeMillis(); + public void updateLocation(ObPartitionLocation newLocation, Long tabletLsId) { + rwLock.writeLock().lock(); + try { + this.partitionLocation = newLocation; + this.tabletLsId = tabletLsId; + this.lastUpdateTime = System.currentTimeMillis(); + } finally { + rwLock.writeLock().unlock(); + } } public Long getTabletLsId() { - return tabletLsId; - } - - public void setTabletLsId(Long tabletLsId) { - this.tabletLsId = tabletLsId; + rwLock.readLock().lock(); + try { + return tabletLsId; + } finally { + rwLock.readLock().unlock(); + } } public Long getLastUpdateTime() { diff --git a/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java b/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java index 5da49bbc..7662aa69 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java +++ b/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/AbstractQueryStreamResult.java @@ -35,6 +35,7 @@ import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.ObTableEntityType; import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.ObTableStreamRequest; import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.QueryStreamResult; +import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.syncquery.ObTableQueryAsyncRequest; import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.syncquery.ObTableQueryAsyncResult; import com.alipay.oceanbase.rpc.table.ObTable; import com.alipay.oceanbase.rpc.table.ObTableParam; @@ -45,6 +46,8 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; +import static com.alipay.oceanbase.rpc.util.TableClientLoggerFactory.RUNTIME; + public abstract class AbstractQueryStreamResult extends AbstractPayload implements QueryStreamResult { @@ -61,14 +64,15 @@ public abstract class AbstractQueryStreamResult extends AbstractPayload implemen // global index: key is index table name (be like: __idx__) protected String indexTableName; protected ObTableEntityType entityType; - protected Map> expectant; // Map> + protected Map> expectant; protected List cacheProperties = new LinkedList(); protected LinkedList> cacheRows = new LinkedList>(); private LinkedList, ObTableQueryResult>> partitionLastResult = new LinkedList, ObTableQueryResult>>(); private ObReadConsistency readConsistency = ObReadConsistency.STRONG; // ObRowKey objs: [startKey, MIN_OBJECT, MIN_OBJECT] public List currentStartKey; - + protected ObTableClient client; + /* * Get pcode. */ @@ -228,7 +232,7 @@ protected ObPayload commonExecute(ObTableClient client, Logger logger, } else if (e instanceof ObTableException) { if ((((ObTableException) e).getErrorCode() == ResultCodes.OB_TABLE_NOT_EXIST.errorCode || ((ObTableException) e) .getErrorCode() == ResultCodes.OB_NOT_SUPPORTED.errorCode) - && ((ObTableQueryRequest) request).getTableQuery().isHbaseQuery() + && ((ObTableQueryAsyncRequest) request).getObTableQueryRequest().getTableQuery().isHbaseQuery() && client.getTableGroupInverted().get(indexTableName) != null) { // table not exists && hbase mode && table group exists , three condition both client.eraseTableGroupFromCache(tableName); @@ -549,9 +553,32 @@ public void init() throws Exception { return; } if (tableQuery.getBatchSize() == -1) { - for (Map.Entry> entry : expectant.entrySet()) { - // mark the refer partition - referToNewPartition(entry.getValue()); + if (!expectant.isEmpty()) { + Iterator>> it = expectant.entrySet() + .iterator(); + int retryTimes = 0; + while (it.hasNext()) { + Map.Entry> entry = it.next(); + try { + // try access new partition, async will not remove useless expectant + referToNewPartition(entry.getValue()); + } catch (Exception e) { + if (e instanceof ObTableNeedFetchAllException) { + setExpectant(refreshPartition(tableQuery, tableName)); + it = expectant.entrySet().iterator(); + retryTimes++; + if (retryTimes > client.getRuntimeRetryTimes()) { + RUNTIME.error("Fail to get refresh table entry response after {}", + retryTimes); + throw new ObTableRetryExhaustedException( + "Fail to get refresh table entry response after " + retryTimes); + + } + } else { + throw e; + } + } + } } expectant.clear(); } else { @@ -692,4 +719,19 @@ public ObReadConsistency getReadConsistency() { public void setReadConsistency(ObReadConsistency readConsistency) { this.readConsistency = readConsistency; } + + /** + * Get client. + * @return client + */ + public ObTableClient getClient() { + return client; + } + + /* + * Set client. + */ + public void setClient(ObTableClient client) { + this.client = client; + } } diff --git a/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryAsyncStreamResult.java b/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryAsyncStreamResult.java index cae7f39c..8a933d64 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryAsyncStreamResult.java +++ b/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryAsyncStreamResult.java @@ -44,7 +44,6 @@ public class ObTableClientQueryAsyncStreamResult extends AbstractQueryStreamResult { private static final Logger logger = LoggerFactory .getLogger(ObTableClientQueryStreamResult.class); - protected ObTableClient client; private boolean isEnd = true; private long sessionId = Constants.OB_INVALID_ID; private ObTableQueryAsyncRequest asyncRequest = new ObTableQueryAsyncRequest(); @@ -338,19 +337,7 @@ public void close() throws Exception { closeLastStreamResult(lastEntry.getValue()); } } - - public ObTableClient getClient() { - return client; - } - - /** - * Set client. - * @param client client want to set - */ - public void setClient(ObTableClient client) { - this.client = client; - } - + public boolean isEnd() { return isEnd; } diff --git a/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryStreamResult.java b/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryStreamResult.java index 2c0bf482..b091b4b7 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryStreamResult.java +++ b/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryStreamResult.java @@ -34,10 +34,8 @@ import java.util.concurrent.atomic.AtomicReference; public class ObTableClientQueryStreamResult extends AbstractQueryStreamResult { - private static final Logger logger = TableClientLoggerFactory .getLogger(ObTableClientQueryStreamResult.class); - protected ObTableClient client; protected ObTableQueryResult referToNewPartition(ObPair partIdWithObTable) throws Exception { @@ -84,19 +82,4 @@ protected Map> refreshPartition(ObTableQuery ta throws Exception { return buildPartitions(client, tableQuery, tableName); } - - /** - * Get client. - * @return client - */ - public ObTableClient getClient() { - return client; - } - - /* - * Set client. - */ - public void setClient(ObTableClient client) { - this.client = client; - } } diff --git a/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java b/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java index 6459398c..885cc96f 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java +++ b/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientLSBatchOpsImpl.java @@ -631,7 +631,7 @@ private void executeWithRetries( Map>>>> currentPartitions = new HashMap<>(); currentPartitions.put(entry.getKey(), entry.getValue()); - while (retryCount < maxRetries && !success) { + while (retryCount <= maxRetries && !success) { boolean allPartitionsSuccess = true; for (Map.Entry>>>> currentEntry : currentPartitions.entrySet()) { From 36d521c33084721eb426c5fe2f5f88a4b6346e5c Mon Sep 17 00:00:00 2001 From: JackShi148 Date: Tue, 5 Nov 2024 10:41:09 +0800 Subject: [PATCH 15/15] correct log level when refresh tableEntry cache expired --- .../com/alipay/oceanbase/rpc/ObTableClient.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java index b1c7ec09..8caf3298 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java +++ b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java @@ -1279,7 +1279,7 @@ public TableEntry getOrRefreshTableEntry(final String tableName, final boolean r RUNTIME.error("getOrRefreshTableEntry meet exception", e); throw e; } catch (ObTableServerCacheExpiredException e) { - RUNTIME.error("getOrRefreshTableEntry meet exception", e); + RUNTIME.warn("getOrRefreshTableEntry meet exception", e); if (logger.isInfoEnabled()) { logger.info("server addr is expired and it will refresh metadata."); @@ -1350,7 +1350,11 @@ public TableEntry refreshTableLocationByTabletId(TableEntry tableEntry, String t tableEntry.prepareForWeakRead(serverRoster.getServerLdcLocation()); } catch (ObTableNotExistException | ObTableServerCacheExpiredException e) { - RUNTIME.error("RefreshTableEntry encountered an exception", e); + if (e instanceof ObTableNotExistException) { + RUNTIME.error("RefreshTableEntry encountered an exception", e); + } else { + RUNTIME.warn("RefreshTableEntry encountered an exception", e); + } throw e; } catch (Exception e) { String errorMsg = String.format("Failed to get table entry. Key=%s, TabletId=%d, message=%s", tableEntryKey, tabletId, e.getMessage()); @@ -1432,7 +1436,7 @@ private TableEntry refreshTableEntry(TableEntry tableEntry, String tableName, bo RUNTIME.error("refreshTableEntry meet exception", e); throw e; } catch (ObTableServerCacheExpiredException e) { - RUNTIME.error("refreshTableEntry meet exception", e); + RUNTIME.warn("refreshTableEntry meet exception", e); throw e; } catch (Exception e) { RUNTIME.error(LCD.convert("01-00020"), tableEntryKey, tableEntry, e); @@ -1477,7 +1481,7 @@ private String refreshTableNameByTableGroup(String physicalTableName, String tab RUNTIME.error("refreshTableNameByTableGroup from tableGroup meet exception", e); throw e; } catch (ObTableServerCacheExpiredException e) { - RUNTIME.error("refreshTableEntry from tableGroup meet exception", e); + RUNTIME.warn("refreshTableEntry from tableGroup meet exception", e); throw e; } catch (Exception e) { RUNTIME.error("refreshTableEntry from tableGroup meet exception", tableEntryKey, @@ -2142,7 +2146,7 @@ public String tryGetTableNameFromTableGroupCache(final String tableGroupName, RUNTIME.error("getOrRefreshTableName from TableGroup meet exception", e); throw e; } catch (ObTableServerCacheExpiredException e) { - RUNTIME.error("getOrRefreshTableName from TableGroup meet exception", e); + RUNTIME.warn("getOrRefreshTableName from TableGroup meet exception", e); if (logger.isInfoEnabled()) { logger.info("server addr is expired and it will refresh metadata.");