diff --git a/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java b/src/main/java/com/alipay/oceanbase/rpc/ObTableClient.java index efd49dc5..203e1679 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,62 @@ 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); + try { + 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 < tableEntryRefreshLockTimeout) { + 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.prepareForWeakRead(serverRoster.getServerLdcLocation()); + } finally { + lock.unlock(); + } + + } catch (ObTableNotExistException | ObTableServerCacheExpiredException e) { + RUNTIME.error("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()); + RUNTIME.error(LCD.convert("01-00020"), tableEntryKey, tableEntry, e); + throw new ObTableEntryRefreshException(errorMsg, e); + } + + tableLocations.put(tableName, tableEntry); + tableEntryRefreshContinuousFailureCount.set(0); + + if (logger.isInfoEnabled()) { + logger.info("Refreshed table entry. DataSource: {}, TableName: {}, Key: {}, Entry: {}", + dataSourceName, tableName, tableEntryKey, JSON.toJSON(tableEntry)); + } + + return tableEntry; + } + /** * 刷新 table entry 元数据 * @param tableEntry @@ -1345,13 +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) { - tableEntry = loadTableEntryLocationWithPriority(serverRoster, // - tableEntryKey,// - tableEntry,// - tableEntryAcquireConnectTimeout,// - tableEntryAcquireSocketTimeout,// - serverAddressPriorityTimeout, // - serverAddressCachingTimeout, sysUA); + 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, // @@ -1360,7 +1419,6 @@ private TableEntry refreshTableEntry(TableEntry tableEntry, String tableName, bo tableEntryAcquireSocketTimeout,// serverAddressPriorityTimeout,// serverAddressCachingTimeout, sysUA); - if (tableEntry.isPartitionTable()) { switch (runningMode) { case HBASE: @@ -1544,17 +1602,15 @@ 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) { + return obPartitionLocationInfo.getPartitionLocation().getReplica(route); } /** @@ -1784,48 +1840,87 @@ 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(); - + ReplicaLocation replica = null; + long tabletId = getTabletIdByPartId(tableEntry, partId); + ObPartitionLocationInfo obPartitionLocationInfo = null; + if (ObGlobal.obVsnMajor() >= 4) { + + 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); - 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 = getOrRefreshTableEntry(tableName, true, waitForRefresh, false); - replica = getPartitionReplica(tableEntry, partId, route).getRight(); + 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); + } + + 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); + + if (obTable == null) { + RUNTIME.error("Cannot get table by addr: " + addr); + throw new ObTableGetException("Cannot get table by addr: " + addr); + } } + 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); + } - if (obTable == null) { - RUNTIME.error("cannot get table by addr: " + addr); - throw new ObTableGetException("cannot get table by addr: " + addr); + 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) { - 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); - - addr.recordAccess(); - return new ObPair(partitionReplica.getLeft(), param); + param.setPartitionId(tabletId); + return param; } /** @@ -1840,39 +1935,58 @@ 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))); + ObServerRoute route) throws Exception { + List> replicas = new ArrayList<>(); + + if (!tableEntry.isPartitionTable() || tableEntry.getPartitionInfo().getLevel() == ObPartitionLevel.LEVEL_ZERO) { + long tabletId = getTabletIdByPartId(tableEntry, 0L); + ObPartitionLocationInfo locationInfo = getOrRefreshPartitionInfo(tableEntry, tableName, tabletId); + replicas.add(new ObPair<>(tabletId, getPartitionLocation(locationInfo, route))); return replicas; - } else if (tableEntry.getPartitionInfo().getLevel() == ObPartitionLevel.LEVEL_ONE) { - List partIds = tableEntry.getPartitionInfo().getFirstPartDesc() + } + + 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 = getOrRefreshPartitionInfo(tableEntry, tableName, tabletId); + 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); - for (Long partId : partIds) { - replicas.add(new ObPair(partId, getPartitionLocation( - tableEntry, partId, route))); - } - } 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))); - } + } 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; + public 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,7 +2061,7 @@ public List> getTables(String tableName, ObTableQuery } } - List> partIdWithReplicaList = getPartitionReplica(tableEntry, + List> partIdWithReplicaList = getPartitionReplica(tableEntry, tableName, startRow, startInclusive, endRow, endInclusive, route); // obTableParams -> List> @@ -1976,12 +2090,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); 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 e3d26f0d..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 @@ -122,31 +122,32 @@ 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); if (needFetchAll(resultCode.getRcode(), resultCode.getPcode())) { - throw new ObTableNeedFetchAllException(errMessage); + throw new ObTableNeedFetchAllException(errMessage, resultCode.getRcode()); } else if (needFetchPartial(resultCode.getRcode())) { - throw new ObTableRoutingWrongException(errMessage); + throw new ObTableRoutingWrongException(errMessage, resultCode.getRcode()); } else { // Encountered an unexpected RoutingWrong error code, // possibly due to the client error code version being behind the observer's version. // Attempting a full refresh here // and delegating to the upper-level call to determine whether to throw the exception to the user based on the retry result. logger.warn("get unexpected error code: {}", response.getMessage()); - throw new ObTableNeedFetchAllException(errMessage); + 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); if (needFetchAll(resultCode.getRcode(), resultCode.getPcode())) { - throw new ObTableNeedFetchAllException(errMessage); + throw new ObTableNeedFetchAllException(errMessage, resultCode.getRcode()); } else if (needFetchPartial(resultCode.getRcode())) { - throw new ObTableRoutingWrongException(errMessage); + throw new ObTableRoutingWrongException(errMessage, resultCode.getRcode()); } else { ExceptionUtil.throwObTableException(conn.getObTable().getIp(), conn .getObTable().getPort(), response.getHeader().getTraceId1(), response @@ -193,6 +194,8 @@ 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 + || 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/location/LocationUtil.java b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java index 20d657e0..ecc37f25 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/LocationUtil.java @@ -56,135 +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)*/ 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_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; @@ -374,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) { @@ -457,6 +479,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 */ @@ -682,9 +735,16 @@ private static TableEntry getTableEntryFromRemote(Connection connection, TableEn } } } - - // get location info - getTableEntryLocationFromRemote(connection, key, tableEntry); + + 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()) { @@ -719,6 +779,17 @@ 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) { + 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; @@ -776,6 +847,44 @@ 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. */ @@ -792,6 +901,7 @@ public static TableEntry getTableEntryLocationFromRemote(Connection connection, 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()); @@ -802,8 +912,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) { @@ -1081,6 +1191,63 @@ 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); + + 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()); + } + + 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); + } + location.addReplicaLocation(replica); + + if (partitionLocationInfo.initialized.compareAndSet(false, true)) { + partitionLocationInfo.initializationLatch.countDown(); + } + } + } finally { + partitionLocationInfo.rwLock.writeLock().unlock(); + } + 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..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 @@ -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,9 @@ public class TableEntry { // partition location private TableEntryKey tableEntryKey = null; private volatile ObPartitionEntry partitionEntry = null; - + + public ConcurrentHashMap refreshLockMap = new ConcurrentHashMap<>(); + /* * Is valid. */ @@ -218,8 +221,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..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 @@ -21,13 +21,23 @@ 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) + private ConcurrentHashMap partitionInfos = new ConcurrentHashMap<>(); + + public ObPartitionLocationInfo getPartitionInfo(long tabletId) { + return partitionInfos.computeIfAbsent(tabletId, id -> new ObPartitionLocationInfo()); + } + public Map getPartitionLocation() { return partitionLocation; } @@ -39,6 +49,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 +106,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..8b9181b9 --- /dev/null +++ b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObPartitionLocationInfo.java @@ -0,0 +1,64 @@ +/*- + * #%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.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 = 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(); + try { + return partitionLocation; + } finally { + rwLock.readLock().unlock(); + } + } + + public void updateLocation(ObPartitionLocation newLocation) { + this.partitionLocation = newLocation; + this.lastUpdateTime = System.currentTimeMillis(); + } + + 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/location/model/partition/ObRangePartDesc.java b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObRangePartDesc.java index b41871e9..e17d7bf2 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObRangePartDesc.java +++ b/src/main/java/com/alipay/oceanbase/rpc/location/model/partition/ObRangePartDesc.java @@ -66,7 +66,8 @@ public ObRangePartDesc() { public List getOrderedCompareColumnTypes() { return orderedCompareColumnTypes; } - private List completeWorks; + + private List completeWorks; /* * Set ordered compare column types. @@ -299,12 +300,12 @@ public int getBoundsIdx(boolean isScan, Row rowKey) { try { List evalParams = evalRowKeyValues(rowKey); List comparableElement = super.initComparableElementByTypes(evalParams, - this.orderedCompareColumns); + this.orderedCompareColumns); ObPartitionKey searchKey = ObPartitionKey.getInstance(orderedCompareColumns, - comparableElement); + comparableElement); int pos = upperBound(this.bounds, new ObComparableKV(searchKey, - (long) -1)); + (long) -1)); if (pos >= this.bounds.size()) { if (isScan) { // if range is bigger than rangeMax while scanning @@ -312,7 +313,7 @@ public int getBoundsIdx(boolean isScan, Row rowKey) { return this.bounds.size() - 1; } throw new ArrayIndexOutOfBoundsException("Table has no partition for value in " - + this.getPartExpr()); + + this.getPartExpr()); } else { return pos; } 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/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), // 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 7cd75788..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; @@ -29,6 +31,7 @@ import com.alipay.oceanbase.rpc.protocol.payload.ResultCodes; import com.alipay.oceanbase.rpc.protocol.payload.impl.ObObj; import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.ObTableApiMove; +import com.alipay.oceanbase.rpc.protocol.payload.impl.ObRowKey; 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; @@ -63,6 +66,8 @@ public abstract class AbstractQueryStreamResult extends AbstractPayload implemen 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; /* * Get pcode. @@ -126,7 +131,6 @@ protected ObPayload commonExecute(ObTableClient client, Logger logger, } tryTimes++; try { - // 重试时重新 getTable if (tryTimes > 1) { if (client.isOdpMode()) { subObTable = client.getOdpTable(); @@ -243,6 +247,11 @@ protected ObPayload commonExecute(ObTableClient client, Logger logger, "tablename:{} partition id:{} stream query retry while meet Exception needing refresh, errorCode: {} , retry times {}", indexTableName, partIdWithIndex.getLeft(), ((ObTableException) e).getErrorCode(), tryTimes, e); + // tablet not exists, refresh table entry + if (e instanceof ObTableNeedFetchAllException) { + client.getOrRefreshTableEntry(tableName, true, true, true); + throw e; + } } else { client.calculateContinuousFailure(indexTableName, e.getMessage()); throw e; @@ -270,7 +279,7 @@ public boolean next() throws Exception { lock.lock(); try { // firstly, refer to the cache - if (cacheRows.size() > 0) { + if (!cacheRows.isEmpty()) { nextRow(); return true; } @@ -293,6 +302,39 @@ public boolean next() throws Exception { // lastly, refer to the new partition boolean hasNext = false; List>> referPartition = new ArrayList>>(); + Iterator>> it = expectant.entrySet() + .iterator(); + while (it.hasNext()) { + Map.Entry> entry = it.next(); + referPartition.add(entry); + try { + // Mark the refer partition + referPartition.add(entry); + + // Try accessing the new partition + ObTableQueryResult tableQueryResult = (ObTableQueryResult) referToNewPartition(entry + .getValue()); + + if (tableQueryResult.getRowCount() == 0) { + continue; + } + + hasNext = true; + nextRow(); + break; + + } catch (Exception e) { + if (e instanceof ObTableNeedFetchAllException) { + setExpectant(refreshPartition(tableQuery, tableName)); + // Reset the iterator to start over + it = expectant.entrySet().iterator(); + referPartition.clear(); // Clear the referPartition if needed + } else { + throw e; + } + } + } + for (Map.Entry> entry : expectant.entrySet()) { // mark the refer partition referPartition.add(entry); @@ -317,9 +359,58 @@ public boolean next() throws Exception { } } + protected Map> buildPartitions(ObTableClient client, ObTableQuery tableQuery, String tableName) throws Exception { + Map> partitionObTables = new HashMap<>(); + String indexName = tableQuery.getIndexName(); + String indexTableName = null; + + if (!client.isOdpMode()) { + indexTableName = client.getIndexTableName(tableName, indexName, tableQuery.getScanRangeColumns(), false); + } + + for (ObNewRange range : tableQuery.getKeyRanges()) { + ObRowKey startKey = range.getStartKey(); + int startKeySize = startKey.getObjs().size(); + ObRowKey endKey = range.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).isMinObj() || startKey.getObj(i).isMaxObj() ? + startKey.getObj(i) : startKey.getObj(i).getValue(); + } + + for (int i = 0; i < endKeySize; i++) { + end[i] = endKey.getObj(i).isMinObj() || endKey.getObj(i).isMaxObj() ? + endKey.getObj(i) : endKey.getObj(i).getValue(); + } + + ObBorderFlag borderFlag = range.getBorderFlag(); + List> pairs = client.getTables(indexTableName, + tableQuery, start, borderFlag.isInclusiveStart(), end, borderFlag.isInclusiveEnd(), + false, false); + + if (tableQuery.getScanOrder() == ObScanOrder.Reverse) { + for (int i = pairs.size() - 1; i >= 0; i--) { + partitionObTables.put(pairs.get(i).getLeft(), pairs.get(i)); + } + } else { + for (ObPair pair : pairs) { + partitionObTables.put(pair.getLeft(), pair); + } + } + } + + return partitionObTables; + } + protected void nextRow() { rowIndex = rowIndex + 1; row = cacheRows.poll(); + if (row != null) { + currentStartKey = row; + } } protected void checkStatus() throws IllegalStateException { @@ -398,6 +489,10 @@ protected abstract ObTableQueryAsyncResult executeAsync(ObPair> refreshPartition(ObTableQuery tableQuery, + String tableName) + throws Exception; + protected void cacheResultRows(ObTableQueryResult tableQueryResult) { cacheRows.addAll(tableQueryResult.getPropertiesRows()); cacheProperties = tableQueryResult.getPropertiesNames(); diff --git a/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/ObTableQuery.java b/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/ObTableQuery.java index 56bdd2d6..13371fcc 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/ObTableQuery.java +++ b/src/main/java/com/alipay/oceanbase/rpc/protocol/payload/impl/execute/query/ObTableQuery.java @@ -21,14 +21,18 @@ import com.alipay.oceanbase.rpc.table.ObHBaseParams; import com.alipay.oceanbase.rpc.table.ObKVParams; import com.alipay.oceanbase.rpc.protocol.payload.AbstractPayload; +import com.alipay.oceanbase.rpc.protocol.payload.impl.ObObj; +import com.alipay.oceanbase.rpc.protocol.payload.impl.ObRowKey; import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.aggregation.ObTableAggregationSingle; import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.aggregation.ObTableAggregationType; import com.alipay.oceanbase.rpc.util.Serialization; import io.netty.buffer.ByteBuf; +import java.util.Arrays; import java.util.LinkedList; import java.util.List; +import static com.alipay.oceanbase.rpc.util.ByteUtil.*; import static com.alipay.oceanbase.rpc.util.Serialization.encodeObUniVersionHeader; import static com.alipay.oceanbase.rpc.util.Serialization.getObUniVersionHeaderLength; @@ -68,6 +72,47 @@ public class ObTableQuery extends AbstractPayload { private List aggregations = new LinkedList<>(); private ObKVParams obKVParams; + + public void adjustStartKey(List key) throws IllegalArgumentException { + List keyRanges = getKeyRanges(); + for (ObNewRange range : keyRanges) { + if (key != null && isKeyInRange(range, key)) { + ObRowKey newStartKey; + if (getScanOrder() == ObScanOrder.Forward) { + // get the real rowkey + newStartKey = ObRowKey.getInstance(new Object[]{key.get(0).getValue(), ObObj.getMax(), ObObj.getMax()}); + } else { + newStartKey = ObRowKey.getInstance(new Object[]{key.get(0).getValue(), ObObj.getMax(), ObObj.getMax()}); + } + range.setStartKey(newStartKey); + return; + } + } + /* keyRanges not changed */ + } + + private byte[] parseStartKeyToBytes(List key) { + if (key != null) { + ObObj obObjKey = key.get(0); + return obObjKey.encode(); + } + return new byte[0]; + } + + private boolean isKeyInRange(ObNewRange range, List key) { + byte[] startKeyBytes = parseStartKeyToBytes(range.getStartKey().getObjs()); + byte[] endKeyBytes = parseStartKeyToBytes(range.getEndKey().getObjs()); + byte[] keyBytes = parseStartKeyToBytes(key); + + int startComparison = compareByteArrays(startKeyBytes, keyBytes); + int endComparison = compareByteArrays(endKeyBytes, keyBytes); + + boolean withinStart = startComparison <= 0; + boolean withinEnd = endComparison > 0; + + return withinStart && withinEnd; + } + /* * Check filter. 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 79096c31..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,25 +17,30 @@ 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; -import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.query.AbstractQueryStreamResult; -import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.query.ObTableQueryRequest; -import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.query.ObTableQueryResult; +import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.query.*; import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.syncquery.ObQueryOperationType; 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.ObTableParam; +import com.alipay.oceanbase.rpc.util.ByteUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.*; import java.util.concurrent.atomic.AtomicReference; +import static com.alipay.oceanbase.rpc.util.TableClientLoggerFactory.RUNTIME; + public class ObTableClientQueryAsyncStreamResult extends AbstractQueryStreamResult { private static final Logger logger = LoggerFactory .getLogger(ObTableClientQueryStreamResult.class); @@ -50,6 +55,7 @@ public void init() throws Exception { if (initialized) { return; } + int maxRetries = client.getTableEntryRefreshTryTimes(); // init request ObTableQueryRequest request = new ObTableQueryRequest(); request.setTableName(tableName); @@ -66,8 +72,31 @@ public void init() throws Exception { if (!expectant.isEmpty()) { Iterator>> it = expectant.entrySet() .iterator(); - Map.Entry> firstEntry = it.next(); - referToNewPartition(firstEntry.getValue()); + int retryTimes = 0; + while (it.hasNext()) { + Map.Entry> firstEntry = it.next(); + try { + // try access new partition, async will not remove useless expectant + referToNewPartition(firstEntry.getValue()); + break; + } catch (Exception e) { + if (e instanceof ObTableNeedFetchAllException) { + setExpectant(refreshPartition(this.asyncRequest.getObTableQueryRequest() + .getTableQuery(), tableName)); + 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); + + } + } else { + throw e; + } + } + } if (isEnd()) it.remove(); } @@ -145,6 +174,13 @@ protected void closeLastStreamResult(ObPair partIdWithObTabl } } + @Override + protected Map> refreshPartition(ObTableQuery tableQuery, + String tableName) + throws Exception { + return buildPartitions(client, tableQuery, tableName); + } + @Override public boolean next() throws Exception { checkStatus(); @@ -160,10 +196,32 @@ public boolean next() throws Exception { if (!isEnd() && !expectant.isEmpty()) { Iterator>> it = expectant.entrySet() .iterator(); - Map.Entry> lastEntry = it.next(); - // try access new partition, async will not remove useless expectant - referToLastStreamResult(lastEntry.getValue()); + Map.Entry> lastEntry = it.next(); + try { + // try access new partition, async will not remove useless expectant + referToLastStreamResult(lastEntry.getValue()); + } catch (Exception e) { + if (e instanceof ObTableNeedFetchAllException) { + + 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; + } + } // remove useless expectant if it is end if (isEnd()) it.remove(); @@ -178,10 +236,36 @@ public boolean next() throws Exception { boolean hasNext = false; Iterator>> it = expectant.entrySet() .iterator(); + int retryTimes = 0; while (it.hasNext()) { Map.Entry> entry = it.next(); - // try access new partition, async will not remove useless expectant - referToNewPartition(entry.getValue()); + try { + // try access new partition, async will not remove useless expectant + referToNewPartition(entry.getValue()); + } catch (Exception e) { + if (e instanceof ObTableNeedFetchAllException) { + 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()) { + 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 { + throw e; + } + } // remove useless expectant if it is end if (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 ef55baeb..2c0bf482 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryStreamResult.java +++ b/src/main/java/com/alipay/oceanbase/rpc/stream/ObTableClientQueryStreamResult.java @@ -22,6 +22,7 @@ import com.alipay.oceanbase.rpc.location.model.partition.ObPair; import com.alipay.oceanbase.rpc.protocol.payload.ObPayload; import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.query.AbstractQueryStreamResult; +import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.query.ObTableQuery; import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.query.ObTableQueryRequest; import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.query.ObTableQueryResult; import com.alipay.oceanbase.rpc.protocol.payload.impl.execute.syncquery.ObTableQueryAsyncResult; @@ -29,6 +30,7 @@ import com.alipay.oceanbase.rpc.util.TableClientLoggerFactory; import org.slf4j.Logger; +import java.util.Map; import java.util.concurrent.atomic.AtomicReference; public class ObTableClientQueryStreamResult extends AbstractQueryStreamResult { @@ -76,6 +78,13 @@ protected ObTableQueryAsyncResult executeAsync(ObPair partId throw new IllegalArgumentException("not support this execute"); } + @Override + protected Map> refreshPartition(ObTableQuery tableQuery, + String tableName) + throws Exception { + return buildPartitions(client, tableQuery, tableName); + } + /** * Get client. * @return client 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; } /* 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 857c3d5e..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; @@ -217,25 +218,41 @@ public List executeWithResult() throws Exception { return results; } - public Map>>> partitionPrepare() - throws Exception { - // consistent can not be sure - List operations = batchOperation.getTableOperations(); - Map>>> partitionOperationsMap = new HashMap>>>(); + // Helper method to calculate RowKey from ObTableOperation + private Object[] calculateRowKey(ObTableOperation operation) { + ObRowKey rowKeyObject = operation.getEntity().getRowKey(); + int rowKeySize = rowKeyObject.getObjs().size(); + Object[] rowKey = new Object[rowKeySize]; + for (int j = 0; j < rowKeySize; j++) { + rowKey[j] = rowKeyObject.getObj(j).getValue(); + } + return rowKey; + } + public List extractOperations(List> operationsPairs) { + List operations = new ArrayList<>(operationsPairs.size()); + for (ObPair pair : operationsPairs) { + operations.add(pair.getRight()); + } + return operations; + } + + public Map>>> prepareOperations(List operations) throws Exception { + Map>>> partitionOperationsMap = new HashMap<>(); + if (obTableClient.isOdpMode()) { - ObPair>> obTableOperations = new ObPair>>( - new ObTableParam(obTableClient.getOdpTable()), - new ArrayList>()); + ObPair>> obTableOperations = new ObPair<>( + new ObTableParam(obTableClient.getOdpTable()), + new ArrayList<>()); for (int i = 0; i < operations.size(); i++) { ObTableOperation operation = operations.get(i); obTableOperations.getRight().add( - new ObPair(i, operation)); + new ObPair<>(i, operation)); } partitionOperationsMap.put(0L, obTableOperations); return partitionOperationsMap; } - + for (int i = 0; i < operations.size(); i++) { ObTableOperation operation = operations.get(i); ObRowKey rowKeyObject = operation.getEntity().getRowKey(); @@ -248,18 +265,20 @@ public Map>>> tableName, rowKey, false, false, obTableClient.getRoute(batchOperation.isReadOnly())); ObPair>> obTableOperations = partitionOperationsMap - .get(tableObPair.getLeft()); - if (obTableOperations == null) { - obTableOperations = new ObPair>>( - tableObPair.getRight(), new ArrayList>()); - partitionOperationsMap.put(tableObPair.getLeft(), obTableOperations); - } - obTableOperations.getRight().add(new ObPair(i, operation)); + .computeIfAbsent(tableObPair.getLeft(), k -> new ObPair<>( + tableObPair.getRight(), new ArrayList<>())); + obTableOperations.getRight().add(new ObPair<>(i, operation)); } - return partitionOperationsMap; } + public Map>>> partitionPrepare() + throws Exception { + // consistent can not be sure + List operations = batchOperation.getTableOperations(); + return prepareOperations(operations); + } + /* * Partition execute. */ @@ -334,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()); } @@ -363,6 +383,7 @@ public void partitionExecute(ObTableOperationResult[] results, } catch (Exception ex) { if (obTableClient.isOdpMode()) { if ((tryTimes - 1) < obTableClient.getRuntimeRetryTimes()) { + assert ex instanceof ObTableException; logger .warn( "batch ops execute while meet Exception, tablename:{}, errorCode: {} , errorMsg: {}, try times {}", @@ -399,9 +420,10 @@ public void partitionExecute(ObTableOperationResult[] results, tableName, partId, ((ObTableException) ex).getErrorCode(), tryTimes, ex); if (ex instanceof ObTableNeedFetchAllException) { - needFetchAllRouteInfo = true; - // reset failure count while fetch all route info - obTableClient.resetExecuteContinuousFailureCount(tableName); + // refresh table info + obTableClient.getOrRefreshTableEntry(tableName, needRefreshTableEntry, + obTableClient.isTableEntryRefreshIntervalWait(), true); + throw ex; } } else { obTableClient.calculateContinuousFailure(tableName, ex.getMessage()); @@ -425,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(); @@ -483,50 +504,94 @@ public void partitionExecute(ObTableOperationResult[] results, obTableClient.getslowQueryMonitorThreshold()); } - /* - * Execute internal. - */ - public ObTableBatchOperationResult executeInternal() throws Exception { + private boolean shouldRetry(Throwable throwable) { + return throwable instanceof ObTableNeedFetchAllException; + } + private void executeWithRetries(ObTableOperationResult[] results, Map.Entry>>> entry, int maxRetries) throws Exception { + int retryCount = 0; + boolean success = false; + + Map>>> currentPartitions = new HashMap<>(); + currentPartitions.put(entry.getKey(), entry.getValue()); + + while (retryCount < maxRetries && !success) { + boolean allPartitionsSuccess = true; + + for (Map.Entry>>> currentEntry : currentPartitions.entrySet()) { + try { + partitionExecute(results, currentEntry); + } catch (Exception e) { + if (shouldRetry(e)) { + retryCount++; + List failedOperations = extractOperations(currentEntry.getValue().getRight()); + currentPartitions = prepareOperations(failedOperations); + allPartitionsSuccess = false; + break; + } else { + throw e; + } + } + } + + if (allPartitionsSuccess) { + success = true; + } + } + + if (!success) { + throw new ObTableUnexpectedException("Failed to execute operation after retrying " + maxRetries + " times."); + } + } + + public ObTableBatchOperationResult executeInternal() throws Exception { if (tableName == null || tableName.isEmpty()) { throw new IllegalArgumentException("table name is null"); } long start = System.currentTimeMillis(); List operations = batchOperation.getTableOperations(); - ObTableOperationResult[] obTableOperationResults = null; - if (returnOneResult) { - obTableOperationResults = new ObTableOperationResult[1]; - } else { - obTableOperationResults = new ObTableOperationResult[operations.size()]; - } + ObTableOperationResult[] obTableOperationResults = returnOneResult ? new ObTableOperationResult[1] + : new ObTableOperationResult[operations.size()]; Map>>> partitions = partitionPrepare(); long getTableTime = System.currentTimeMillis(); final Map context = ThreadLocalMap.getContextMap(); + final int maxRetries = obTableClient.getRuntimeRetryTimes(); + + ConcurrentTaskExecutor executor = null; if (executorService != null && !executorService.isShutdown() && partitions.size() > 1) { - final ConcurrentTaskExecutor executor = new ConcurrentTaskExecutor(executorService, - partitions.size()); - for (final Map.Entry>>> entry : partitions - .entrySet()) { - ObTableOperationResult[] finalObTableOperationResults = obTableOperationResults; - executor.execute(new ConcurrentTask() { - /* - * Do task. - */ - @Override - public void doTask() { - try { - ThreadLocalMap.transmitContextMap(context); - partitionExecute(finalObTableOperationResults, entry); - } catch (Exception e) { - logger.error(LCD.convert("01-00026"), e); - executor.collectExceptions(e); - } finally { - ThreadLocalMap.reset(); + executor = new ConcurrentTaskExecutor(executorService, partitions.size()); + } + for (final Map.Entry>>> entry : partitions + .entrySet()) { + try { + if (executor != null) { + // Concurrent execution + ConcurrentTaskExecutor finalExecutor = executor; + executor.execute(new ConcurrentTask() { + @Override + public void doTask() { + try { + ThreadLocalMap.transmitContextMap(context); + executeWithRetries(obTableOperationResults, entry, maxRetries); + } catch (Exception e) { + logger.error(LCD.convert("01-00026"), e); + finalExecutor.collectExceptions(e); + } finally { + ThreadLocalMap.reset(); + } } - } - }); + }); + } else { + // Sequential execution + executeWithRetries(obTableOperationResults, entry, maxRetries); + } + } catch (Exception e) { + logger.error("Error executing retry: {}", entry.getKey(), e); + throw e; } + } + if (executor != null) { long estimate = obTableClient.getRuntimeBatchMaxWait() * 1000L * 1000L; try { while (estimate > 0) { @@ -538,7 +603,7 @@ public void doTask() { "Batch Concurrent Execute interrupted", e); } - if (executor.getThrowableList().size() > 0) { + if (!executor.getThrowableList().isEmpty()) { throw new ObTableUnexpectedException("Batch Concurrent Execute Error", executor.getThrowableList().get(0)); } @@ -553,22 +618,11 @@ public void doTask() { executor.stop(); } - if (executor.getThrowableList().size() > 0) { - throw new ObTableUnexpectedException("Batch Concurrent Execute Error", executor - .getThrowableList().get(0)); - } - if (!executor.isComplete()) { throw new ObTableUnexpectedException("Batch Concurrent Execute Error [" + obTableClient.getRpcExecuteTimeout() + "]/ms"); } - - } else { - for (final Map.Entry>>> entry : partitions - .entrySet()) { - partitionExecute(obTableOperationResults, entry); - } } ObTableBatchOperationResult batchOperationResult = new ObTableBatchOperationResult(); 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 d58c4f4e..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; @@ -279,7 +280,7 @@ public void addOperation(Mutation mutation) throws Exception { * Execute. */ public List execute() throws Exception { - List results = new ArrayList(batchOperation.size()); + List results = new ArrayList<>(batchOperation.size()); for (ObTableSingleOpResult result : executeInternal()) { int errCode = result.getHeader().getErrno(); if (errCode == ResultCodes.OB_SUCCESS.errorCode) { @@ -311,18 +312,33 @@ public List executeWithResult() throws Exception { return results; } - public Map>>>> partitionPrepare() - throws Exception { - List operations = getSingleOperations(); - // map: >>> - Map>>>> lsOperationsMap = - new HashMap(); + private Object[] calculateRowKey(ObTableSingleOp operation) { + List rowKeyObject = operation.getRowkeyObjs(); + int rowKeySize = rowKeyObject.size(); + Object[] rowKey = new Object[rowKeySize]; + for (int j = 0; j < rowKeySize; j++) { + rowKey[j] = rowKeyObject.get(j).getValue(); + } + return rowKey; + } + + private List extractOperations(Map>>> tabletOperationsMap) { + List operations = new ArrayList<>(); + for (ObPair>> pair : tabletOperationsMap.values()) { + for (ObPair operationWithIndex : pair.getRight()) { + operations.add(operationWithIndex.getRight()); + } + } + return operations; + } + + public Map>>>> prepareOperations(List operations) throws Exception { + Map>>>> lsOperationsMap = new HashMap<>(); - // In ODP mode, client send the request to ODP directly without route if (obTableClient.isOdpMode()) { Map>>> tabletOperationsMap = new HashMap<>(); ObPair>> obTableOperations = - new ObPair(new ObTableParam(obTableClient.getOdpTable()), + new ObPair<>(new ObTableParam(obTableClient.getOdpTable()), new ArrayList>()); for (int i = 0; i < operations.size(); i++) { ObTableSingleOp operation = operations.get(i); @@ -335,12 +351,7 @@ public Map rowkeyObjs = operation.getRowkeyObjs(); - int rowKeySize = rowkeyObjs.size(); - Object[] rowKey = new Object[rowKeySize]; - for (int j = 0; j < rowKeySize; j++) { - rowKey[j] = rowkeyObjs.get(j).getValue(); - } + Object[] rowKey = calculateRowKey(operation); String real_tableName = tableName; if (this.entityType == ObTableEntityType.HKV && obTableClient.isTableGroupName(tableName)) { @@ -351,27 +362,24 @@ public Map>>> tabletOperations - = lsOperationsMap.get(lsId); + = lsOperationsMap.computeIfAbsent(lsId, k -> new HashMap<>()); // if ls id not exists - if (tabletOperations == null) { - tabletOperations = new HashMap<>(); - lsOperationsMap.put(lsId, tabletOperations); - } ObPair>> singleOperations = - tabletOperations.get(tableObPair.getLeft()); + tabletOperations.computeIfAbsent(tableObPair.getLeft(), k -> new ObPair<>(tableObPair.getRight(), new ArrayList<>())); // if tablet id not exists - if (singleOperations == null) { - singleOperations = new ObPair<>(tableObPair.getRight(), new ArrayList<>()); - tabletOperations.put(tableObPair.getLeft(), singleOperations); - } - - singleOperations.getRight().add(new ObPair(i, operation)); + singleOperations.getRight().add(new ObPair<>(i, operation)); } return lsOperationsMap; } + public Map>>>> partitionPrepare() + throws Exception { + List operations = getSingleOperations(); + return prepareOperations(operations); + } + /* * Partition execute. */ @@ -380,7 +388,7 @@ public void partitionExecute(ObTableSingleOpResult[] results, throws Exception { long lsId = lsOperation.getKey(); Map>>> tabletOperationsMap = lsOperation.getValue(); - if (tabletOperationsMap.size() == 0) { + if (tabletOperationsMap.isEmpty()) { logger.warn("the size of tablet operations in ls operation is zero"); throw new ObTableUnexpectedException("the size of tablet operations in ls operation is zero"); } @@ -466,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(); @@ -515,6 +526,11 @@ public void partitionExecute(ObTableSingleOpResult[] results, logger.warn("tablename:{} ls id:{} batch ops retry while meet ObTableMasterChangeException, errorCode: {} , retry times {}", tableName, lsId, ((ObTableException) ex).getErrorCode(), tryTimes, ex); + if (ex instanceof ObTableNeedFetchAllException) { + obTableClient.getOrRefreshTableEntry(tableName, needRefreshTableEntry, + obTableClient.isTableEntryRefreshIntervalWait(), true); + throw ex; + } } else { obTableClient.calculateContinuousFailure(tableName, ex.getMessage()); throw ex; @@ -530,7 +546,7 @@ public void partitionExecute(ObTableSingleOpResult[] results, long endExecute = System.currentTimeMillis(); if (subLSOpResult == null) { - RUNTIME.error("tablename:{} ls id:{} check batch operation result error: client get unexpected NULL result", + RUNTIME.error("table name:{} ls id:{} check batch operation result error: client get unexpected NULL result", tableName, lsId); throw new ObTableUnexpectedException("check batch operation result error: client get unexpected NULL result"); } @@ -600,11 +616,54 @@ public void partitionExecute(ObTableSingleOpResult[] results, obTableClient.getslowQueryMonitorThreshold()); } + private boolean shouldRetry(Throwable throwable) { + return throwable instanceof ObTableNeedFetchAllException; + } + + private void executeWithRetries( + ObTableSingleOpResult[] results, + Map.Entry>>>> entry, + int maxRetries) throws Exception { + + int retryCount = 0; + boolean success = false; + + Map>>>> currentPartitions = new HashMap<>(); + currentPartitions.put(entry.getKey(), entry.getValue()); + + while (retryCount < maxRetries && !success) { + boolean allPartitionsSuccess = true; + + for (Map.Entry>>>> currentEntry : currentPartitions.entrySet()) { + try { + partitionExecute(results, currentEntry); + } catch (Exception e) { + if (shouldRetry(e)) { + retryCount++; + List failedOperations = extractOperations(currentEntry.getValue()); + currentPartitions = prepareOperations(failedOperations); + allPartitionsSuccess = false; + break; + } else { + throw e; + } + } + } + + if (allPartitionsSuccess) { + success = true; + } + } + + if (!success) { + throw new ObTableUnexpectedException("Failed to execute operation after retrying " + maxRetries + " times."); + } + } + /* * Execute internal. */ public ObTableSingleOpResult[] executeInternal() throws Exception { - if (tableName == null || tableName.isEmpty()) { throw new IllegalArgumentException("table name is null"); } @@ -618,6 +677,8 @@ public ObTableSingleOpResult[] executeInternal() throws Exception { Map>>>> lsOperations = partitionPrepare(); long getTableTime = System.currentTimeMillis(); final Map context = ThreadLocalMap.getContextMap(); + final int maxRetries = obTableClient.getRuntimeRetryTimes(); + if (executorService != null && !executorService.isShutdown() && lsOperations.size() > 1) { // execute sub-batch operation in parallel final ConcurrentTaskExecutor executor = new ConcurrentTaskExecutor(executorService, @@ -633,7 +694,7 @@ public ObTableSingleOpResult[] executeInternal() throws Exception { public void doTask() { try { ThreadLocalMap.transmitContextMap(context); - partitionExecute(finalObTableOperationResults, entry); + executeWithRetries(finalObTableOperationResults, entry, maxRetries); } catch (Exception e) { logger.error(LCD.convert("01-00026"), e); executor.collectExceptions(e); @@ -654,7 +715,7 @@ public void doTask() { "Batch Concurrent Execute interrupted", e); } - if (executor.getThrowableList().size() > 0) { + if (!executor.getThrowableList().isEmpty()) { throw new ObTableUnexpectedException("Batch Concurrent Execute Error", executor.getThrowableList().get(0)); } @@ -667,7 +728,7 @@ public void doTask() { executor.stop(); } - if (executor.getThrowableList().size() > 0) { + if (!executor.getThrowableList().isEmpty()) { throw new ObTableUnexpectedException("Batch Concurrent Execute Error", executor .getThrowableList().get(0)); } @@ -682,7 +743,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); } } 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 ecf435d0..46cf35a8 100644 --- a/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientQueryImpl.java +++ b/src/main/java/com/alipay/oceanbase/rpc/table/ObTableClientQueryImpl.java @@ -32,6 +32,7 @@ import com.alipay.oceanbase.rpc.table.api.TableQuery; import com.alipay.oceanbase.rpc.util.MonitorUtil; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -240,52 +241,57 @@ ObTableClientQueryAsyncStreamResult execute() throws Exception { }); } - /* - * Init partition tables involved in this query - */ - public void initPartitions() throws Exception { + public Map> initPartitions(ObTableQuery tableQuery, String tableName) throws Exception { + Map> partitionObTables = new HashMap<>(); String indexName = tableQuery.getIndexName(); + String indexTableName = null; + if (!this.obTableClient.isOdpMode()) { - indexTableName = obTableClient.getIndexTableName(tableName, indexName, - tableQuery.getScanRangeColumns(), false); + indexTableName = obTableClient.getIndexTableName(tableName, indexName, tableQuery.getScanRangeColumns(), false); } - for (ObNewRange rang : this.tableQuery.getKeyRanges()) { - ObRowKey startKey = rang.getStartKey(); + for (ObNewRange range : tableQuery.getKeyRanges()) { + ObRowKey startKey = range.getStartKey(); int startKeySize = startKey.getObjs().size(); - ObRowKey endKey = rang.getEndKey(); + ObRowKey endKey = range.getEndKey(); int endKeySize = endKey.getObjs().size(); Object[] start = new Object[startKeySize]; Object[] end = new Object[endKeySize]; + for (int i = 0; i < startKeySize; i++) { - if (startKey.getObj(i).isMinObj() || startKey.getObj(i).isMaxObj()) { - start[i] = startKey.getObj(i); - } else { - start[i] = startKey.getObj(i).getValue(); - } + start[i] = startKey.getObj(i).isMinObj() || startKey.getObj(i).isMaxObj() ? + startKey.getObj(i) : startKey.getObj(i).getValue(); } + for (int i = 0; i < endKeySize; i++) { - if (endKey.getObj(i).isMinObj() || endKey.getObj(i).isMaxObj()) { - end[i] = endKey.getObj(i); - } else { - end[i] = endKey.getObj(i).getValue(); - } + end[i] = endKey.getObj(i).isMinObj() || endKey.getObj(i).isMaxObj() ? + endKey.getObj(i) : endKey.getObj(i).getValue(); } - ObBorderFlag borderFlag = rang.getBorderFlag(); - // pairs -> List> + + ObBorderFlag borderFlag = range.getBorderFlag(); List> pairs = this.obTableClient.getTables(indexTableName, tableQuery, start, borderFlag.isInclusiveStart(), end, borderFlag.isInclusiveEnd(), false, false); - if (this.tableQuery.getScanOrder() == ObScanOrder.Reverse) { + + if (tableQuery.getScanOrder() == ObScanOrder.Reverse) { for (int i = pairs.size() - 1; i >= 0; i--) { - this.partitionObTables.put(pairs.get(i).getLeft(), pairs.get(i)); + partitionObTables.put(pairs.get(i).getLeft(), pairs.get(i)); } } else { for (ObPair pair : pairs) { - this.partitionObTables.put(pair.getLeft(), pair); + partitionObTables.put(pair.getLeft(), pair); } } } + + return partitionObTables; + } + + /* + * Init partition tables involved in this query + */ + public void initPartitions() throws Exception { + this.partitionObTables = initPartitions(tableQuery, tableName); } /* diff --git a/src/main/java/com/alipay/oceanbase/rpc/util/ByteUtil.java b/src/main/java/com/alipay/oceanbase/rpc/util/ByteUtil.java new file mode 100644 index 00000000..4a324192 --- /dev/null +++ b/src/main/java/com/alipay/oceanbase/rpc/util/ByteUtil.java @@ -0,0 +1,132 @@ +/*- + * #%L + * com.oceanbase:obkv-table-client + * %% + * Copyright (C) 2021 - 2023 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.util; + +public class ByteUtil { + /** + * Compares two byte arrays lexicographically. + * + * This method interprets each byte as an unsigned value between 0 and 255 + * and performs a comparison on the arrays byte-by-byte. If two corresponding + * bytes differ, it returns the difference between the first non-equal byte + * values (treated as unsigned). If one array is a prefix of the other, + * the shorter array is considered to be less than the longer one. + * + * @param array1 the first byte array to be compared + * @param array2 the second byte array to be compared + * @return a negative integer if array1 comes before array2, + * a positive integer if array1 comes after array2, + * or zero if they are equal + */ + static public int compareByteArrays(byte[] array1, byte[] array2) { + for (int i = 0; i < Math.min(array1.length, array2.length); i++) { + int a = (array1[i] & 0xFF); + int b = (array2[i] & 0xFF); + if (a != b) { + return a - b; + } + } + return array1.length - array2.length; + } + + /** + * Increments a byte array treated as an unsigned integer by one. + * + * This method treats the input byte array as a non-negative big-endian + * integer, and increments its value by 1. The most significant byte is at + * the beginning of the array. If the increment operation causes an overflow, + * the resulting array will be extended to accommodate the carry. + * + * @param input the byte array representing a non-negative integer + * @return a new byte array representing the incremented value + */ + static public byte[] incrementByteArray(byte[] input) { + if (input == null || input.length == 0) { + return new byte[] { 1 }; + } + byte[] result = input.clone(); + for (int i = result.length - 1; i >= 0; i--) { + result[i] += 1; + + if ((result[i] & 0xFF) != 0) { + return result; + } + result[i] = 0; + } + + byte[] extendedResult = new byte[result.length + 1]; + extendedResult[0] = 1; + return extendedResult; + } + + /** + * Decrements a byte array treated as an unsigned integer by one. + * + * This method treats the input byte array as a non-negative big-endian + * integer, and decrements its value by 1. If the entire array represents + * zero, the function should handle it appropriately, typically returning + * an array representing zero. + * + * @param input the byte array representing a non-negative integer + * @return a new byte array representing the decremented value + * @throws IllegalArgumentException if the input represents zero + */ + static public byte[] decrementByteArray(byte[] input) { + if (input == null || input.length == 0 || isZero(input)) { + throw new IllegalArgumentException("Input array must represent a positive integer."); + } + + byte[] result = input.clone(); + + // Traverse the array from the least significant byte to the most + for (int i = result.length - 1; i >= 0; i--) { + if ((result[i] & 0xFF) > 0) { + result[i] -= 1; + return result; + } + + // If the byte is zero, we need to borrow from the next + result[i] = (byte) 0xFF; // Set current byte to 255 after borrow + } + + // Handle cases where we subtract from a number like 1000 -> 999 + if (result[0] == 0) { + byte[] shortenedResult = new byte[result.length - 1]; + System.arraycopy(result, 1, shortenedResult, 0, shortenedResult.length); + return shortenedResult; + } + + return result; + } + + /** + * Helper method to check if a byte array represents zero. + * + * @param array the byte array to check + * @return true if the byte array is zero, false otherwise + */ + static private boolean isZero(byte[] array) { + for (byte b : array) { + if (b != 0) { + return false; + } + } + return true; + } + +} \ No newline at end of file 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)); + } + } +}