diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterJobV2Builder.java b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterJobV2Builder.java index fecad154bd0c4a..926e11fb27e6d1 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterJobV2Builder.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterJobV2Builder.java @@ -84,8 +84,8 @@ public AlterJobV2 build() throws UserException { properties.put(LakeTablet.PROPERTY_KEY_PARTITION_ID, Long.toString(physicalPartitionId)); properties.put(LakeTablet.PROPERTY_KEY_INDEX_ID, Long.toString(shadowIndexId)); List shadowTabletIds = - createShards(originTablets.size(), table.getPartitionFilePathInfo(partitionId), - table.getPartitionFileCacheInfo(partitionId), shardGroupId, + createShards(originTablets.size(), table.getPartitionFilePathInfo(physicalPartitionId), + table.getPartitionFileCacheInfo(physicalPartitionId), shardGroupId, originTabletIds, properties, warehouseId); Preconditions.checkState(originTablets.size() == shadowTabletIds.size()); diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterMetaJobBase.java b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterMetaJobBase.java index 5b012f1f01b2fa..553f639deca36a 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterMetaJobBase.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterMetaJobBase.java @@ -385,13 +385,13 @@ public void updateIndexTabletMeta(Database db, OlapTable table, PhysicalPartitio void updateNextVersion(@NotNull LakeTable table) { for (long partitionId : physicalPartitionIndexMap.rowKeySet()) { - PhysicalPartition partition = table.getPhysicalPartition(partitionId); - long commitVersion = commitVersionMap.get(partitionId); - Preconditions.checkState(partition.getNextVersion() == commitVersion, - "partitionNextVersion=" + partition.getNextVersion() + " commitVersion=" + commitVersion); - partition.setNextVersion(commitVersion + 1); + PhysicalPartition physicalPartition = table.getPhysicalPartition(partitionId); + long commitVersion = commitVersionMap.get(physicalPartition.getId()); + Preconditions.checkState(physicalPartition.getNextVersion() == commitVersion, + "partitionNextVersion=" + physicalPartition.getNextVersion() + " commitVersion=" + commitVersion); + physicalPartition.setNextVersion(commitVersion + 1); LOG.info("LakeTableAlterMetaJob id: {} update next version of partition: {}, commitVersion: {}", - jobId, partition.getId(), commitVersion); + jobId, physicalPartition.getId(), commitVersion); } } diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableSchemaChangeJob.java b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableSchemaChangeJob.java index 72e11235cf4e2a..9356f3a6a09e7e 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableSchemaChangeJob.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableSchemaChangeJob.java @@ -344,12 +344,13 @@ protected void runPendingJob() throws AlterCancelException { countDownLatch = new MarkedCountDownLatch<>((int) numTablets); createReplicaLatch = countDownLatch; long baseIndexId = table.getBaseIndexId(); - for (long partitionId : physicalPartitionIndexMap.rowKeySet()) { - PhysicalPartition partition = table.getPhysicalPartition(partitionId); - Preconditions.checkState(partition != null); - TStorageMedium storageMedium = table.getPartitionInfo().getDataProperty(partitionId).getStorageMedium(); + for (long physicalPartitionId : physicalPartitionIndexMap.rowKeySet()) { + PhysicalPartition physicalPartition = table.getPhysicalPartition(physicalPartitionId); + Preconditions.checkState(physicalPartition != null); + TStorageMedium storageMedium = table.getPartitionInfo() + .getDataProperty(physicalPartition.getParentId()).getStorageMedium(); - Map shadowIndexMap = physicalPartitionIndexMap.row(partitionId); + Map shadowIndexMap = physicalPartitionIndexMap.row(physicalPartitionId); for (Map.Entry entry : shadowIndexMap.entrySet()) { long shadowIdxId = entry.getKey(); MaterializedIndex shadowIdx = entry.getValue(); @@ -387,7 +388,7 @@ protected void runPendingJob() throws AlterCancelException { .setNodeId(computeNode.getId()) .setDbId(dbId) .setTableId(tableId) - .setPartitionId(partitionId) + .setPartitionId(physicalPartitionId) .setIndexId(shadowIdxId) .setTabletId(shadowTabletId) .setVersion(Partition.PARTITION_INIT_VERSION) diff --git a/fe/fe-core/src/main/java/com/starrocks/backup/RestoreJob.java b/fe/fe-core/src/main/java/com/starrocks/backup/RestoreJob.java index 1bac29250e30ea..56e51d5d919a99 100644 --- a/fe/fe-core/src/main/java/com/starrocks/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/com/starrocks/backup/RestoreJob.java @@ -198,7 +198,7 @@ public enum RestoreJobState { private AgentBatchTask batchTask; boolean enableColocateRestore = Config.enable_colocate_restore; - + public RestoreJob() { super(JobType.RESTORE); } @@ -424,19 +424,20 @@ private void checkIfNeedCancel() { locker.lockDatabase(db.getId(), LockType.READ); try { for (IdChain idChain : fileMapping.getMapping().keySet()) { - OlapTable tbl = (OlapTable) globalStateMgr.getLocalMetastore() - .getTable(db.getId(), idChain.getTblId()); + OlapTable tbl = (OlapTable) globalStateMgr.getLocalMetastore().getTable(db.getId(), idChain.getTblId()); if (tbl == null) { status = new Status(ErrCode.NOT_FOUND, "table " + idChain.getTblId() + " has been dropped"); return; } - PhysicalPartition part = tbl.getPhysicalPartition(idChain.getPartId()); - if (part == null) { + + PhysicalPartition physicalPartition = tbl.getPhysicalPartition(idChain.getPartId()); + + if (physicalPartition == null) { status = new Status(ErrCode.NOT_FOUND, "partition " + idChain.getPartId() + " has been dropped"); return; } - MaterializedIndex index = part.getIndex(idChain.getIdxId()); + MaterializedIndex index = physicalPartition.getIndex(idChain.getIdxId()); if (index == null) { status = new Status(ErrCode.NOT_FOUND, "index " + idChain.getIdxId() + " has been dropped"); return; @@ -497,8 +498,8 @@ private void checkAndPrepareMeta() { if (!tbl.isSupportBackupRestore()) { status = new Status(ErrCode.UNSUPPORTED, - "Table: " + tbl.getName() + - " can not support backup restore, type: {}" + tbl.getType()); + "Table: " + tbl.getName() + + " can not support backup restore, type: {}" + tbl.getType()); return; } @@ -532,11 +533,11 @@ private void checkAndPrepareMeta() { Table remoteTbl = backupMeta.getTable(tblInfo.name); Preconditions.checkNotNull(remoteTbl); Table localTbl = globalStateMgr.getLocalMetastore() - .getTable(db.getFullName(), jobInfo.getAliasByOriginNameIfSet(tblInfo.name)); + .getTable(db.getFullName(), jobInfo.getAliasByOriginNameIfSet(tblInfo.name)); if (localTbl != null && remoteTbl.isOlapView() && !localTbl.isOlapView()) { status = new Status(ErrCode.BAD_REPLACE, - "Table: " + localTbl.getName() + " has existed and it is not a View"); + "Table: " + localTbl.getName() + " has existed and it is not a View"); return; } @@ -569,8 +570,8 @@ private void checkAndPrepareMeta() { // table already exist, check schema if (!localTbl.isSupportBackupRestore()) { status = new Status(ErrCode.UNSUPPORTED, - "Table: " + localTbl.getName() + - " can not support backup restore, type: {}" + localTbl.getType()); + "Table: " + localTbl.getName() + + " can not support backup restore, type: {}" + localTbl.getType()); return; } OlapTable localOlapTbl = (OlapTable) localTbl; @@ -735,7 +736,7 @@ private void checkAndPrepareMeta() { ColocateTableIndex.GroupId groupId = colocateTableIndex.getGroup(remoteOlapTbl.getId()); List> backendsPerBucketSeq = colocateTableIndex.getBackendsPerBucketSeq(groupId); ColocatePersistInfo colocatePersistInfo = ColocatePersistInfo - .createForAddTable(groupId, remoteOlapTbl.getId(), backendsPerBucketSeq); + .createForAddTable(groupId, remoteOlapTbl.getId(), backendsPerBucketSeq); colocatePersistInfos.add(colocatePersistInfo); } @@ -755,7 +756,7 @@ private void checkAndPrepareMeta() { // generate create replica tasks for all restored partitions for (Pair entry : restoredPartitions) { OlapTable localTbl = (OlapTable) globalStateMgr.getLocalMetastore() - .getTable(db.getFullName(), entry.first); + .getTable(db.getFullName(), entry.first); Preconditions.checkNotNull(localTbl, localTbl.getName()); Partition restorePart = entry.second; OlapTable remoteTbl = (OlapTable) backupMeta.getTable(entry.first); @@ -908,17 +909,17 @@ protected void prepareAndSendSnapshotTasks(Database db) { locker.lockDatabase(db.getId(), LockType.READ); try { for (IdChain idChain : fileMapping.getMapping().keySet()) { - OlapTable tbl = (OlapTable) globalStateMgr.getLocalMetastore() - .getTable(db.getId(), idChain.getTblId()); - PhysicalPartition part = tbl.getPhysicalPartition(idChain.getPartId()); - MaterializedIndex index = part.getIndex(idChain.getIdxId()); + OlapTable tbl = (OlapTable) globalStateMgr.getLocalMetastore().getTable(db.getId(), idChain.getTblId()); + + PhysicalPartition physicalPartition = tbl.getPhysicalPartition(idChain.getPartId()); + MaterializedIndex index = physicalPartition.getIndex(idChain.getIdxId()); LocalTablet tablet = (LocalTablet) index.getTablet(idChain.getTabletId()); Replica replica = tablet.getReplicaById(idChain.getReplicaId()); long signature = globalStateMgr.getNextId(); SnapshotTask task = new SnapshotTask(null, replica.getBackendId(), signature, jobId, db.getId(), - tbl.getId(), part.getId(), index.getId(), tablet.getId(), - part.getVisibleVersion(), + tbl.getId(), physicalPartition.getId(), index.getId(), tablet.getId(), + physicalPartition.getVisibleVersion(), tbl.getSchemaHashByIndexId(index.getId()), timeoutMs, true /* is restore task*/); batchTask.addTask(task); @@ -1098,7 +1099,7 @@ protected void genFileMappingWithPartition(OlapTable localTbl, Partition localPa for (Replica localReplica : localTablet.getImmutableReplicas()) { IdChain src = new IdChain(remoteTblId, backupPartInfo.id, backupIdxInfo.id, backupTabletInfo.id, -1L /* no replica id */); - IdChain dest = new IdChain(localTbl.getId(), localPartition.getId(), + IdChain dest = new IdChain(localTbl.getId(), localPartition.getDefaultPhysicalPartition().getId(), localIdx.getId(), localTablet.getId(), localReplica.getId()); fileMapping.putMapping(dest, src, overwrite); LOG.debug("tablet mapping: {} to {} file mapping: {} to {}", @@ -1142,7 +1143,7 @@ private void replayCheckAndPrepareMeta() { // replay set all existing tables's state to RESTORE for (BackupTableInfo tblInfo : jobInfo.tables.values()) { Table tbl = globalStateMgr.getLocalMetastore() - .getTable(db.getFullName(), jobInfo.getAliasByOriginNameIfSet(tblInfo.name)); + .getTable(db.getFullName(), jobInfo.getAliasByOriginNameIfSet(tblInfo.name)); if (tbl == null || tbl.isOlapView()) { continue; } @@ -1174,7 +1175,7 @@ private void replayCheckAndPrepareMeta() { protected void addRestoredPartitions(Database db, boolean modify) { for (Pair entry : restoredPartitions) { OlapTable localTbl = (OlapTable) globalStateMgr.getLocalMetastore() - .getTable(db.getFullName(), entry.first); + .getTable(db.getFullName(), entry.first); Partition restorePart = entry.second; OlapTable remoteTbl = (OlapTable) backupMeta.getTable(entry.first); RangePartitionInfo localPartitionInfo = (RangePartitionInfo) localTbl.getPartitionInfo(); @@ -1374,6 +1375,7 @@ protected void prepareDownloadTasks(List beSnapshotInfos, Database IdChain catalogIds = new IdChain(tbl.getId(), part.getId(), idx.getId(), info.getTabletId(), replica.getId()); + IdChain repoIds = fileMapping.get(catalogIds); if (repoIds == null) { status = new Status(ErrCode.NOT_FOUND, @@ -1527,7 +1529,7 @@ private Status allTabletCommitted(boolean isReplay) { try { for (BackupTableInfo tblInfo : jobInfo.tables.values()) { Table tbl = globalStateMgr.getLocalMetastore() - .getTable(db.getFullName(), jobInfo.getAliasByOriginNameIfSet(tblInfo.name)); + .getTable(db.getFullName(), jobInfo.getAliasByOriginNameIfSet(tblInfo.name)); // skip to restore table ids if (skipRestoreRemoteTableIds.contains(tblInfo.id)) { continue; @@ -1720,7 +1722,7 @@ public void cancelInternal(boolean isReplay) { // remove restored partitions for (Pair entry : restoredPartitions) { OlapTable restoreTbl = (OlapTable) globalStateMgr.getLocalMetastore() - .getTable(db.getFullName(), entry.first); + .getTable(db.getFullName(), entry.first); if (restoreTbl == null) { continue; } @@ -1744,7 +1746,7 @@ public void cancelInternal(boolean isReplay) { for (Table restoreTbl : restoredTbls) { if (restoreTbl instanceof OlapTable && restoreTbl.getId() == colocatePersistInfo.getTableId()) { globalStateMgr.getColocateTableIndex() - .removeTable(restoreTbl.getId(), (OlapTable) restoreTbl, isReplay); + .removeTable(restoreTbl.getId(), (OlapTable) restoreTbl, isReplay); } } } @@ -1773,7 +1775,7 @@ private void setTableStateToNormal(Database db) { continue; } Table tbl = globalStateMgr.getLocalMetastore() - .getTable(db.getFullName(), jobInfo.getAliasByOriginNameIfSet(tblInfo.name)); + .getTable(db.getFullName(), jobInfo.getAliasByOriginNameIfSet(tblInfo.name)); if (tbl == null) { continue; } diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/ExternalOlapTable.java b/fe/fe-core/src/main/java/com/starrocks/catalog/ExternalOlapTable.java index d8ed84afceb427..07371163487611 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/ExternalOlapTable.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/ExternalOlapTable.java @@ -465,7 +465,6 @@ private void updateMetaInternal(String dbName, TTableMeta meta, List { if (physicalPartition.getId() != newPartitionId) { - physicalPartition.setIdForRestore(globalStateMgr.getNextId()); + physicalPartition.setIdForRestore(GlobalStateMgr.getCurrentState().getNextId()); physicalPartition.setParentId(newPartitionId); partition.addSubPartition(physicalPartition); } @@ -1357,6 +1357,7 @@ public void addPartition(Partition partition) { public void addPhysicalPartition(PhysicalPartition physicalPartition) { physicalPartitionIdToPartitionId.put(physicalPartition.getId(), physicalPartition.getParentId()); + physicalPartitionNameToPartitionId.put(physicalPartition.getName(), physicalPartition.getParentId()); } // This is a private method. @@ -2250,9 +2251,11 @@ public List> getArbitraryTabletBucketsSeq() throws DdlException { List> backendsPerBucketSeq = Lists.newArrayList(); Optional optionalPartition = idToPartition.values().stream().findFirst(); if (optionalPartition.isPresent()) { - PhysicalPartition partition = optionalPartition.get().getDefaultPhysicalPartition(); + Partition partition = optionalPartition.get(); + PhysicalPartition physicalPartition = partition.getDefaultPhysicalPartition(); + short replicationNum = partitionInfo.getReplicationNum(partition.getId()); - MaterializedIndex baseIdx = partition.getBaseIndex(); + MaterializedIndex baseIdx = physicalPartition.getBaseIndex(); for (Long tabletId : baseIdx.getTabletIdsInOrder()) { LocalTablet tablet = (LocalTablet) baseIdx.getTablet(tabletId); List replicaBackendIds = tablet.getNormalReplicaBackendIds(); @@ -3412,10 +3415,10 @@ public FilePathInfo getDefaultFilePathInfo() { } @Nullable - public FilePathInfo getPartitionFilePathInfo(long partitionId) { + public FilePathInfo getPartitionFilePathInfo(long physicalPartitionId) { FilePathInfo pathInfo = getDefaultFilePathInfo(); if (pathInfo != null) { - return StarOSAgent.allocatePartitionFilePathInfo(pathInfo, partitionId); + return StarOSAgent.allocatePartitionFilePathInfo(pathInfo, physicalPartitionId); } return null; } diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/Partition.java b/fe/fe-core/src/main/java/com/starrocks/catalog/Partition.java index 5cd1c7391958ae..c9ceca675c510d 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/Partition.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/Partition.java @@ -94,13 +94,13 @@ public Partition(long id, String name, DistributionInfo distributionInfo) { private Partition() { } - public Partition(long id, String name, + public Partition(long id, long physicalPartitionId, String name, MaterializedIndex baseIndex, DistributionInfo distributionInfo) { - this(id, name, baseIndex, distributionInfo, 0); + this(id, physicalPartitionId, name, baseIndex, distributionInfo, 0); } - public Partition(long id, String name, + public Partition(long id, long physicalPartitionId, String name, MaterializedIndex baseIndex, DistributionInfo distributionInfo, long shardGroupId) { this.id = id; @@ -120,9 +120,9 @@ public Partition(long id, String name, this.distributionInfo = distributionInfo; this.shardGroupId = shardGroupId; - long physicalPartitionId = id; this.defaultPhysicalPartitionId = physicalPartitionId; - PhysicalPartition physicalPartition = new PhysicalPartition(physicalPartitionId, name, id, this.shardGroupId, baseIndex); + PhysicalPartition physicalPartition = new PhysicalPartition(physicalPartitionId, + generatePhysicalPartitionName(physicalPartitionId), id, shardGroupId, baseIndex); this.idToSubPartition.put(physicalPartitionId, physicalPartition); this.nameToSubPartition.put(name, physicalPartition); } @@ -189,6 +189,7 @@ public void addSubPartition(PhysicalPartition subPartition) { if (subPartition.getName() == null) { subPartition.setName(generatePhysicalPartitionName(subPartition.getId())); } + idToSubPartition.put(subPartition.getId(), subPartition); nameToSubPartition.put(subPartition.getName(), subPartition); } @@ -251,7 +252,7 @@ public long getReplicaCount() { @Override public int hashCode() { - return Objects.hashCode(id, visibleVersion, baseIndex, distributionInfo); + return Objects.hashCode(id, distributionInfo); } @Override @@ -264,11 +265,7 @@ public boolean equals(Object obj) { } Partition partition = (Partition) obj; - return (id == partition.id) - && (visibleVersion == partition.visibleVersion) - && (baseIndex.equals(partition.baseIndex) - && distributionInfo.equals(partition.distributionInfo)) - && Objects.equal(idToVisibleRollupIndex, partition.idToVisibleRollupIndex); + return (id == partition.id) && distributionInfo.equals(partition.distributionInfo); } @Override @@ -314,13 +311,25 @@ public void gsonPostProcess() throws IOException { PhysicalPartition physicalPartition = GsonUtils.GSON.fromJson(partitionJson, PhysicalPartition.class); physicalPartition.setParentId(id); - long nextId = GlobalStateMgr.getCurrentState().getNextId(); - defaultPhysicalPartitionId = nextId; - idToSubPartition.put(nextId, physicalPartition); - nameToSubPartition.put(name, physicalPartition); + long physicalPartitionId = id; + defaultPhysicalPartitionId = physicalPartitionId; + idToSubPartition.put(physicalPartitionId, physicalPartition); + nameToSubPartition.put(generatePhysicalPartitionName(physicalPartitionId), physicalPartition); } } + public void setDefaultPhysicalPartitionId(long defaultPhysicalPartitionId) { + PhysicalPartition physicalPartition = getDefaultPhysicalPartition(); + physicalPartition.setIdForRestore(defaultPhysicalPartitionId); + + idToSubPartition.remove(this.defaultPhysicalPartitionId); + nameToSubPartition.remove(generatePhysicalPartitionName(defaultPhysicalPartitionId)); + + this.defaultPhysicalPartitionId = defaultPhysicalPartitionId; + idToSubPartition.put(defaultPhysicalPartitionId, physicalPartition); + nameToSubPartition.put(generatePhysicalPartitionName(defaultPhysicalPartitionId), physicalPartition); + } + /**************************************PhysicalPartition **********************************************/ @SerializedName(value = "shardGroupId") diff --git a/fe/fe-core/src/main/java/com/starrocks/clone/DiskAndTabletLoadReBalancer.java b/fe/fe-core/src/main/java/com/starrocks/clone/DiskAndTabletLoadReBalancer.java index b6bca9384f4e54..ce581bf7f2a635 100644 --- a/fe/fe-core/src/main/java/com/starrocks/clone/DiskAndTabletLoadReBalancer.java +++ b/fe/fe-core/src/main/java/com/starrocks/clone/DiskAndTabletLoadReBalancer.java @@ -514,11 +514,12 @@ private List balanceClusterDisk(ClusterLoadStatistic clusterStat // in the second round, we will migrate the tablet that will break the tablet distribution balance. for (int round = 1; round <= 2; round++) { PARTITION: - for (Pair partitionMVId : hState.sortedPartitions) { - List hPartitionTablets = hState.partitionTablets.get(partitionMVId); - List lPartitionTablets = lState.partitionTablets.computeIfAbsent(partitionMVId, - pmId -> new LinkedList<>()); - int replicaTotalCnt = partitionReplicaCnt.getOrDefault(partitionMVId.first, 0); + for (Pair physicalPartitionAndMaterializedIndexId : hState.sortedPartitions) { + List hPartitionTablets = hState.partitionTablets.get(physicalPartitionAndMaterializedIndexId); + List lPartitionTablets = + lState.partitionTablets.computeIfAbsent(physicalPartitionAndMaterializedIndexId, + pmId -> new LinkedList<>()); + int replicaTotalCnt = partitionReplicaCnt.getOrDefault(physicalPartitionAndMaterializedIndexId.first, 0); int slotOfHighBE = hPartitionTablets.size() - (replicaTotalCnt / beStats.size()); int slotOfLowBE = ((replicaTotalCnt + beStats.size() - 1) / beStats.size()) - lPartitionTablets.size(); @@ -794,7 +795,6 @@ private void balanceBackendDisk(TStorageMedium medium, double avgUsedPercent, continue OUT; } - OlapTable olapTable = getOlapTableById(tabletMeta.getDbId(), tabletMeta.getTableId()); if (olapTable == null) { continue; @@ -960,7 +960,7 @@ private boolean isTabletExistsInBackends(Long tabletId, List backends) { } /** - * @return map : (partition, index) => tablets + * @return map : (physical partition id, index) => tablets */ private Map, Set> getPartitionTablets(long beId, TStorageMedium medium, long pathHash) { Map, Set> partitionTablets = Maps.newHashMap(); @@ -1807,7 +1807,9 @@ private Map getPartitionReplicaCnt() { * globalStateMgr.getLocalMetastore().getReplicationNumIncludeRecycleBin(olapTbl.getPartitionInfo(), partition.getId()); - partitionReplicaCnt.put(partition.getId(), replicaTotalCnt); + for (PhysicalPartition physicalPartition : partition.getSubPartitions()) { + partitionReplicaCnt.put(physicalPartition.getId(), replicaTotalCnt); + } } } } finally { @@ -1824,20 +1826,20 @@ private BackendBalanceState getBackendBalanceState(long backendId, Map partitionReplicaCnt, int backendCnt, boolean sortPartition) { - Map, Set> partitionTablets = getPartitionTablets(backendId, medium, -1L); + Map, Set> physicalPartitionTablets = getPartitionTablets(backendId, medium, -1L); Map, List> partitionTabletList = new HashMap<>(); - for (Map.Entry, Set> entry : partitionTablets.entrySet()) { + for (Map.Entry, Set> entry : physicalPartitionTablets.entrySet()) { partitionTabletList.put(entry.getKey(), new LinkedList<>(entry.getValue())); } - Map, Double> partitionAvgReplicaSize = getPartitionAvgReplicaSize(backendId, partitionTablets); - List> partitions = new ArrayList<>(partitionTablets.keySet()); + Map, Double> partitionAvgReplicaSize = getPartitionAvgReplicaSize(backendId, physicalPartitionTablets); + List> physicalPartitionAndMaterializedIndexId = new ArrayList<>(physicalPartitionTablets.keySet()); if (sortPartition) { - partitions.sort((p1, p2) -> { + physicalPartitionAndMaterializedIndexId.sort((p1, p2) -> { // skew is (tablet cnt on current BE - average tablet cnt on every BE) // sort partitions by skew in desc order, if skew is same, sort by avgReplicaSize in desc order. - int skew1 = partitionTablets.get(p1).size() + int skew1 = physicalPartitionTablets.get(p1).size() - partitionReplicaCnt.getOrDefault(p1.first, 0) / backendCnt; - int skew2 = partitionTablets.get(p2).size() + int skew2 = physicalPartitionTablets.get(p2).size() - partitionReplicaCnt.getOrDefault(p2.first, 0) / backendCnt; if (skew2 != skew1) { return skew2 - skew1; @@ -1864,7 +1866,7 @@ private BackendBalanceState getBackendBalanceState(long backendId, GlobalStateMgr.getCurrentState().getTabletInvertedIndex(), medium, partitionTabletList, - partitions); + physicalPartitionAndMaterializedIndexId); backendBalanceState.init(); return backendBalanceState; } @@ -2011,13 +2013,13 @@ public static class BackendBalanceState { TabletInvertedIndex tabletInvertedIndex, TStorageMedium medium, Map, List> partitionTablets, - List> partitions) { + List> physicalPartitionAndMaterializedIndexId) { this.backendId = backendId; this.statistic = statistic; this.tabletInvertedIndex = tabletInvertedIndex; this.medium = medium; this.partitionTablets = partitionTablets; - this.sortedPartitions = partitions; + this.sortedPartitions = physicalPartitionAndMaterializedIndexId; } void init() { diff --git a/fe/fe-core/src/main/java/com/starrocks/journal/JournalEntity.java b/fe/fe-core/src/main/java/com/starrocks/journal/JournalEntity.java index 4f5cfe2a66af98..57ed10f830d699 100644 --- a/fe/fe-core/src/main/java/com/starrocks/journal/JournalEntity.java +++ b/fe/fe-core/src/main/java/com/starrocks/journal/JournalEntity.java @@ -121,6 +121,7 @@ import com.starrocks.persist.RolePrivilegeCollectionInfo; import com.starrocks.persist.RoutineLoadOperation; import com.starrocks.persist.SetDefaultStorageVolumeLog; +import com.starrocks.persist.SetPhysicalPartitionIdLog; import com.starrocks.persist.SetReplicaStatusOperationLog; import com.starrocks.persist.SwapTableOperationLog; import com.starrocks.persist.TableAddOrDropColumnsInfo; @@ -775,6 +776,11 @@ public void readFields(DataInput in) throws IOException { data = DropWarehouseLog.read(in); break; } + + case OperationType.OP_SET_PHYSICAL_PARTITION_ID: { + data = GsonUtils.GSON.fromJson(Text.readString(in), SetPhysicalPartitionIdLog.class); + break; + } default: { if (Config.metadata_ignore_unknown_operation_type) { LOG.warn("UNKNOWN Operation Type {}", opCode); diff --git a/fe/fe-core/src/main/java/com/starrocks/lake/LakeTable.java b/fe/fe-core/src/main/java/com/starrocks/lake/LakeTable.java index b343bfdfb884e8..68fd356f5c0bb3 100644 --- a/fe/fe-core/src/main/java/com/starrocks/lake/LakeTable.java +++ b/fe/fe-core/src/main/java/com/starrocks/lake/LakeTable.java @@ -173,11 +173,11 @@ public Map getUniqueProperties() { @Override public Status createTabletsForRestore(int tabletNum, MaterializedIndex index, GlobalStateMgr globalStateMgr, int replicationNum, long version, int schemaHash, - long partitionId, long shardGroupId, Database db) { - FilePathInfo fsInfo = getPartitionFilePathInfo(partitionId); - FileCacheInfo cacheInfo = getPartitionFileCacheInfo(partitionId); + long physicalPartitionId, long shardGroupId, Database db) { + FilePathInfo fsInfo = getPartitionFilePathInfo(physicalPartitionId); + FileCacheInfo cacheInfo = getPartitionFileCacheInfo(physicalPartitionId); Map properties = new HashMap<>(); - properties.put(LakeTablet.PROPERTY_KEY_PARTITION_ID, Long.toString(partitionId)); + properties.put(LakeTablet.PROPERTY_KEY_PARTITION_ID, Long.toString(physicalPartitionId)); properties.put(LakeTablet.PROPERTY_KEY_INDEX_ID, Long.toString(index.getId())); List shardIds = null; try { diff --git a/fe/fe-core/src/main/java/com/starrocks/lake/LakeTableHelper.java b/fe/fe-core/src/main/java/com/starrocks/lake/LakeTableHelper.java index 4e5731b095c316..ffa7d861a08ed2 100644 --- a/fe/fe-core/src/main/java/com/starrocks/lake/LakeTableHelper.java +++ b/fe/fe-core/src/main/java/com/starrocks/lake/LakeTableHelper.java @@ -154,12 +154,13 @@ static boolean removePartitionDirectory(Partition partition, long warehouseId) t return ret; } - public static boolean isSharedPartitionDirectory(PhysicalPartition partition, long warehouseId) throws StarClientException { - ShardInfo shardInfo = getAssociatedShardInfo(partition, warehouseId).orElse(null); + public static boolean isSharedPartitionDirectory(PhysicalPartition physicalPartition, long warehouseId) + throws StarClientException { + ShardInfo shardInfo = getAssociatedShardInfo(physicalPartition, warehouseId).orElse(null); if (shardInfo == null) { return false; } - return isSharedDirectory(shardInfo.getFilePath().getFullPath(), partition.getId()); + return isSharedDirectory(shardInfo.getFilePath().getFullPath(), physicalPartition.getId()); } /** @@ -171,8 +172,8 @@ public static boolean isSharedPartitionDirectory(PhysicalPartition partition, lo * * @return true if the directory is a shared directory, false otherwise */ - public static boolean isSharedDirectory(String path, long partitionId) { - return !path.endsWith(String.format("/%d", partitionId)); + public static boolean isSharedDirectory(String path, long physicalPartitionId) { + return !path.endsWith(String.format("/%d", physicalPartitionId)); } /** diff --git a/fe/fe-core/src/main/java/com/starrocks/lake/StarOSAgent.java b/fe/fe-core/src/main/java/com/starrocks/lake/StarOSAgent.java index f3a062723d2881..10faeb1a3d0de7 100644 --- a/fe/fe-core/src/main/java/com/starrocks/lake/StarOSAgent.java +++ b/fe/fe-core/src/main/java/com/starrocks/lake/StarOSAgent.java @@ -782,8 +782,8 @@ public ShardInfo getShardInfo(long shardId, long workerGroupId) throws StarClien return shardInfos.get(0); } - public static FilePathInfo allocatePartitionFilePathInfo(FilePathInfo tableFilePathInfo, long partitionId) { - String allocPath = StarClient.allocateFilePath(tableFilePathInfo, Long.hashCode(partitionId)); - return tableFilePathInfo.toBuilder().setFullPath(String.format("%s/%d", allocPath, partitionId)).build(); + public static FilePathInfo allocatePartitionFilePathInfo(FilePathInfo tableFilePathInfo, long physicalPartitionId) { + String allocPath = StarClient.allocateFilePath(tableFilePathInfo, Long.hashCode(physicalPartitionId)); + return tableFilePathInfo.toBuilder().setFullPath(String.format("%s/%d", allocPath, physicalPartitionId)).build(); } } diff --git a/fe/fe-core/src/main/java/com/starrocks/leader/LeaderImpl.java b/fe/fe-core/src/main/java/com/starrocks/leader/LeaderImpl.java index 60e58dc5a8110f..9ad6098b309b0f 100644 --- a/fe/fe-core/src/main/java/com/starrocks/leader/LeaderImpl.java +++ b/fe/fe-core/src/main/java/com/starrocks/leader/LeaderImpl.java @@ -519,7 +519,7 @@ private void finishRealtimePush(AgentTask task, TFinishTaskRequest request) { } long tableId = pushTask.getTableId(); - long partitionId = pushTask.getPartitionId(); + long physicalPartitionId = pushTask.getPartitionId(); long pushIndexId = pushTask.getIndexId(); long pushTabletId = pushTask.getTabletId(); // push finish type: @@ -552,9 +552,9 @@ private void finishRealtimePush(AgentTask task, TFinishTaskRequest request) { throw new MetaNotFoundException("cannot find table[" + tableId + "] when push finished"); } - PhysicalPartition physicalPartition = olapTable.getPhysicalPartition(partitionId); + PhysicalPartition physicalPartition = olapTable.getPhysicalPartition(physicalPartitionId); if (physicalPartition == null) { - throw new MetaNotFoundException("cannot find partition[" + partitionId + "] when push finished"); + throw new MetaNotFoundException("cannot find partition[" + physicalPartitionId + "] when push finished"); } MaterializedIndex pushIndex = physicalPartition.getIndex(pushIndexId); @@ -592,7 +592,7 @@ private void finishRealtimePush(AgentTask task, TFinishTaskRequest request) { Replica replica = findRelatedReplica(olapTable, physicalPartition, backendId, tabletId, tabletMeta.getIndexId()); if (replica != null) { - olapDeleteJob.addFinishedReplica(partitionId, pushTabletId, replica); + olapDeleteJob.addFinishedReplica(physicalPartitionId, pushTabletId, replica); pushTask.countDownLatch(backendId, pushTabletId); } } diff --git a/fe/fe-core/src/main/java/com/starrocks/load/OlapDeleteJob.java b/fe/fe-core/src/main/java/com/starrocks/load/OlapDeleteJob.java index 0f2c51c3d191e4..00ff9b8638cd9a 100644 --- a/fe/fe-core/src/main/java/com/starrocks/load/OlapDeleteJob.java +++ b/fe/fe-core/src/main/java/com/starrocks/load/OlapDeleteJob.java @@ -304,10 +304,10 @@ public void checkAndUpdateQuorum() throws MetaNotFoundException { } for (TabletDeleteInfo tDeleteInfo : getTabletDeleteInfo()) { - Short replicaNum = partitionToReplicateNum.get(tDeleteInfo.getPartitionId()); + Short replicaNum = partitionToReplicateNum.get(tDeleteInfo.getPhysicalPartitionId()); if (replicaNum == null) { // should not happen - throw new MetaNotFoundException("Unknown partition " + tDeleteInfo.getPartitionId() + + throw new MetaNotFoundException("Unknown partition " + tDeleteInfo.getPhysicalPartitionId() + " when commit delete job"); } if (tDeleteInfo.getFinishedReplicas().size() == replicaNum) { @@ -339,8 +339,8 @@ public Set getPushTasks() { return pushTasks; } - public boolean addFinishedReplica(long partitionId, long tabletId, Replica replica) { - tabletDeleteInfoMap.putIfAbsent(tabletId, new TabletDeleteInfo(partitionId, tabletId)); + public boolean addFinishedReplica(long physicalPartitionId, long tabletId, Replica replica) { + tabletDeleteInfoMap.putIfAbsent(tabletId, new TabletDeleteInfo(physicalPartitionId, tabletId)); TabletDeleteInfo tDeleteInfo = tabletDeleteInfoMap.get(tabletId); return tDeleteInfo.addFinishedReplica(replica); } diff --git a/fe/fe-core/src/main/java/com/starrocks/load/TabletDeleteInfo.java b/fe/fe-core/src/main/java/com/starrocks/load/TabletDeleteInfo.java index 54224a99acda01..495fd2e9d8d78f 100644 --- a/fe/fe-core/src/main/java/com/starrocks/load/TabletDeleteInfo.java +++ b/fe/fe-core/src/main/java/com/starrocks/load/TabletDeleteInfo.java @@ -40,12 +40,12 @@ import java.util.Set; public class TabletDeleteInfo { - private final long partitionId; + private final long physicalPartitionId; private final long tabletId; private final Set finishedReplicas; public TabletDeleteInfo(long partitionId, long tabletId) { - this.partitionId = partitionId; + this.physicalPartitionId = partitionId; this.tabletId = tabletId; this.finishedReplicas = Sets.newConcurrentHashSet(); } @@ -54,8 +54,8 @@ public long getTabletId() { return tabletId; } - public long getPartitionId() { - return partitionId; + public long getPhysicalPartitionId() { + return physicalPartitionId; } public Set getFinishedReplicas() { diff --git a/fe/fe-core/src/main/java/com/starrocks/mv/MVRepairHandler.java b/fe/fe-core/src/main/java/com/starrocks/mv/MVRepairHandler.java index bea5ba430544bd..b1daf7d3a963af 100644 --- a/fe/fe-core/src/main/java/com/starrocks/mv/MVRepairHandler.java +++ b/fe/fe-core/src/main/java/com/starrocks/mv/MVRepairHandler.java @@ -101,7 +101,7 @@ default void handleMVRepair(TransactionState transactionState) { locker.lockTableWithIntensiveDbLock(db.getId(), table.getId(), LockType.READ); try { for (PartitionCommitInfo partitionCommitInfo : partitionCommitInfos.values()) { - long partitionId = partitionCommitInfo.getPartitionId(); + long partitionId = partitionCommitInfo.getPhysicalPartitionId(); Partition partition = olapTable.getPartition(partitionId); if (partition == null || olapTable.isTempPartition(partitionId)) { continue; diff --git a/fe/fe-core/src/main/java/com/starrocks/persist/EditLog.java b/fe/fe-core/src/main/java/com/starrocks/persist/EditLog.java index ac919f88d9a0c6..6fbfc0dd2a456b 100644 --- a/fe/fe-core/src/main/java/com/starrocks/persist/EditLog.java +++ b/fe/fe-core/src/main/java/com/starrocks/persist/EditLog.java @@ -1107,6 +1107,12 @@ public void loadJournal(GlobalStateMgr globalStateMgr, JournalEntity journal) warehouseMgr.replayAlterWarehouse(wh); break; } + + case OperationType.OP_SET_PHYSICAL_PARTITION_ID: { + SetPhysicalPartitionIdLog log = (SetPhysicalPartitionIdLog) journal.getData(); + GlobalStateMgr.getCurrentState().getLocalMetastore().replaySetPhysicalPartitionId(log); + break; + } default: { if (Config.metadata_ignore_unknown_operation_type) { LOG.warn("UNKNOWN Operation Type {}", opCode); @@ -1949,4 +1955,8 @@ public void logCancelDisableDisk(CancelDisableDiskInfo info) { public void logRecoverPartitionVersion(PartitionVersionRecoveryInfo info) { logEdit(OperationType.OP_RECOVER_PARTITION_VERSION, info); } + + public void logSetPhysicalPartitionId(SetPhysicalPartitionIdLog setPhysicalPartitionIdLog) { + logEdit(OperationType.OP_SET_PHYSICAL_PARTITION_ID, setPhysicalPartitionIdLog); + } } diff --git a/fe/fe-core/src/main/java/com/starrocks/persist/OperationType.java b/fe/fe-core/src/main/java/com/starrocks/persist/OperationType.java index a592628306b048..0ea98f579e364e 100644 --- a/fe/fe-core/src/main/java/com/starrocks/persist/OperationType.java +++ b/fe/fe-core/src/main/java/com/starrocks/persist/OperationType.java @@ -584,6 +584,9 @@ public class OperationType { @IgnorableOnReplayFailed public static final short OP_ADD_KEY = 13512; + @IgnorableOnReplayFailed + public static final short OP_SET_PHYSICAL_PARTITION_ID = 13513; + /** * NOTICE: OperationType cannot use a value exceeding 20000, please follow the above sequence number */ diff --git a/fe/fe-core/src/main/java/com/starrocks/persist/SetPhysicalPartitionIdLog.java b/fe/fe-core/src/main/java/com/starrocks/persist/SetPhysicalPartitionIdLog.java new file mode 100644 index 00000000000000..315d43d62031f9 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/persist/SetPhysicalPartitionIdLog.java @@ -0,0 +1,48 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.persist; + +import com.google.gson.annotations.SerializedName; +import com.starrocks.common.io.Text; +import com.starrocks.common.io.Writable; +import com.starrocks.lake.compaction.PartitionIdentifier; +import com.starrocks.persist.gson.GsonUtils; + +import java.io.DataOutput; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class SetPhysicalPartitionIdLog implements Writable { + @SerializedName(value = "phId") + private final Map phId; + + public SetPhysicalPartitionIdLog() { + this.phId = new HashMap<>(); + } + + public void addPhysicalPartitionId(Long dbId, Long tableId, Long partitionId, Long physicalPartitionId) { + PartitionIdentifier partitionIdentifier = new PartitionIdentifier(dbId, tableId, partitionId); + phId.put(partitionIdentifier, physicalPartitionId); + } + + public Map getPhId() { + return phId; + } + + @Override + public void write(DataOutput out) throws IOException { + Text.writeString(out, GsonUtils.GSON.toJson(this)); + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/MetaScanNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/MetaScanNode.java index 4d5746aa43bcef..481ae5a720fe1e 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/MetaScanNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/MetaScanNode.java @@ -20,6 +20,7 @@ import com.starrocks.catalog.LocalTablet; import com.starrocks.catalog.MaterializedIndex; import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; import com.starrocks.catalog.PhysicalPartition; import com.starrocks.catalog.Replica; import com.starrocks.catalog.Tablet; @@ -72,7 +73,8 @@ public void computeRangeLocations() { if (selectPartitionNames.isEmpty()) { partitions = olapTable.getPhysicalPartitions(); } else { - partitions = selectPartitionNames.stream().map(olapTable::getPartition).collect(Collectors.toList()); + partitions = selectPartitionNames.stream().map(olapTable::getPartition) + .map(Partition::getDefaultPhysicalPartition).collect(Collectors.toList()); } for (PhysicalPartition partition : partitions) { diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/OlapTableSink.java b/fe/fe-core/src/main/java/com/starrocks/planner/OlapTableSink.java index ccd70356c4b8b2..8fd93a9870f9f3 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/OlapTableSink.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/OlapTableSink.java @@ -761,13 +761,13 @@ public static TOlapTableLocationParam createLocation(OlapTable table, TOlapTable if (partitionParam.getPartitions() == null) { return locationParam; } - for (TOlapTablePartition partition : partitionParam.getPartitions()) { - PhysicalPartition physicalPartition = table.getPhysicalPartition(partition.getId()); + for (TOlapTablePartition tPhysicalPartition : partitionParam.getPartitions()) { + PhysicalPartition physicalPartition = table.getPhysicalPartition(tPhysicalPartition.getId()); int quorum = table.getPartitionInfo().getQuorumNum(physicalPartition.getParentId(), table.writeQuorum()); // `selectedBackedIds` keeps the selected backendIds for 1st index which will be used to choose the later index's // tablets' replica in colocate mv index optimization. List selectedBackedIds = Lists.newArrayList(); - LOG.debug("partition: {}, physical partition: {}", partition, physicalPartition); + LOG.debug("partition: {}, physical partition: {}", tPhysicalPartition, physicalPartition); for (MaterializedIndex index : physicalPartition.getMaterializedIndices(IndexExtState.ALL)) { for (int idx = 0; idx < index.getTablets().size(); ++idx) { Tablet tablet = index.getTablets().get(idx); diff --git a/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java b/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java index ff22460cdfa11f..da9a6f16e962f9 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java @@ -1274,6 +1274,8 @@ private void transferToLeader() { initDefaultWarehouse(); } + localMetastore.setPhysicalPartitionId(); + MetricRepo.init(); isReady.set(true); diff --git a/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java b/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java index 93cd984b3333f2..4d7bc434f2e1c4 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java @@ -116,6 +116,7 @@ import com.starrocks.common.util.UUIDUtil; import com.starrocks.common.util.Util; import com.starrocks.common.util.concurrent.CountingLatch; +import com.starrocks.common.util.concurrent.lock.AutoCloseableLock; import com.starrocks.common.util.concurrent.lock.LockType; import com.starrocks.common.util.concurrent.lock.Locker; import com.starrocks.connector.ConnectorMetadata; @@ -125,6 +126,7 @@ import com.starrocks.lake.LakeTable; import com.starrocks.lake.LakeTablet; import com.starrocks.lake.StorageInfo; +import com.starrocks.lake.compaction.PartitionIdentifier; import com.starrocks.load.pipe.PipeManager; import com.starrocks.memory.MemoryTrackable; import com.starrocks.mv.MVMetaVersionRepairer; @@ -162,6 +164,7 @@ import com.starrocks.persist.RecoverInfo; import com.starrocks.persist.ReplacePartitionOperationLog; import com.starrocks.persist.ReplicaPersistInfo; +import com.starrocks.persist.SetPhysicalPartitionIdLog; import com.starrocks.persist.SetReplicaStatusOperationLog; import com.starrocks.persist.TableInfo; import com.starrocks.persist.TruncateTableInfo; @@ -361,6 +364,47 @@ public void recreateTabletInvertIndex() { } // end for dbs } + public void setPhysicalPartitionId() { + SetPhysicalPartitionIdLog setPhysicalPartitionIdLog = new SetPhysicalPartitionIdLog(); + for (Database db : this.fullNameToDb.values()) { + for (Table table : db.getTables()) { + if (!table.isNativeTableOrMaterializedView()) { + continue; + } + + OlapTable olapTable = (OlapTable) table; + for (Partition partition : olapTable.getAllPartitions()) { + PhysicalPartition physicalPartition = partition.getDefaultPhysicalPartition(); + if (physicalPartition.getId() == partition.getId()) { + long physicalPartitionId = GlobalStateMgr.getCurrentState().getNextId(); + setPhysicalPartitionIdLog.addPhysicalPartitionId(db.getId(), olapTable.getId(), partition.getId(), + physicalPartitionId); + } + } + } // end for tables + } // end for dbs + + replaySetPhysicalPartitionId(setPhysicalPartitionIdLog); + GlobalStateMgr.getCurrentState().getEditLog().logSetPhysicalPartitionId(setPhysicalPartitionIdLog); + } + + public void replaySetPhysicalPartitionId(SetPhysicalPartitionIdLog setPhysicalPartitionIdLog) { + LocalMetastore localMetastore = GlobalStateMgr.getCurrentState().getLocalMetastore(); + for (Map.Entry phIdEntry : setPhysicalPartitionIdLog.getPhId().entrySet()) { + PartitionIdentifier partitionIdentifier = phIdEntry.getKey(); + OlapTable table = (OlapTable) localMetastore.getTable(partitionIdentifier.getDbId(), + partitionIdentifier.getTableId()); + + try (AutoCloseableLock ignore = + new AutoCloseableLock(new Locker(), partitionIdentifier.getDbId(), + Lists.newArrayList(partitionIdentifier.getTableId()), LockType.WRITE)) { + Partition partition = table.getPartition(partitionIdentifier.getPartitionId()); + partition.setDefaultPhysicalPartitionId(phIdEntry.getValue()); + table.addPhysicalPartition(partition.getDefaultPhysicalPartition()); + } + } + } + @Override public void createDb(String dbName, Map properties) throws DdlException, AlreadyExistsException { long id = 0L; @@ -1514,7 +1558,7 @@ public void replayRecoverPartition(RecoverInfo info) { } } - private PhysicalPartition createPhysicalPartition(String name, Database db, OlapTable olapTable, + private PhysicalPartition createPhysicalPartition(Database db, OlapTable olapTable, Partition partition, long warehouseId) throws DdlException { long partitionId = partition.getId(); DistributionInfo distributionInfo = olapTable.getDefaultDistributionInfo().copy(); @@ -1531,11 +1575,9 @@ private PhysicalPartition createPhysicalPartition(String name, Database db, Olap // so that the shards of this logical partition are more evenly distributed. long shardGroupId = partition.getDefaultPhysicalPartition().getShardGroupId(); - if (name == null) { - name = partition.generatePhysicalPartitionName(id); - } PhysicalPartition physicalPartition = new PhysicalPartition( - id, name, partition.getId(), shardGroupId, indexMap.get(olapTable.getBaseIndexId())); + id, partition.generatePhysicalPartitionName(id), + partition.getId(), shardGroupId, indexMap.get(olapTable.getBaseIndexId())); PartitionInfo partitionInfo = olapTable.getPartitionInfo(); short replicationNum = partitionInfo.getReplicationNum(partitionId); @@ -1571,67 +1613,62 @@ public void addSubPartitions(Database db, OlapTable table, Partition partition, int numSubPartition, long warehouseId) throws DdlException { try { table.setAutomaticBucketing(true); - addSubPartitions(db, table, partition, numSubPartition, null, warehouseId); - } finally { - table.setAutomaticBucketing(false); - } - } - private void addSubPartitions(Database db, OlapTable table, Partition partition, - int numSubPartition, String[] subPartitionNames, long warehouseId) throws DdlException { - OlapTable olapTable; - OlapTable copiedTable; + OlapTable olapTable; + OlapTable copiedTable; - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.READ); - try { - olapTable = checkTable(db, table.getId()); - - if (partition.getDistributionInfo().getType() != DistributionInfo.DistributionInfoType.RANDOM) { - throw new DdlException("Only support adding physical partition to random distributed table"); - } + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.READ); + try { + olapTable = checkTable(db, table.getId()); - copiedTable = getShadowCopyTable(olapTable); - } finally { - locker.unLockDatabase(db.getId(), LockType.READ); - } + if (partition.getDistributionInfo().getType() != DistributionInfo.DistributionInfoType.RANDOM) { + throw new DdlException("Only support adding physical partition to random distributed table"); + } - Preconditions.checkNotNull(olapTable); - Preconditions.checkNotNull(copiedTable); + copiedTable = getShadowCopyTable(olapTable); + } finally { + locker.unLockDatabase(db.getId(), LockType.READ); + } - List subPartitions = new ArrayList<>(); - // create physical partition - for (int i = 0; i < numSubPartition; i++) { - String name = subPartitionNames != null && subPartitionNames.length > i ? subPartitionNames[i] : null; - PhysicalPartition subPartition = createPhysicalPartition(name, db, copiedTable, partition, warehouseId); - subPartitions.add(subPartition); - } + Preconditions.checkNotNull(olapTable); + Preconditions.checkNotNull(copiedTable); - // build partitions - buildPartitions(db, copiedTable, subPartitions, warehouseId); + List subPartitions = new ArrayList<>(); + // create physical partition + for (int i = 0; i < numSubPartition; i++) { + PhysicalPartition subPartition = createPhysicalPartition(db, copiedTable, partition, warehouseId); + subPartitions.add(subPartition); + } - // check again - if (!locker.lockDatabaseAndCheckExist(db, LockType.WRITE)) { - throw new DdlException("db " + db.getFullName() - + "(" + db.getId() + ") has been dropped"); - } - try { - olapTable = checkTable(db, table.getId()); - // check if meta changed - checkIfMetaChange(olapTable, copiedTable, table.getName()); + // build partitions + buildPartitions(db, copiedTable, subPartitions, warehouseId); - for (PhysicalPartition subPartition : subPartitions) { - // add sub partition - partition.addSubPartition(subPartition); - olapTable.addPhysicalPartition(subPartition); + // check again + if (!locker.lockDatabaseAndCheckExist(db, LockType.WRITE)) { + throw new DdlException("db " + db.getFullName() + + "(" + db.getId() + ") has been dropped"); } + try { + olapTable = checkTable(db, table.getId()); + // check if meta changed + checkIfMetaChange(olapTable, copiedTable, table.getName()); - olapTable.setShardGroupChanged(true); + for (PhysicalPartition subPartition : subPartitions) { + // add sub partition + partition.addSubPartition(subPartition); + olapTable.addPhysicalPartition(subPartition); + } - // add partition log - addSubPartitionLog(db, olapTable, partition, subPartitions); + olapTable.setShardGroupChanged(true); + + // add partition log + addSubPartitionLog(db, olapTable, partition, subPartitions); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); + table.setAutomaticBucketing(false); } } @@ -1690,6 +1727,11 @@ Partition createPartition(Database db, OlapTable table, long partitionId, String indexMap.put(indexId, rollup); } + Partition logicalPartition = new Partition( + partitionId, + partitionName, + distributionInfo); + // create shard group long shardGroupId = 0; if (table.isCloudNativeTableOrMaterializedView()) { @@ -1697,14 +1739,10 @@ Partition createPartition(Database db, OlapTable table, long partitionId, String createShardGroup(db.getId(), table.getId(), partitionId); } - Partition logicalPartition = new Partition( - partitionId, - partitionName, - distributionInfo); - + long physicalPartitionId = GlobalStateMgr.getCurrentState().getNextId(); PhysicalPartition physicalPartition = new PhysicalPartition( - GlobalStateMgr.getCurrentState().getNextId(), - partitionName, + physicalPartitionId, + logicalPartition.generatePhysicalPartitionName(physicalPartitionId), partitionId, shardGroupId, indexMap.get(table.getBaseIndexId())); @@ -1725,11 +1763,11 @@ Partition createPartition(Database db, OlapTable table, long partitionId, String // create tablets TabletMeta tabletMeta = - new TabletMeta(db.getId(), table.getId(), partitionId, indexId, indexMeta.getSchemaHash(), + new TabletMeta(db.getId(), table.getId(), physicalPartitionId, indexId, indexMeta.getSchemaHash(), storageMedium, table.isCloudNativeTableOrMaterializedView()); if (table.isCloudNativeTableOrMaterializedView()) { - createLakeTablets(table, partitionId, shardGroupId, index, distributionInfo, + createLakeTablets(table, physicalPartitionId, shardGroupId, index, distributionInfo, tabletMeta, tabletIdSet, warehouseId); } else { createOlapTablets(table, index, Replica.ReplicaState.NORMAL, distributionInfo, @@ -1968,7 +2006,7 @@ public void replayCreateTable(CreateTableInfo info) { } } - private void createLakeTablets(OlapTable table, long partitionId, long shardGroupId, MaterializedIndex index, + private void createLakeTablets(OlapTable table, long physicalPartitionId, long shardGroupId, MaterializedIndex index, DistributionInfo distributionInfo, TabletMeta tabletMeta, Set tabletIdSet, long warehouseId) throws DdlException { @@ -1982,7 +2020,7 @@ private void createLakeTablets(OlapTable table, long partitionId, long shardGrou Map properties = new HashMap<>(); properties.put(LakeTablet.PROPERTY_KEY_TABLE_ID, Long.toString(table.getId())); - properties.put(LakeTablet.PROPERTY_KEY_PARTITION_ID, Long.toString(partitionId)); + properties.put(LakeTablet.PROPERTY_KEY_PARTITION_ID, Long.toString(physicalPartitionId)); properties.put(LakeTablet.PROPERTY_KEY_INDEX_ID, Long.toString(index.getId())); int bucketNum = distributionInfo.getBucketNum(); WarehouseManager warehouseManager = GlobalStateMgr.getCurrentState().getWarehouseMgr(); @@ -1992,7 +2030,9 @@ private void createLakeTablets(OlapTable table, long partitionId, long shardGrou throw ErrorReportException.report(ErrorCode.ERR_NO_NODES_IN_WAREHOUSE, warehouse.getName()); } List shardIds = stateMgr.getStarOSAgent().createShards(bucketNum, - table.getPartitionFilePathInfo(partitionId), table.getPartitionFileCacheInfo(partitionId), shardGroupId, + table.getPartitionFilePathInfo(physicalPartitionId), + table.getPartitionFileCacheInfo(physicalPartitionId), + shardGroupId, null, properties, workerGroupId.get()); for (long shardId : shardIds) { Tablet tablet = new LakeTablet(shardId); @@ -3286,7 +3326,7 @@ public void alterTableComment(Database db, Table table, AlterTableCommentClause public static void inactiveRelatedMaterializedView(Database db, Table olapTable, String reason) { for (MvId mvId : olapTable.getRelatedMaterializedViews()) { MaterializedView mv = (MaterializedView) GlobalStateMgr.getCurrentState().getLocalMetastore() - .getTable(db.getId(), mvId.getId()); + .getTable(db.getId(), mvId.getId()); if (mv != null) { LOG.warn("Inactive MV {}/{} because {}", mv.getName(), mv.getId(), reason); mv.setInactiveAndReason(reason); diff --git a/fe/fe-core/src/main/java/com/starrocks/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/com/starrocks/service/FrontendServiceImpl.java index bbf3081e39b4ce..7f17662f46bfa0 100644 --- a/fe/fe-core/src/main/java/com/starrocks/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/com/starrocks/service/FrontendServiceImpl.java @@ -2378,7 +2378,7 @@ private static TCreatePartitionResult buildCreatePartitionResponse(OlapTable ola Partition partition = olapTable.getPartition(partitionName); tPartition = new TOlapTablePartition(); - tPartition.setId(partition.getId()); + tPartition.setId(partition.getDefaultPhysicalPartition().getId()); buildPartitionInfo(olapTable, partitions, partition, tPartition, txnState); // tablet int quorum = olapTable.getPartitionInfo().getQuorumNum(partition.getId(), olapTable.writeQuorum()); diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/ExpressionAnalyzer.java b/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/ExpressionAnalyzer.java index 70b780424df1b3..c8cc16c28be830 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/ExpressionAnalyzer.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/ExpressionAnalyzer.java @@ -1649,8 +1649,7 @@ public Void visitDictQueryExpr(DictQueryExpr node, Scope context) { dictQueryExpr.setTbl_name(tableName.getTbl()); Map partitionVersion = new HashMap<>(); - dictTable.getPartitions().forEach(p -> - partitionVersion.put(p.getId(), p.getDefaultPhysicalPartition().getVisibleVersion())); + dictTable.getAllPhysicalPartitions().forEach(p -> partitionVersion.put(p.getId(), p.getVisibleVersion())); dictQueryExpr.setPartition_version(partitionVersion); List keyFields = keyColumns.stream().map(Column::getName).collect(Collectors.toList()); diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/common/MetaUtils.java b/fe/fe-core/src/main/java/com/starrocks/sql/common/MetaUtils.java index 844b3c9372aa09..1ff46ed781f651 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/common/MetaUtils.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/common/MetaUtils.java @@ -172,7 +172,7 @@ public static boolean isPartitionExist(GlobalStateMgr stateMgr, long dbId, long Locker locker = new Locker(); locker.lockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(table.getId()), LockType.READ); try { - return table.getPhysicalPartition(partitionId) != null; + return table.getPartition(partitionId) != null; } finally { locker.unLockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(table.getId()), LockType.READ); } diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/plan/PlanFragmentBuilder.java b/fe/fe-core/src/main/java/com/starrocks/sql/plan/PlanFragmentBuilder.java index 34083a01383a74..82abd25d4765d3 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/plan/PlanFragmentBuilder.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/plan/PlanFragmentBuilder.java @@ -888,7 +888,7 @@ public PlanFragment visitPhysicalOlapScan(OptExpression optExpr, ExecPlan contex if (CollectionUtils.isEmpty(selectTabletIds)) { continue; } - selectedNonEmptyPartitionIds.add(physicalPartition.getId()); + selectedNonEmptyPartitionIds.add(partitionId); Map tabletId2BucketSeq = Maps.newHashMap(); Preconditions.checkState(selectTabletIds != null && !selectTabletIds.isEmpty()); final MaterializedIndex selectedTable = physicalPartition.getIndex(selectedIndexId); diff --git a/fe/fe-core/src/main/java/com/starrocks/task/TabletTaskExecutor.java b/fe/fe-core/src/main/java/com/starrocks/task/TabletTaskExecutor.java index e2994e93aeb8c9..2146354643b1ac 100644 --- a/fe/fe-core/src/main/java/com/starrocks/task/TabletTaskExecutor.java +++ b/fe/fe-core/src/main/java/com/starrocks/task/TabletTaskExecutor.java @@ -167,7 +167,7 @@ public static void buildPartitionsConcurrently(long dbId, OlapTable table, List< } private static List buildCreateReplicaTasks(long dbId, OlapTable table, List partitions, - long warehouseId, boolean enableTabletCreationOptimization) + long warehouseId, boolean enableTabletCreationOptimization) throws DdlException { List tasks = new ArrayList<>(); for (PhysicalPartition partition : partitions) { @@ -177,27 +177,33 @@ private static List buildCreateReplicaTasks(long dbId, OlapTa return tasks; } - private static List buildCreateReplicaTasks(long dbId, OlapTable table, PhysicalPartition partition, - long warehouseId, boolean enableTabletCreationOptimization) + private static List buildCreateReplicaTasks(long dbId, OlapTable table, + PhysicalPartition physicalPartition, + long warehouseId, boolean enableTabletCreationOptimization) throws DdlException { - ArrayList tasks = new ArrayList<>((int) partition.storageReplicaCount()); - for (MaterializedIndex index : partition.getMaterializedIndices(MaterializedIndex.IndexExtState.VISIBLE)) { - tasks.addAll(buildCreateReplicaTasks(dbId, table, partition, index, warehouseId, enableTabletCreationOptimization)); + ArrayList tasks = new ArrayList<>((int) physicalPartition.storageReplicaCount()); + for (MaterializedIndex index : physicalPartition.getMaterializedIndices(MaterializedIndex.IndexExtState.VISIBLE)) { + tasks.addAll(buildCreateReplicaTasks(dbId, table, physicalPartition, index, warehouseId, + enableTabletCreationOptimization)); } return tasks; } - private static List buildCreateReplicaTasks(long dbId, OlapTable table, PhysicalPartition partition, - MaterializedIndex index, long warehouseId, - boolean enableTabletCreationOptimization) throws DdlException { + private static List buildCreateReplicaTasks(long dbId, + OlapTable table, + PhysicalPartition physicalPartition, + MaterializedIndex index, + long warehouseId, + boolean enableTabletCreationOptimization) { LOG.info("build create replica tasks for index {} db {} table {} partition {}", - index, dbId, table.getId(), partition); + index, dbId, table.getId(), physicalPartition); boolean isCloudNativeTable = table.isCloudNativeTableOrMaterializedView(); boolean createSchemaFile = true; List tasks = new ArrayList<>((int) index.getReplicaCount()); MaterializedIndexMeta indexMeta = table.getIndexMetaByIndexId(index.getId()); TTabletType tabletType = isCloudNativeTable ? TTabletType.TABLET_TYPE_LAKE : TTabletType.TABLET_TYPE_DISK; - TStorageMedium storageMedium = table.getPartitionInfo().getDataProperty(partition.getParentId()).getStorageMedium(); + TStorageMedium storageMedium = + table.getPartitionInfo().getDataProperty(physicalPartition.getParentId()).getStorageMedium(); TTabletSchema tabletSchema = SchemaInfo.newBuilder() .setId(indexMeta.getSchemaId()) .setVersion(indexMeta.getSchemaVersion()) @@ -232,10 +238,10 @@ private static List buildCreateReplicaTasks(long dbId, OlapTa .setNodeId(nodeId) .setDbId(dbId) .setTableId(table.getId()) - .setPartitionId(partition.getId()) + .setPartitionId(physicalPartition.getId()) .setIndexId(index.getId()) .setTabletId(tablet.getId()) - .setVersion(partition.getVisibleVersion()) + .setVersion(physicalPartition.getVisibleVersion()) .setStorageMedium(storageMedium) .setEnablePersistentIndex(table.enablePersistentIndex()) .setPersistentIndexType(table.getPersistentIndexType()) @@ -268,7 +274,7 @@ private static void sendCreateReplicaTasksAndWaitForFinished(List tasks, - MarkedCountDownLatch countDownLatch) { + MarkedCountDownLatch countDownLatch) { HashMap> backendToBatchTask = new HashMap<>(); for (CreateReplicaTask task : tasks) { diff --git a/fe/fe-core/src/main/java/com/starrocks/transaction/DatabaseTransactionMgr.java b/fe/fe-core/src/main/java/com/starrocks/transaction/DatabaseTransactionMgr.java index 702cb38d1053b0..8ce2f13a60c8e6 100644 --- a/fe/fe-core/src/main/java/com/starrocks/transaction/DatabaseTransactionMgr.java +++ b/fe/fe-core/src/main/java/com/starrocks/transaction/DatabaseTransactionMgr.java @@ -897,8 +897,8 @@ public boolean canTxnFinished(TransactionState txn, Set errReplicas, Set errReplicas, Set allIndices = txn.getPartitionLoadedTblIndexes(tableId, partition); - int quorumNum = partitionInfo.getQuorumNum(partitionId, table.writeQuorum()); - int replicaNum = partitionInfo.getReplicationNum(partitionId); + int quorumNum = partitionInfo.getQuorumNum(partition.getParentId(), table.writeQuorum()); + int replicaNum = partitionInfo.getReplicationNum(partition.getParentId()); for (MaterializedIndex index : allIndices) { for (Tablet tablet : index.getTablets()) { int successHealthyReplicaNum = 0; @@ -1038,13 +1038,13 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) thr continue; } for (PartitionCommitInfo partitionCommitInfo : idToPartitionCommitInfo.values()) { - long partitionId = partitionCommitInfo.getPartitionId(); - PhysicalPartition partition = table.getPhysicalPartition(partitionId); + long physicalPartitionId = partitionCommitInfo.getPhysicalPartitionId(); + PhysicalPartition physicalPartition = table.getPhysicalPartition(physicalPartitionId); // partition maybe dropped between commit and publish version, ignore this error - if (partition == null) { - droppedPartitionIds.add(partitionId); + if (physicalPartition == null) { + droppedPartitionIds.add(physicalPartitionId); LOG.warn("partition {} is dropped, skip version check and remove it from transaction state {}", - partitionId, + physicalPartitionId, transactionState); continue; } @@ -1052,19 +1052,19 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) thr if (transactionState.getSourceType() != TransactionState.LoadJobSourceType.REPLICATION && !transactionState.isVersionOverwrite() && !partitionCommitInfo.isDoubleWrite() && - partition.getVisibleVersion() != partitionCommitInfo.getVersion() - 1) { + physicalPartition.getVisibleVersion() != partitionCommitInfo.getVersion() - 1) { // prevent excessive logging if (transactionState.getLastErrTimeMs() + 3000 < System.nanoTime() / 1000000) { LOG.debug("transactionId {} partition {} commitInfo version {} is not equal with " + "partition visible version {} plus one, need wait", transactionId, - partitionId, + physicalPartitionId, partitionCommitInfo.getVersion(), - partition.getVisibleVersion()); + physicalPartition.getVisibleVersion()); } String errMsg = String.format("wait for publishing partition %d version %d. self version: %d. table %d", - partitionId, partition.getVisibleVersion() + 1, + physicalPartitionId, physicalPartition.getVisibleVersion() + 1, partitionCommitInfo.getVersion(), tableId); transactionState.setErrorMsg(errMsg); return; @@ -1074,10 +1074,10 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) thr continue; } - int quorumReplicaNum = partitionInfo.getQuorumNum(partitionId, table.writeQuorum()); + int quorumReplicaNum = partitionInfo.getQuorumNum(physicalPartition.getParentId(), table.writeQuorum()); List allIndices = - transactionState.getPartitionLoadedTblIndexes(tableId, partition); + transactionState.getPartitionLoadedTblIndexes(tableId, physicalPartition); for (MaterializedIndex index : allIndices) { for (Tablet tablet : index.getTablets()) { int healthReplicaNum = 0; @@ -1099,7 +1099,7 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) thr } // this means the replica is a healthy replica, // it is healthy in the past and does not have error in current load - if (replica.checkVersionCatchUp(partition.getVisibleVersion(), true)) { + if (replica.checkVersionCatchUp(physicalPartition.getVisibleVersion(), true)) { // during rollup, the rollup replica's last failed version < 0, // it may be treated as a normal replica. @@ -1121,7 +1121,7 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) thr // then we will detect this and set C's last failed version to 10 and last success version to 11 // this logic has to be replayed in checkpoint thread replica.updateVersionInfo(replica.getVersion(), - partition.getVisibleVersion(), + physicalPartition.getVisibleVersion(), partitionCommitInfo.getVersion()); LOG.warn("transaction state {} has error, the replica [{}] not appeared " + "in error replica list and its version not equal to partition " + @@ -1147,8 +1147,8 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) thr String errMsg = String.format( "publish on tablet %d failed. succeed replica num %d less than quorum %d." + " table: %d, partition: %d, publish version: %d", - tablet.getId(), healthReplicaNum, quorumReplicaNum, tableId, partitionId, - partition.getVisibleVersion() + 1); + tablet.getId(), healthReplicaNum, quorumReplicaNum, tableId, physicalPartitionId, + physicalPartition.getVisibleVersion() + 1); transactionState.setErrorMsg(errMsg); hasError = true; } @@ -1247,7 +1247,7 @@ protected void unprotectedCommitPreparedTransaction(TransactionState transaction .values().iterator(); while (partitionCommitInfoIterator.hasNext()) { PartitionCommitInfo partitionCommitInfo = partitionCommitInfoIterator.next(); - long partitionId = partitionCommitInfo.getPartitionId(); + long partitionId = partitionCommitInfo.getPhysicalPartitionId(); PhysicalPartition partition = table.getPhysicalPartition(partitionId); // partition maybe dropped between commit and publish version, ignore this error if (partition == null) { @@ -1261,13 +1261,13 @@ protected void unprotectedCommitPreparedTransaction(TransactionState transaction ReplicationTxnCommitAttachment replicationTxnAttachment = (ReplicationTxnCommitAttachment) transactionState .getTxnCommitAttachment(); Map partitionVersions = replicationTxnAttachment.getPartitionVersions(); - long newVersion = partitionVersions.get(partitionCommitInfo.getPartitionId()); + long newVersion = partitionVersions.get(partitionCommitInfo.getPhysicalPartitionId()); long versionDiff = newVersion - partition.getVisibleVersion(); partitionCommitInfo.setVersion(newVersion); partitionCommitInfo.setDataVersion(partition.getDataVersion() + versionDiff); Map partitionVersionEpochs = replicationTxnAttachment.getPartitionVersionEpochs(); if (partitionVersionEpochs != null) { - long newVersionEpoch = partitionVersionEpochs.get(partitionCommitInfo.getPartitionId()); + long newVersionEpoch = partitionVersionEpochs.get(partitionCommitInfo.getPhysicalPartitionId()); partitionCommitInfo.setVersionEpoch(newVersionEpoch); } } else if (transactionState.isVersionOverwrite()) { @@ -1502,7 +1502,7 @@ protected List> getTableTransInfo(long txnId) throws AnalysisEx List tableInfo = new ArrayList<>(); tableInfo.add(entry.getKey()); tableInfo.add(Joiner.on(", ").join(entry.getValue().getIdToPartitionCommitInfo().values().stream().map( - PartitionCommitInfo::getPartitionId).collect(Collectors.toList()))); + PartitionCommitInfo::getPhysicalPartitionId).collect(Collectors.toList()))); tableInfos.add(tableInfo); } } finally { diff --git a/fe/fe-core/src/main/java/com/starrocks/transaction/LakeTableTxnLogApplier.java b/fe/fe-core/src/main/java/com/starrocks/transaction/LakeTableTxnLogApplier.java index dcb34a9571f5f4..3933e547b641b5 100644 --- a/fe/fe-core/src/main/java/com/starrocks/transaction/LakeTableTxnLogApplier.java +++ b/fe/fe-core/src/main/java/com/starrocks/transaction/LakeTableTxnLogApplier.java @@ -42,7 +42,7 @@ public class LakeTableTxnLogApplier implements TransactionLogApplier { @Override public void applyCommitLog(TransactionState txnState, TableCommitInfo commitInfo) { for (PartitionCommitInfo partitionCommitInfo : commitInfo.getIdToPartitionCommitInfo().values()) { - long partitionId = partitionCommitInfo.getPartitionId(); + long partitionId = partitionCommitInfo.getPhysicalPartitionId(); PhysicalPartition partition = table.getPhysicalPartition(partitionId); if (partition == null) { LOG.warn("ignored dropped partition {} when applying commit log", partitionId); @@ -71,7 +71,7 @@ public void applyVisibleLog(TransactionState txnState, TableCommitInfo commitInf long tableId = table.getId(); CompactionMgr compactionManager = GlobalStateMgr.getCurrentState().getCompactionMgr(); for (PartitionCommitInfo partitionCommitInfo : commitInfo.getIdToPartitionCommitInfo().values()) { - long partitionId = partitionCommitInfo.getPartitionId(); + long partitionId = partitionCommitInfo.getPhysicalPartitionId(); PhysicalPartition partition = table.getPhysicalPartition(partitionId); if (partition == null) { LOG.warn("ignored dropped partition {} when applying visible log", partitionId); diff --git a/fe/fe-core/src/main/java/com/starrocks/transaction/LakeTableTxnStateListener.java b/fe/fe-core/src/main/java/com/starrocks/transaction/LakeTableTxnStateListener.java index 851285db7b8ace..e0cdce94db4124 100644 --- a/fe/fe-core/src/main/java/com/starrocks/transaction/LakeTableTxnStateListener.java +++ b/fe/fe-core/src/main/java/com/starrocks/transaction/LakeTableTxnStateListener.java @@ -137,7 +137,12 @@ public void preCommit(TransactionState txnState, List finished if (enableIngestSlowdown()) { long currentTimeMs = System.currentTimeMillis(); - new CommitRateLimiter(compactionMgr, txnState, table.getId()).check(dirtyPartitionSet, currentTimeMs); + Set partitionIds = Sets.newHashSet(); + for (Long partitionId : dirtyPartitionSet) { + PhysicalPartition partition = table.getPhysicalPartition(partitionId); + partitionIds.add(partition.getParentId()); + } + new CommitRateLimiter(compactionMgr, txnState, table.getId()).check(partitionIds, currentTimeMs); } List unfinishedTablets = null; diff --git a/fe/fe-core/src/main/java/com/starrocks/transaction/OlapTableTxnLogApplier.java b/fe/fe-core/src/main/java/com/starrocks/transaction/OlapTableTxnLogApplier.java index 815947d97f909b..4c0e0c68fcbe6e 100644 --- a/fe/fe-core/src/main/java/com/starrocks/transaction/OlapTableTxnLogApplier.java +++ b/fe/fe-core/src/main/java/com/starrocks/transaction/OlapTableTxnLogApplier.java @@ -45,7 +45,7 @@ public OlapTableTxnLogApplier(OlapTable table) { public void applyCommitLog(TransactionState txnState, TableCommitInfo commitInfo) { Set errorReplicaIds = txnState.getErrorReplicas(); for (PartitionCommitInfo partitionCommitInfo : commitInfo.getIdToPartitionCommitInfo().values()) { - long partitionId = partitionCommitInfo.getPartitionId(); + long partitionId = partitionCommitInfo.getPhysicalPartitionId(); PhysicalPartition partition = table.getPhysicalPartition(partitionId); if (partition == null) { LOG.warn("partition {} is dropped, ignore", partitionId); @@ -101,7 +101,7 @@ public void applyVisibleLog(TransactionState txnState, TableCommitInfo commitInf long maxPartitionVersionTime = -1; for (PartitionCommitInfo partitionCommitInfo : commitInfo.getIdToPartitionCommitInfo().values()) { - long partitionId = partitionCommitInfo.getPartitionId(); + long partitionId = partitionCommitInfo.getPhysicalPartitionId(); PhysicalPartition partition = table.getPhysicalPartition(partitionId); if (partition == null) { LOG.warn("partition {} is dropped, ignore", partitionId); diff --git a/fe/fe-core/src/main/java/com/starrocks/transaction/OlapTableTxnStateListener.java b/fe/fe-core/src/main/java/com/starrocks/transaction/OlapTableTxnStateListener.java index 3b938da3a0ec1c..6012327ee38c20 100644 --- a/fe/fe-core/src/main/java/com/starrocks/transaction/OlapTableTxnStateListener.java +++ b/fe/fe-core/src/main/java/com/starrocks/transaction/OlapTableTxnStateListener.java @@ -54,7 +54,7 @@ public class OlapTableTxnStateListener implements TransactionStateListener { private Set totalInvolvedBackends; private Set errorReplicaIds; - private Set dirtyPartitionSet; + private Set dirtyPhysicalPartitionSet; private Set invalidDictCacheColumns; private Map validDictCacheColumns; @@ -78,7 +78,7 @@ public void preCommit(TransactionState txnState, List tabletCo } totalInvolvedBackends = Sets.newHashSet(); errorReplicaIds = Sets.newHashSet(); - dirtyPartitionSet = Sets.newHashSet(); + dirtyPhysicalPartitionSet = Sets.newHashSet(); invalidDictCacheColumns = Sets.newHashSet(); validDictCacheColumns = Maps.newHashMap(); @@ -110,14 +110,14 @@ public void preCommit(TransactionState txnState, List tabletCo if (tableId != table.getId()) { continue; } - long partitionId = tabletMeta.getPhysicalPartitionId(); - if (table.getPhysicalPartition(partitionId) == null) { + long physicalPartitionId = tabletMeta.getPhysicalPartitionId(); + if (table.getPhysicalPartition(physicalPartitionId) == null) { // this can happen when partitionId == -1 (tablet being dropping) // or partition really not exist. - LOG.warn("partition {} not exist, ignore tablet {}", partitionId, tabletId); + LOG.warn("partition {} not exist, ignore tablet {}", physicalPartitionId, tabletId); continue; } - dirtyPartitionSet.add(partitionId); + dirtyPhysicalPartitionSet.add(physicalPartitionId); tabletToBackends.computeIfAbsent(tabletId, id -> new HashSet<>()) .add(tabletCommitInfos.get(i).getBackendId()); allCommittedBackends.add(tabletCommitInfos.get(i).getBackendId()); @@ -173,7 +173,7 @@ public void preCommit(TransactionState txnState, List tabletCo } for (PhysicalPartition partition : table.getAllPhysicalPartitions()) { - if (!dirtyPartitionSet.contains(partition.getId())) { + if (!dirtyPhysicalPartitionSet.contains(partition.getId())) { continue; } @@ -253,7 +253,7 @@ public void preWriteCommitLog(TransactionState txnState) { TableCommitInfo tableCommitInfo = new TableCommitInfo(table.getId()); boolean isFirstPartition = true; txnState.getErrorReplicas().addAll(errorReplicaIds); - for (long partitionId : dirtyPartitionSet) { + for (long physicalPartitionId : dirtyPhysicalPartitionSet) { PartitionCommitInfo partitionCommitInfo; if (isFirstPartition) { @@ -264,14 +264,14 @@ public void preWriteCommitLog(TransactionState txnState) { validDictCacheColumnNames.add(name); validDictCacheColumnVersions.add(dictVersion); }); - partitionCommitInfo = new PartitionCommitInfo(partitionId, + partitionCommitInfo = new PartitionCommitInfo(physicalPartitionId, -1, System.currentTimeMillis(), Lists.newArrayList(invalidDictCacheColumns), validDictCacheColumnNames, validDictCacheColumnVersions); } else { - partitionCommitInfo = new PartitionCommitInfo(partitionId, + partitionCommitInfo = new PartitionCommitInfo(physicalPartitionId, -1, System.currentTimeMillis() /* use as partition visible time */); } diff --git a/fe/fe-core/src/main/java/com/starrocks/transaction/PartitionCommitInfo.java b/fe/fe-core/src/main/java/com/starrocks/transaction/PartitionCommitInfo.java index 702415b079742b..778172acca9043 100644 --- a/fe/fe-core/src/main/java/com/starrocks/transaction/PartitionCommitInfo.java +++ b/fe/fe-core/src/main/java/com/starrocks/transaction/PartitionCommitInfo.java @@ -51,7 +51,7 @@ public class PartitionCommitInfo implements Writable { @SerializedName(value = "partitionId") - private long partitionId; + private long physicalPartitionId; @SerializedName(value = "version") private long version; @@ -92,19 +92,19 @@ public PartitionCommitInfo() { } - public PartitionCommitInfo(long partitionId, long version, long visibleTime) { + public PartitionCommitInfo(long physicalPartitionId, long version, long visibleTime) { super(); - this.partitionId = partitionId; + this.physicalPartitionId = physicalPartitionId; this.version = version; this.versionTime = visibleTime; } - public PartitionCommitInfo(long partitionId, long version, long visibleTime, + public PartitionCommitInfo(long physicalPartitionId, long version, long visibleTime, List invalidDictCacheColumns, List validDictCacheColumns, List dictCollectedVersions) { super(); - this.partitionId = partitionId; + this.physicalPartitionId = physicalPartitionId; this.version = version; this.versionTime = visibleTime; this.invalidDictCacheColumns = invalidDictCacheColumns; @@ -127,8 +127,8 @@ public void setVersionTime(long time) { this.versionTime = time; } - public long getPartitionId() { - return partitionId; + public long getPhysicalPartitionId() { + return physicalPartitionId; } public long getVersion() { @@ -191,7 +191,7 @@ public Quantiles getCompactionScore() { @Override public String toString() { StringBuilder sb = new StringBuilder("partitionid="); - sb.append(partitionId); + sb.append(physicalPartitionId); sb.append(", version=").append(version); sb.append(", versionHash=").append(0); sb.append(", versionTime=").append(versionTime); diff --git a/fe/fe-core/src/main/java/com/starrocks/transaction/PublishVersionDaemon.java b/fe/fe-core/src/main/java/com/starrocks/transaction/PublishVersionDaemon.java index 7354de274ae704..9d97d702834a8a 100644 --- a/fe/fe-core/src/main/java/com/starrocks/transaction/PublishVersionDaemon.java +++ b/fe/fe-core/src/main/java/com/starrocks/transaction/PublishVersionDaemon.java @@ -764,7 +764,7 @@ private boolean publishPartition(@NotNull Database db, @NotNull TableCommitInfo LOG.info("Removed non-exist table {} from transaction {}. txn_id={}", tableId, txnLabel, txnId); return true; } - long partitionId = partitionCommitInfo.getPartitionId(); + long partitionId = partitionCommitInfo.getPhysicalPartitionId(); PhysicalPartition partition = table.getPhysicalPartition(partitionId); if (partition == null) { LOG.info("Ignore non-exist partition {} of table {} in txn {}", partitionId, table.getName(), txnLabel); @@ -813,7 +813,7 @@ private boolean publishPartition(@NotNull Database db, @NotNull TableCommitInfo Math.abs(partitionCommitInfo.getVersionTime()) + 10000 < System.currentTimeMillis()) { return false; } - LOG.error("Fail to publish partition {} of txn {}: {}", partitionCommitInfo.getPartitionId(), + LOG.error("Fail to publish partition {} of txn {}: {}", partitionCommitInfo.getPhysicalPartitionId(), txnId, e.getMessage()); return false; } diff --git a/fe/fe-core/src/main/java/com/starrocks/transaction/TableCommitInfo.java b/fe/fe-core/src/main/java/com/starrocks/transaction/TableCommitInfo.java index 5c145e2a36a4b8..8b7bc127b87159 100644 --- a/fe/fe-core/src/main/java/com/starrocks/transaction/TableCommitInfo.java +++ b/fe/fe-core/src/main/java/com/starrocks/transaction/TableCommitInfo.java @@ -81,7 +81,7 @@ public void readFields(DataInput in) throws IOException { int elementNum = in.readInt(); for (int i = 0; i < elementNum; ++i) { PartitionCommitInfo partitionCommitInfo = PartitionCommitInfo.read(in); - idToPartitionCommitInfo.put(partitionCommitInfo.getPartitionId(), partitionCommitInfo); + idToPartitionCommitInfo.put(partitionCommitInfo.getPhysicalPartitionId(), partitionCommitInfo); } } } @@ -95,7 +95,7 @@ public Map getIdToPartitionCommitInfo() { } public void addPartitionCommitInfo(PartitionCommitInfo info) { - this.idToPartitionCommitInfo.put(info.getPartitionId(), info); + this.idToPartitionCommitInfo.put(info.getPhysicalPartitionId(), info); } public void removePartition(long partitionId) { diff --git a/fe/fe-core/src/main/java/com/starrocks/transaction/TransactionChecker.java b/fe/fe-core/src/main/java/com/starrocks/transaction/TransactionChecker.java index 3bb98a819381dd..21edc6b11cb1ef 100644 --- a/fe/fe-core/src/main/java/com/starrocks/transaction/TransactionChecker.java +++ b/fe/fe-core/src/main/java/com/starrocks/transaction/TransactionChecker.java @@ -109,7 +109,7 @@ public static TransactionChecker create(TransactionState txn, Database db) { locker.lockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(table.getId()), LockType.READ); for (PartitionCommitInfo partitionCommitInfo : tableCommitInfo.getIdToPartitionCommitInfo().values()) { - long partitionId = partitionCommitInfo.getPartitionId(); + long partitionId = partitionCommitInfo.getPhysicalPartitionId(); PhysicalPartition partition = table.getPhysicalPartition(partitionId); if (partition == null) { continue; diff --git a/fe/fe-core/src/main/java/com/starrocks/transaction/TransactionState.java b/fe/fe-core/src/main/java/com/starrocks/transaction/TransactionState.java index 5b91fbb3350349..7a0564b3df1438 100644 --- a/fe/fe-core/src/main/java/com/starrocks/transaction/TransactionState.java +++ b/fe/fe-core/src/main/java/com/starrocks/transaction/TransactionState.java @@ -894,7 +894,7 @@ public List createPublishVersionTask() { List partitionVersions = new ArrayList<>(partitionCommitInfos.size()); for (PartitionCommitInfo commitInfo : partitionCommitInfos) { - TPartitionVersionInfo version = new TPartitionVersionInfo(commitInfo.getPartitionId(), + TPartitionVersionInfo version = new TPartitionVersionInfo(commitInfo.getPhysicalPartitionId(), commitInfo.getVersion(), 0); if (commitInfo.isDoubleWrite()) { version.setIs_double_write(true); diff --git a/fe/fe-core/src/test/java/com/starrocks/alter/AlterTest.java b/fe/fe-core/src/test/java/com/starrocks/alter/AlterTest.java index 330009ba036d09..fa77d1284401c0 100644 --- a/fe/fe-core/src/test/java/com/starrocks/alter/AlterTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/alter/AlterTest.java @@ -2749,8 +2749,8 @@ public void testAlterForeignKey() throws Exception { replayNextJournal(OperationType.OP_ALTER_MATERIALIZED_VIEW_PROPERTIES); Assert.assertNotNull(modifyMvLog); if (modifyMvLog.getProperties().containsKey("foreign_key_constraints")) { - Assert.assertEquals("default_catalog.10001.10133(site_id) " + - "REFERENCES default_catalog.10001.10118(site_id)", + Assert.assertEquals("default_catalog.10001.10145(site_id) " + + "REFERENCES default_catalog.10001.10129(site_id)", modifyMvLog.getProperties().get("foreign_key_constraints")); break; } diff --git a/fe/fe-core/src/test/java/com/starrocks/alter/LakeTableAlterDataCachePartitionDurationTest.java b/fe/fe-core/src/test/java/com/starrocks/alter/LakeTableAlterDataCachePartitionDurationTest.java index 733f17281e57f7..cc86684c89a55d 100644 --- a/fe/fe-core/src/test/java/com/starrocks/alter/LakeTableAlterDataCachePartitionDurationTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/alter/LakeTableAlterDataCachePartitionDurationTest.java @@ -131,7 +131,7 @@ public void logSaveNextId(long nextId) { table = new LakeTable(tableId, "t0", Collections.singletonList(c0), keysType, partitionInfo, dist); MaterializedIndex index = new MaterializedIndex(indexId, MaterializedIndex.IndexState.NORMAL); - Partition partition = new Partition(partitionId, "t0", index, dist); + Partition partition = new Partition(partitionId, partitionId + 100, "t0", index, dist); TStorageMedium storage = TStorageMedium.HDD; TabletMeta tabletMeta = new TabletMeta(db.getId(), table.getId(), partition.getId(), index.getId(), 0, storage, true); for (int i = 0; i < NUM_BUCKETS; i++) { diff --git a/fe/fe-core/src/test/java/com/starrocks/alter/LakeTableAlterMetaJobTest.java b/fe/fe-core/src/test/java/com/starrocks/alter/LakeTableAlterMetaJobTest.java index 5e2b84938938bc..93cec5777f8dbc 100644 --- a/fe/fe-core/src/test/java/com/starrocks/alter/LakeTableAlterMetaJobTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/alter/LakeTableAlterMetaJobTest.java @@ -17,7 +17,7 @@ import com.google.common.collect.Table; import com.starrocks.catalog.Database; import com.starrocks.catalog.MaterializedIndex; -import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PhysicalPartition; import com.starrocks.common.Config; import com.starrocks.common.DdlException; import com.starrocks.common.MetaNotFoundException; @@ -236,10 +236,10 @@ public void testReplay() { // for replay will check partition.getVisibleVersion() // here we reduce the visibleVersion for test for (long partitionId : partitionIndexMap.rowKeySet()) { - Partition partition = table.getPartition(partitionId); + PhysicalPartition physicalPartition = table.getPhysicalPartition(partitionId); long commitVersion = commitVersionMap.get(partitionId); - Assert.assertEquals(partition.getDefaultPhysicalPartition().getVisibleVersion(), commitVersion); - partition.getDefaultPhysicalPartition().updateVisibleVersion(commitVersion - 1); + Assert.assertEquals(physicalPartition.getVisibleVersion(), commitVersion); + physicalPartition.updateVisibleVersion(commitVersion - 1); } replayAlterMetaJob.replay(job); @@ -254,9 +254,9 @@ public void testReplay() { Assert.assertEquals(job.getPartitionIndexMap(), replayAlterMetaJob.getPartitionIndexMap()); for (long partitionId : partitionIndexMap.rowKeySet()) { - Partition partition = table.getPartition(partitionId); + PhysicalPartition physicalPartition = table.getPhysicalPartition(partitionId); long commitVersion = commitVersionMap.get(partitionId); - Assert.assertEquals(partition.getDefaultPhysicalPartition().getVisibleVersion(), commitVersion); + Assert.assertEquals(physicalPartition.getVisibleVersion(), commitVersion); } } diff --git a/fe/fe-core/src/test/java/com/starrocks/alter/LakeTableAsyncFastSchemaChangeJobTest.java b/fe/fe-core/src/test/java/com/starrocks/alter/LakeTableAsyncFastSchemaChangeJobTest.java index b75068d9a913bd..fbccc3c7d1a947 100644 --- a/fe/fe-core/src/test/java/com/starrocks/alter/LakeTableAsyncFastSchemaChangeJobTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/alter/LakeTableAsyncFastSchemaChangeJobTest.java @@ -185,8 +185,8 @@ public void testReplay() throws Exception { Assert.assertEquals(OlapTable.OlapTableState.SCHEMA_CHANGE, table.getState()); replaySourceJob.setJobState(AlterJobV2.JobState.FINISHED_REWRITING); - replaySourceJob.getCommitVersionMap().put(partition.getId(), initNextVersion); - replaySourceJob.addDirtyPartitionIndex(partition.getId(), baseIndexId, + replaySourceJob.getCommitVersionMap().put(partition.getDefaultPhysicalPartition().getId(), initNextVersion); + replaySourceJob.addDirtyPartitionIndex(partition.getDefaultPhysicalPartition().getId(), baseIndexId, partition.getDefaultPhysicalPartition().getIndex(baseIndexId)); job.replay(replaySourceJob); Assert.assertEquals(initNextVersion + 1, partition.getDefaultPhysicalPartition().getNextVersion()); diff --git a/fe/fe-core/src/test/java/com/starrocks/analysis/AccessTestUtil.java b/fe/fe-core/src/test/java/com/starrocks/analysis/AccessTestUtil.java index 7a717daa64b9ee..efa5a0da3fadeb 100644 --- a/fe/fe-core/src/test/java/com/starrocks/analysis/AccessTestUtil.java +++ b/fe/fe-core/src/test/java/com/starrocks/analysis/AccessTestUtil.java @@ -38,6 +38,7 @@ import com.google.common.collect.Sets; import com.starrocks.catalog.Column; import com.starrocks.catalog.Database; +import com.starrocks.catalog.FakeEditLog; import com.starrocks.catalog.InternalCatalog; import com.starrocks.catalog.KeysType; import com.starrocks.catalog.MaterializedIndex; @@ -74,10 +75,12 @@ public static GlobalStateMgr fetchAdminCatalog() { EditLog editLog = new EditLog(journalQueue); globalStateMgr.setEditLog(editLog); + FakeEditLog fakeEditLog = new FakeEditLog(); + Database db = new Database(50000L, "testCluster:testDb"); MaterializedIndex baseIndex = new MaterializedIndex(30001, IndexState.NORMAL); RandomDistributionInfo distributionInfo = new RandomDistributionInfo(10); - Partition partition = new Partition(20000L, "testTbl", baseIndex, distributionInfo); + Partition partition = new Partition(20000L, 20001L,"testTbl", baseIndex, distributionInfo); List baseSchema = new LinkedList(); Column column = new Column("k1", Type.INT); baseSchema.add(column); diff --git a/fe/fe-core/src/test/java/com/starrocks/analysis/DropMaterializedViewStmtTest.java b/fe/fe-core/src/test/java/com/starrocks/analysis/DropMaterializedViewStmtTest.java index 4e38653dda8195..65039352042eba 100644 --- a/fe/fe-core/src/test/java/com/starrocks/analysis/DropMaterializedViewStmtTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/analysis/DropMaterializedViewStmtTest.java @@ -80,7 +80,7 @@ public void setUp() { baseSchema, KeysType.AGG_KEYS, singlePartitionInfo, null); table.setBaseIndexId(100); db.registerTableUnlocked(table); - table.addPartition(new Partition(100, "p", + table.addPartition(new Partition(100, 101, "p", new MaterializedIndex(200, MaterializedIndex.IndexState.NORMAL), null)); table.setIndexMeta(200, "mvname", baseSchema, 0, 0, (short) 0, TStorageType.COLUMN, KeysType.AGG_KEYS); diff --git a/fe/fe-core/src/test/java/com/starrocks/analysis/RuntimeFilterTest.java b/fe/fe-core/src/test/java/com/starrocks/analysis/RuntimeFilterTest.java index 3a4be32c5126a4..6bb397bff4ee91 100644 --- a/fe/fe-core/src/test/java/com/starrocks/analysis/RuntimeFilterTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/analysis/RuntimeFilterTest.java @@ -101,24 +101,20 @@ void testRuntimeFilterPushDown() ") tbl order by 1 desc limit 15"; String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql); System.out.println(plan); - Assert.assertTrue(plan, plan.contains("4:Project\n" + + Assert.assertTrue(plan, plan.contains("7:Project\n" + " | output columns:\n" + - " | 14 <-> [14: k1, DATE, true]\n" + - " | 26 <-> [26: k13, DECIMAL128(27,9), true]\n" + + " | 39 <-> [39: k13, DECIMAL128(27,9), true]\n" + " | cardinality: 1\n" + " | \n" + - " 3:OlapScanNode\n" + + " 6:OlapScanNode\n" + " table: duplicate_par_tbl, rollup: duplicate_par_tbl\n" + " preAggregation: on\n" + - " Predicates: [26: k13, DECIMAL128(27,9), true] > 60, 16: k3 IN ('beijing', '')\n" + + " Predicates: [39: k13, DECIMAL128(27,9), true] > 60, 31: k5 IS NULL\n" + " partitionsRatio=1/3, tabletsRatio=3/3\n" + - " tabletList=10007,10009,10011\n" + - " actualRows=0, avgRowSize=3.0\n" + + " tabletList=10015,10017,10019\n" + + " actualRows=0, avgRowSize=2.0\n" + " cardinality: 1\n" + " probe runtime filters:\n" + - " - filter_id = 0, probe_expr = (26: k13)\n" + - " - filter_id = 1, probe_expr = (26: k13)\n" + - "\n" + - "PLAN FRAGMENT 7(F00)")); + " - filter_id = 1, probe_expr = (39: k13)")); } } \ No newline at end of file diff --git a/fe/fe-core/src/test/java/com/starrocks/backup/BackupJobPrimaryKeyTest.java b/fe/fe-core/src/test/java/com/starrocks/backup/BackupJobPrimaryKeyTest.java index 72cc02e534fd94..9c904da3d1b134 100644 --- a/fe/fe-core/src/test/java/com/starrocks/backup/BackupJobPrimaryKeyTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/backup/BackupJobPrimaryKeyTest.java @@ -47,6 +47,7 @@ import com.starrocks.common.FeConstants; import com.starrocks.common.jmockit.Deencapsulation; import com.starrocks.common.util.UnitTestUtil; +import com.starrocks.common.util.concurrent.lock.LockManager; import com.starrocks.metric.MetricRepo; import com.starrocks.persist.EditLog; import com.starrocks.server.GlobalStateMgr; @@ -61,6 +62,7 @@ import com.starrocks.thrift.TStatus; import com.starrocks.thrift.TStatusCode; import com.starrocks.thrift.TTaskType; +import com.starrocks.transaction.GtidGenerator; import mockit.Delegate; import mockit.Expectations; import mockit.Mock; @@ -174,8 +176,14 @@ public void setUp() { db = UnitTestUtil.createDbByName(dbId, tblId, partId, idxId, tabletId, backendId, version, KeysType.PRIMARY_KEYS, testDbName, testTableName); + LockManager lockManager = new LockManager(); + new Expectations(globalStateMgr) { { + GlobalStateMgr.getCurrentState(); + minTimes = 0; + result = globalStateMgr; + globalStateMgr.getLocalMetastore().getDb(anyLong); minTimes = 0; result = db; @@ -188,6 +196,14 @@ public void setUp() { minTimes = 0; result = editLog; + globalStateMgr.getLockManager(); + minTimes = 0; + result = lockManager; + + globalStateMgr.getGtidGenerator(); + minTimes = 0; + result = new GtidGenerator(); + globalStateMgr.getLocalMetastore().getTable(testDbName, testTableName); minTimes = 0; result = db.getTable(tblId); diff --git a/fe/fe-core/src/test/java/com/starrocks/backup/BackupJobTest.java b/fe/fe-core/src/test/java/com/starrocks/backup/BackupJobTest.java index d77e70b4395ed5..454685bea89353 100644 --- a/fe/fe-core/src/test/java/com/starrocks/backup/BackupJobTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/backup/BackupJobTest.java @@ -48,6 +48,7 @@ import com.starrocks.common.FeConstants; import com.starrocks.common.jmockit.Deencapsulation; import com.starrocks.common.util.UnitTestUtil; +import com.starrocks.common.util.concurrent.lock.LockManager; import com.starrocks.metric.MetricRepo; import com.starrocks.persist.EditLog; import com.starrocks.server.GlobalStateMgr; @@ -62,6 +63,7 @@ import com.starrocks.thrift.TStatus; import com.starrocks.thrift.TStatusCode; import com.starrocks.thrift.TTaskType; +import com.starrocks.transaction.GtidGenerator; import mockit.Delegate; import mockit.Expectations; import mockit.Mock; @@ -171,8 +173,22 @@ public void setUp() { View view = UnitTestUtil.createTestView(viewId); db.registerTableUnlocked(view); + LockManager lockManager = new LockManager(); + new Expectations(globalStateMgr) { { + GlobalStateMgr.getCurrentState(); + minTimes = 0; + result = globalStateMgr; + + globalStateMgr.getLockManager(); + minTimes = 0; + result = lockManager; + + globalStateMgr.getGtidGenerator(); + minTimes = 0; + result = new GtidGenerator(); + globalStateMgr.getLocalMetastore().getDb(anyLong); minTimes = 0; result = db; diff --git a/fe/fe-core/src/test/java/com/starrocks/backup/CatalogMocker.java b/fe/fe-core/src/test/java/com/starrocks/backup/CatalogMocker.java index 7a9f22feaa980f..8cd180885398dd 100644 --- a/fe/fe-core/src/test/java/com/starrocks/backup/CatalogMocker.java +++ b/fe/fe-core/src/test/java/com/starrocks/backup/CatalogMocker.java @@ -104,6 +104,7 @@ public class CatalogMocker { public static final String TEST_SINGLE_PARTITION_NAME = TEST_TBL_NAME; public static final long TEST_SINGLE_PARTITION_ID = 40000; + public static final long TEST_SINGLE_PHYSICAL_PARTITION_ID = 40011; public static final long TEST_TABLET0_ID = 60000; public static final long TEST_REPLICA0_ID = 70000; public static final long TEST_REPLICA1_ID = 70001; @@ -141,12 +142,16 @@ public class CatalogMocker { public static final String TEST_PARTITION1_NAME = "p1"; public static final long TEST_PARTITION1_ID = 40001; + public static final long TEST_PH_PARTITION1_ID = 40011; public static final String TEST_PARTITION2_NAME = "p2"; public static final long TEST_PARTITION2_ID = 40002; + public static final long TEST_PH_PARTITION2_ID = 40012; public static final String TEST_PARTITION1_NAME_PK = "p1_pk"; public static final long TEST_PARTITION1_PK_ID = 40003; + public static final long TEST_PH_PARTITION1_PK_ID = 40013; public static final String TEST_PARTITION2_NAME_PK = "p2_pk"; public static final long TEST_PARTITION2_PK_ID = 40004; + public static final long TEST_PH_PARTITION2_PK_ID = 40014; public static final long TEST_BASE_TABLET_P1_ID = 60001; public static final long TEST_REPLICA3_ID = 70003; @@ -260,7 +265,8 @@ public static Database mockDb() throws AnalysisException { MaterializedIndex baseIndex = new MaterializedIndex(TEST_TBL_ID, IndexState.NORMAL); DistributionInfo distributionInfo = new RandomDistributionInfo(32); Partition partition = - new Partition(TEST_SINGLE_PARTITION_ID, TEST_SINGLE_PARTITION_NAME, baseIndex, distributionInfo); + new Partition(TEST_SINGLE_PARTITION_ID, TEST_SINGLE_PHYSICAL_PARTITION_ID, + TEST_SINGLE_PARTITION_NAME, baseIndex, distributionInfo); PartitionInfo partitionInfo = new SinglePartitionInfo(); partitionInfo.setReplicationNum(TEST_SINGLE_PARTITION_ID, (short) 3); partitionInfo.setIsInMemory(TEST_SINGLE_PARTITION_ID, false); @@ -309,9 +315,9 @@ public static Database mockDb() throws AnalysisException { DistributionInfo distributionInfo2 = new HashDistributionInfo(32, Lists.newArrayList(TEST_TBL_BASE_SCHEMA.get(1))); Partition partition1 = - new Partition(TEST_PARTITION1_ID, TEST_PARTITION1_NAME, baseIndexP1, distributionInfo2); + new Partition(TEST_PARTITION1_ID, TEST_PH_PARTITION1_ID, TEST_PARTITION1_NAME, baseIndexP1, distributionInfo2); Partition partition2 = - new Partition(TEST_PARTITION2_ID, TEST_PARTITION2_NAME, baseIndexP2, distributionInfo2); + new Partition(TEST_PARTITION2_ID, TEST_PH_PARTITION2_ID, TEST_PARTITION2_NAME, baseIndexP2, distributionInfo2); RangePartitionInfo rangePartitionInfo = new RangePartitionInfo(Lists.newArrayList(TEST_TBL_BASE_SCHEMA.get(0))); DataProperty dataPropertyP1 = new DataProperty(TStorageMedium.HDD); PartitionKey rangeP1Lower = @@ -408,9 +414,11 @@ public static Database mockDb() throws AnalysisException { DistributionInfo distributionInfo3 = new HashDistributionInfo(32, Lists.newArrayList(TEST_TBL_BASE_SCHEMA.get(1))); Partition partition1Pk = - new Partition(TEST_PARTITION1_PK_ID, TEST_PARTITION1_NAME_PK, baseIndexP1Pk, distributionInfo3); + new Partition(TEST_PARTITION1_PK_ID, TEST_PH_PARTITION1_ID, + TEST_PARTITION1_NAME_PK, baseIndexP1Pk, distributionInfo3); Partition partition2Pk = - new Partition(TEST_PARTITION2_PK_ID, TEST_PARTITION2_NAME_PK, baseIndexP2Pk, distributionInfo3); + new Partition(TEST_PARTITION2_PK_ID, TEST_PH_PARTITION2_ID, + TEST_PARTITION2_NAME_PK, baseIndexP2Pk, distributionInfo3); RangePartitionInfo rangePartitionInfoPk = new RangePartitionInfo(Lists.newArrayList(TEST_TBL_BASE_SCHEMA.get(0))); PartitionKey rangeP1LowerPk = @@ -477,7 +485,8 @@ public static Database mockDb() throws AnalysisException { baseIndexP2 = new MaterializedIndex(TEST_TBL4_ID, IndexState.NORMAL); DistributionInfo distributionInfo4 = new RandomDistributionInfo(1); partition1 = - new Partition(TEST_PARTITION1_ID, TEST_PARTITION1_NAME, baseIndexP1, distributionInfo4); + new Partition(TEST_PARTITION1_ID, TEST_PH_PARTITION1_ID, + TEST_PARTITION1_NAME, baseIndexP1, distributionInfo4); PhysicalPartition physicalPartition2 = new PhysicalPartition( TEST_PARTITION2_ID, "", TEST_PARTITION1_ID, 0, baseIndexP2); @@ -526,7 +535,8 @@ public static Database mockDb() throws AnalysisException { baseIndexP1 = new MaterializedIndex(TEST_TBL5_ID, IndexState.NORMAL); baseIndexP2 = new MaterializedIndex(TEST_TBL5_ID, IndexState.NORMAL); DistributionInfo distributionInfo5 = new RandomDistributionInfo(1); - partition1 = new Partition(TEST_PARTITION1_ID, TEST_PARTITION1_NAME, baseIndexP1, distributionInfo5); + partition1 = new Partition(TEST_PARTITION1_ID, TEST_PH_PARTITION1_ID, + TEST_PARTITION1_NAME, baseIndexP1, distributionInfo5); ListPartitionInfo listPartitionInfo = new ListPartitionInfo(PartitionType.LIST, Lists.newArrayList(TEST_TBL_BASE_SCHEMA.get(0))); diff --git a/fe/fe-core/src/test/java/com/starrocks/backup/RestoreJobTest.java b/fe/fe-core/src/test/java/com/starrocks/backup/RestoreJobTest.java index 897ada9c8011f9..fc88f908762625 100644 --- a/fe/fe-core/src/test/java/com/starrocks/backup/RestoreJobTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/backup/RestoreJobTest.java @@ -46,6 +46,7 @@ import com.starrocks.backup.RestoreJob.RestoreJobState; import com.starrocks.backup.mv.MvRestoreContext; import com.starrocks.catalog.Database; +import com.starrocks.catalog.FakeEditLog; import com.starrocks.catalog.Function; import com.starrocks.catalog.KeysType; import com.starrocks.catalog.MaterializedIndex; @@ -107,13 +108,15 @@ public class RestoreJobTest { private long repoId = 20000; - @Mocked private GlobalStateMgr globalStateMgr; private MockBackupHandler backupHandler; private MockRepositoryMgr repoMgr; + @Mocked + private EditLog editLog; + // Thread is not mockable in Jmockit, use subclass instead private final class MockBackupHandler extends BackupHandler { public MockBackupHandler(GlobalStateMgr globalStateMgr) { @@ -138,8 +141,6 @@ public Repository getRepo(long repoId) { } } - @Mocked - private EditLog editLog; @Mocked private SystemInfoService systemInfoService; @@ -151,11 +152,28 @@ public Repository getRepo(long repoId) { @Before public void setUp() throws AnalysisException { + globalStateMgr = GlobalStateMgr.getCurrentState(); + new FakeEditLog(); + db = CatalogMocker.mockDb(); backupHandler = new MockBackupHandler(globalStateMgr); repoMgr = new MockRepositoryMgr(); Deencapsulation.setField(globalStateMgr, "backupHandler", backupHandler); MetricRepo.init(); + + new Expectations(globalStateMgr) { + { + GlobalStateMgr.getCurrentState(); + minTimes = 0; + result = globalStateMgr; + + globalStateMgr.getEditLog(); + minTimes = 0; + result = editLog; + } + }; + + AgentTaskQueue.clearAllTasks(); } public void testResetPartitionForRestore() { @@ -176,21 +194,14 @@ public void testResetPartitionForRestore() { public void testRunBackupMultiSubPartitionTable() { new Expectations() { { - globalStateMgr.getLocalMetastore().getDb(anyLong); - minTimes = 0; - result = db; - globalStateMgr.getNextId(); + GlobalStateMgr.getCurrentState(); minTimes = 0; - result = 50000; + result = globalStateMgr; - globalStateMgr.getNextId(); - minTimes = 0; - result = 50001; - - globalStateMgr.getEditLog(); + globalStateMgr.getLocalMetastore().getDb(anyLong); minTimes = 0; - result = editLog; + result = db; GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo(); minTimes = 0; @@ -198,6 +209,8 @@ public void testRunBackupMultiSubPartitionTable() { } }; + FakeEditLog fakeEditLog = new FakeEditLog(); + List beIds = Lists.newArrayList(); beIds.add(CatalogMocker.BACKEND1_ID); beIds.add(CatalogMocker.BACKEND2_ID); @@ -214,18 +227,6 @@ public void testRunBackupMultiSubPartitionTable() { } }; - new Expectations() { - { - editLog.logBackupJob((BackupJob) any); - minTimes = 0; - result = new Delegate() { - public void logBackupJob(BackupJob job) { - System.out.println("log backup job: " + job); - } - }; - } - }; - new Expectations() { { repo.upload(anyString, anyString); @@ -326,13 +327,13 @@ public Table getTable(Long dbId, Long tableId) { job.run(); Assert.assertEquals(Status.OK, job.getStatus()); Assert.assertEquals(RestoreJobState.SNAPSHOTING, job.getState()); - Assert.assertEquals(1, job.getFileMapping().getMapping().size()); + Assert.assertEquals(6, job.getFileMapping().getMapping().size()); // 2. snapshoting job.run(); Assert.assertEquals(Status.OK, job.getStatus()); Assert.assertEquals(RestoreJobState.SNAPSHOTING, job.getState()); - Assert.assertEquals(4, AgentTaskQueue.getTaskNum()); + Assert.assertEquals(12, AgentTaskQueue.getTaskNum()); // 3. snapshot finished List agentTasks = Lists.newArrayList(); @@ -340,7 +341,7 @@ public Table getTable(Long dbId, Long tableId) { agentTasks.addAll(AgentTaskQueue.getDiffTasks(CatalogMocker.BACKEND1_ID, runningTasks)); agentTasks.addAll(AgentTaskQueue.getDiffTasks(CatalogMocker.BACKEND2_ID, runningTasks)); agentTasks.addAll(AgentTaskQueue.getDiffTasks(CatalogMocker.BACKEND3_ID, runningTasks)); - Assert.assertEquals(4, agentTasks.size()); + Assert.assertEquals(12, agentTasks.size()); for (AgentTask agentTask : agentTasks) { if (agentTask.getTaskType() != TTaskType.MAKE_SNAPSHOT) { @@ -380,10 +381,6 @@ public void testRunBackupRangeTable() { minTimes = 0; result = id.incrementAndGet(); - globalStateMgr.getEditLog(); - minTimes = 0; - result = editLog; - GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo(); minTimes = 0; result = systemInfoService; @@ -406,18 +403,6 @@ public void testRunBackupRangeTable() { } }; - new Expectations() { - { - editLog.logBackupJob((BackupJob) any); - minTimes = 0; - result = new Delegate() { - public void logBackupJob(BackupJob job) { - System.out.println("log backup job: " + job); - } - }; - } - }; - new Expectations() { { repo.upload(anyString, anyString); @@ -559,10 +544,6 @@ public void testRunBackupListTable() { minTimes = 0; result = id.incrementAndGet(); - globalStateMgr.getEditLog(); - minTimes = 0; - result = editLog; - GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo(); minTimes = 0; result = systemInfoService; @@ -585,18 +566,6 @@ public void testRunBackupListTable() { } }; - new Expectations() { - { - editLog.logBackupJob((BackupJob) any); - minTimes = 0; - result = new Delegate() { - public void logBackupJob(BackupJob job) { - System.out.println("log backup job: " + job); - } - }; - } - }; - new Expectations() { { repo.upload(anyString, anyString); @@ -786,28 +755,12 @@ public void testRestoreView() { minTimes = 0; result = id.incrementAndGet(); - globalStateMgr.getEditLog(); - minTimes = 0; - result = editLog; - GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo(); minTimes = 0; result = systemInfoService; } }; - new Expectations() { - { - editLog.logBackupJob((BackupJob) any); - minTimes = 0; - result = new Delegate() { - public void logBackupJob(BackupJob job) { - System.out.println("log backup job: " + job); - } - }; - } - }; - new Expectations() { { repo.upload(anyString, anyString); @@ -967,13 +920,13 @@ public synchronized QueryStatement init() throws UserException { public void testRestoreAddFunction() { backupMeta = new BackupMeta(Lists.newArrayList()); Function f1 = new Function(new FunctionName(db.getFullName(), "test_function"), - new Type[] {Type.INT}, new String[] {"argName"}, Type.INT, false); + new Type[] {Type.INT}, new String[] {"argName"}, Type.INT, false); backupMeta.setFunctions(Lists.newArrayList(f1)); job = new RestoreJob(label, "2018-01-01 01:01:01", db.getId(), db.getFullName(), - new BackupJobInfo(), false, 3, 100000, - globalStateMgr, repo.getId(), backupMeta, new MvRestoreContext()); - + new BackupJobInfo(), false, 3, 100000, + globalStateMgr, repo.getId(), backupMeta, new MvRestoreContext()); + job.addRestoredFunctions(db); } } \ No newline at end of file diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/CatalogRecycleBinLakeTableTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/CatalogRecycleBinLakeTableTest.java index 269d935746dbc1..e72799294d9474 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/CatalogRecycleBinLakeTableTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/CatalogRecycleBinLakeTableTest.java @@ -52,6 +52,7 @@ public class CatalogRecycleBinLakeTableTest { @BeforeClass public static void beforeClass() { UtFrameUtils.createMinStarRocksCluster(RunMode.SHARED_DATA); + GlobalStateMgr.getCurrentState().getWarehouseMgr().initDefaultWarehouse(); } private static Table createTable(ConnectContext connectContext, String sql) throws Exception { @@ -571,7 +572,7 @@ public void testRecycleLakePartitionWithSharedDirectory(@Mocked LakeService lake @Mock public boolean isSharedDirectory(String path, long partitionId) { Assert.assertTrue(path.endsWith("/" + partitionId)); - return partitionId == p1.getId(); + return partitionId == p1.getDefaultPhysicalPartition().getId(); } }; new MockUp() { diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/CatalogRecycleBinTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/CatalogRecycleBinTest.java index e750d5110b7670..3baaf9688f6961 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/CatalogRecycleBinTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/CatalogRecycleBinTest.java @@ -118,6 +118,8 @@ public void testGetTable() { @Test public void testGetPartition() throws Exception { + FakeEditLog fakeEditLog = new FakeEditLog(); + CatalogRecycleBin bin = new CatalogRecycleBin(); List columns = Lists.newArrayList(new Column("k1", ScalarType.createVarcharType(10))); Range range = @@ -126,9 +128,9 @@ public void testGetPartition() throws Exception { PartitionKey.createPartitionKey(Lists.newArrayList(new PartitionValue("3")), columns), BoundType.CLOSED); DataProperty dataProperty = new DataProperty(TStorageMedium.HDD); - Partition partition = new Partition(1L, "pt", new MaterializedIndex(), null); + Partition partition = new Partition(1L, 3L, "pt", new MaterializedIndex(), null); bin.recyclePartition(new RecycleRangePartitionInfo(11L, 22L, partition, range, dataProperty, (short) 1, false, null)); - Partition partition2 = new Partition(2L, "pt", new MaterializedIndex(), null); + Partition partition2 = new Partition(2L, 4L, "pt", new MaterializedIndex(), null); bin.recyclePartition(new RecycleRangePartitionInfo(11L, 22L, partition2, range, dataProperty, (short) 1, false, null)); Partition recycledPart = bin.getPartition(1L); @@ -151,15 +153,15 @@ public void testGetPhysicalPartition() throws Exception { PartitionKey.createPartitionKey(Lists.newArrayList(new PartitionValue("3")), columns), BoundType.CLOSED); DataProperty dataProperty = new DataProperty(TStorageMedium.HDD); - Partition partition = new Partition(1L, "pt", new MaterializedIndex(), null); + Partition partition = new Partition(1L, 3L, "pt", new MaterializedIndex(), null); bin.recyclePartition(new RecycleRangePartitionInfo(11L, 22L, partition, range, dataProperty, (short) 1, false, null)); - Partition partition2 = new Partition(2L, "pt", new MaterializedIndex(), null); + Partition partition2 = new Partition(2L, 4L, "pt", new MaterializedIndex(), null); bin.recyclePartition(new RecycleRangePartitionInfo(11L, 22L, partition2, range, dataProperty, (short) 1, false, null)); - PhysicalPartition recycledPart = bin.getPhysicalPartition(1L); + PhysicalPartition recycledPart = bin.getPhysicalPartition(3L); Assert.assertNotNull(recycledPart); - recycledPart = bin.getPhysicalPartition(2L); - Assert.assertEquals(2L, recycledPart.getId()); + recycledPart = bin.getPhysicalPartition(4L); + Assert.assertEquals(4L, recycledPart.getId()); } @Test @@ -222,6 +224,7 @@ public void testAddTabletToInvertedIndexWithLocalTablet(@Mocked GlobalStateMgr g long partitionId = 3L; long indexId = 4L; long tabletId = 5L; + long physicalPartitionId = 6L; long replicaId = 10L; long backendId = 20L; @@ -257,7 +260,7 @@ public void testAddTabletToInvertedIndexWithLocalTablet(@Mocked GlobalStateMgr g index.addTablet(tablet, tabletMeta); // Partition - Partition partition = new Partition(partitionId, "p1", index, distributionInfo); + Partition partition = new Partition(partitionId, physicalPartitionId, "p1", index, distributionInfo); // Table OlapTable table = new OlapTable(tableId, "t1", columns, KeysType.AGG_KEYS, partitionInfo, distributionInfo); @@ -290,12 +293,13 @@ public void testAddTabletToInvertedIndexWithLocalTablet(@Mocked GlobalStateMgr g @Test public void testAddTabletToInvertedIndexWithLocalTabletError(@Mocked GlobalStateMgr globalStateMgr, - @Mocked Database db) { + @Mocked Database db) { long dbId = 1L; long tableId = 2L; long partitionId = 3L; long indexId = 4L; long tabletId = 5L; + long physicalPartitionId = 6L; long replicaId = 10L; long backendId = 20L; @@ -330,7 +334,7 @@ public void testAddTabletToInvertedIndexWithLocalTabletError(@Mocked GlobalState index.addTablet(tablet, tabletMeta); // Partition - Partition partition = new Partition(partitionId, "p1", index, distributionInfo); + Partition partition = new Partition(partitionId, physicalPartitionId, "p1", index, distributionInfo); // Table OlapTable table = new OlapTable(tableId, "t1", columns, KeysType.AGG_KEYS, partitionInfo, distributionInfo); @@ -574,9 +578,9 @@ public void testRecycleTable(@Mocked GlobalStateMgr globalStateMgr, @Mocked Edit @Test public void testRecyclePartition(@Mocked GlobalStateMgr globalStateMgr, @Mocked EditLog editLog) { - Partition p1 = new Partition(111, "uno", null, null); - Partition p2SameName = new Partition(22, "dos", null, null); - Partition p2 = new Partition(222, "dos", null, null); + Partition p1 = new Partition(111, 112, "uno", null, null); + Partition p2SameName = new Partition(22, 221, "dos", null, null); + Partition p2 = new Partition(222, 223, "dos", null, null); new Expectations() { { diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/DatabaseTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/DatabaseTest.java index abde7c6ad8cc33..e1d35aba242ccc 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/DatabaseTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/DatabaseTest.java @@ -110,7 +110,8 @@ public void createAndDropPartitionTest() { Assert.assertEquals(dbId, db.getId()); MaterializedIndex baseIndex = new MaterializedIndex(10001, IndexState.NORMAL); - Partition partition = new Partition(20000L, "baseTable", baseIndex, new RandomDistributionInfo(10)); + Partition partition = new Partition(20000L, 20001L, + "baseTable", baseIndex, new RandomDistributionInfo(10)); List baseSchema = new LinkedList(); OlapTable table = new OlapTable(2000, "baseTable", baseSchema, KeysType.AGG_KEYS, new SinglePartitionInfo(), new RandomDistributionInfo(10)); diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/FakeEditLog.java b/fe/fe-core/src/test/java/com/starrocks/catalog/FakeEditLog.java index a151b2d0668868..8d9c6dc625fb28 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/FakeEditLog.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/FakeEditLog.java @@ -36,6 +36,7 @@ import com.starrocks.alter.AlterJobV2; import com.starrocks.alter.BatchAlterJobPersistInfo; +import com.starrocks.backup.BackupJob; import com.starrocks.cluster.Cluster; import com.starrocks.persist.EditLog; import com.starrocks.persist.ModifyTablePropertyOperationLog; @@ -112,6 +113,10 @@ public void logAddReplica(ReplicaPersistInfo info) { } + @Mock + public void logBackupJob(BackupJob job) { + } + public TransactionState getTransaction(long transactionId) { return allTransactionState.get(transactionId); } diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/GlobalStateMgrTestUtil.java b/fe/fe-core/src/test/java/com/starrocks/catalog/GlobalStateMgrTestUtil.java index 121a809f199e1a..83e317fa637d15 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/GlobalStateMgrTestUtil.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/GlobalStateMgrTestUtil.java @@ -184,7 +184,7 @@ public static Database createSimpleDb(long dbId, long tableId, long partitionId, // index MaterializedIndex index = new MaterializedIndex(indexId, IndexState.NORMAL); - TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partitionId, indexId, 0, TStorageMedium.HDD); + TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partitionId + 100, indexId, 0, TStorageMedium.HDD); index.addTablet(tablet, tabletMeta); tablet.addReplica(replica1); @@ -193,7 +193,7 @@ public static Database createSimpleDb(long dbId, long tableId, long partitionId, // partition RandomDistributionInfo distributionInfo = new RandomDistributionInfo(10); - Partition partition = new Partition(partitionId, testPartition1, index, distributionInfo); + Partition partition = new Partition(partitionId, partitionId + 100, testPartition1, index, distributionInfo); partition.getDefaultPhysicalPartition().updateVisibleVersion(testStartVersion); partition.getDefaultPhysicalPartition().setNextVersion(testStartVersion + 1); diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/ListPartitionInfoTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/ListPartitionInfoTest.java index aec1b3c59422ad..5a230a64a30c8c 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/ListPartitionInfoTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/ListPartitionInfoTest.java @@ -138,7 +138,7 @@ public void testMultiListPartition(@Injectable OlapTable dstTable) throws UserEx MaterializedIndex index = new MaterializedIndex(1, MaterializedIndex.IndexState.NORMAL); HashDistributionInfo distInfo = new HashDistributionInfo( 3, Lists.newArrayList(new Column("id", Type.BIGINT))); - Partition partition = new Partition(1, "p1", index, distInfo); + Partition partition = new Partition(1, 11, "p1", index, distInfo); Map idToColumn = Maps.newTreeMap(ColumnId.CASE_INSENSITIVE_ORDER); idToColumn.put(ColumnId.create("dt"), new Column("dt", Type.STRING)); @@ -207,8 +207,8 @@ public OlapTable findTableForSingleListPartition() { HashDistributionInfo distributionInfo = new HashDistributionInfo(1, Lists.newArrayList(new Column("id", Type.BIGINT))); - Partition p1 = new Partition(10001L, "p1", materializedIndex, distributionInfo); - Partition p2 = new Partition(10002L, "p2", materializedIndex, distributionInfo); + Partition p1 = new Partition(10001L, 10003L, "p1", materializedIndex, distributionInfo); + Partition p2 = new Partition(10002L, 10004L, "p2", materializedIndex, distributionInfo); table.addPartition(p1); table.addPartition(p2); return table; @@ -233,8 +233,8 @@ public OlapTable findTableForMultiListPartition() { HashDistributionInfo distributionInfo = new HashDistributionInfo(1, Lists.newArrayList(new Column("id", Type.BIGINT))); - Partition p1 = new Partition(10001L, "p1", materializedIndex, distributionInfo); - Partition p2 = new Partition(10002L, "p2", materializedIndex, distributionInfo); + Partition p1 = new Partition(10001L, 10003L, "p1", materializedIndex, distributionInfo); + Partition p2 = new Partition(10002L, 10004L, "p2", materializedIndex, distributionInfo); table.addPartition(p1); table.addPartition(p2); return table; diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/MaterializedViewTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/MaterializedViewTest.java index d7e5ff769a4472..1487eb9d8f78b0 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/MaterializedViewTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/MaterializedViewTest.java @@ -169,7 +169,7 @@ public void testPartition() { Assert.assertTrue(pInfo1 instanceof SinglePartitionInfo); MaterializedIndex index = new MaterializedIndex(3, IndexState.NORMAL); - Partition partition = new Partition(2, "mv_name", index, distributionInfo); + Partition partition = new Partition(2, 22, "mv_name", index, distributionInfo); mv.addPartition(partition); Partition tmpPartition = mv.getPartition("mv_name"); Assert.assertTrue(tmpPartition != null); @@ -194,7 +194,7 @@ public void testPartition() { Assert.assertEquals("new_name_2", mv2.getName()); PartitionInfo pInfo2 = mv2.getPartitionInfo(); Assert.assertTrue(pInfo2 instanceof RangePartitionInfo); - Partition partition2 = new Partition(3, "p1", index, distributionInfo); + Partition partition2 = new Partition(3, 33, "p1", index, distributionInfo); mv2.addPartition(partition2); Partition tmpPartition2 = mv2.getPartition("p1"); Assert.assertTrue(tmpPartition2 != null); diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/OlapTableTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/OlapTableTest.java index 9deb7f8b2ac3c6..6df687d227d2c3 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/OlapTableTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/OlapTableTest.java @@ -167,7 +167,7 @@ public void testMVPartitionDurationTimeUintMismatch1() throws AnalysisException olapTable.setTableProperty(new TableProperty(new HashMap<>())); olapTable.setDataCachePartitionDuration(TimeUtils.parseHumanReadablePeriodOrDuration("25 hour")); - Partition partition = new Partition(1, "p1", null, null); + Partition partition = new Partition(1, 11, "p1", null, null); Assert.assertTrue(olapTable.isEnableFillDataCache(partition)); new MockUp>() { @@ -204,7 +204,7 @@ public void testMVPartitionDurationTimeUintMismatch2() throws AnalysisException olapTable.setTableProperty(new TableProperty(new HashMap<>())); olapTable.setDataCachePartitionDuration(TimeUtils.parseHumanReadablePeriodOrDuration("25 hour")); - Partition partition = new Partition(1, "p1", null, null); + Partition partition = new Partition(1, 11, "p1", null, null); Assert.assertFalse(olapTable.isEnableFillDataCache(partition)); } @@ -232,7 +232,7 @@ public void testListPartitionSupportPeriodDurationTestDateColumn() throws Analys listPartitionInfo.setMultiValues(1L, multiValuesList1); OlapTable olapTable = new OlapTable(1L, "tb1", partitionColumns, null, (PartitionInfo) listPartitionInfo, null); olapTable.setTableProperty(new TableProperty(new HashMap<>())); - Partition partition1 = new Partition(1L, "p1", null, null); + Partition partition1 = new Partition(1L, 11, "p1", null, null); // Datacache.partition_duration is not set, cache is valid Assert.assertTrue(olapTable.isEnableFillDataCache(partition1)); @@ -245,7 +245,7 @@ public void testListPartitionSupportPeriodDurationTestDateColumn() throws Analys List> multiValuesList2 = new ArrayList<>(Arrays.asList(multiValues2)); listPartitionInfo.setMultiValues(2L, multiValuesList2); olapTable.setDataCachePartitionDuration(TimeUtils.parseHumanReadablePeriodOrDuration("28 hour")); - Partition partition2 = new Partition(2L, "p2", null, null); + Partition partition2 = new Partition(2L, 21, "p2", null, null); // cache is valid Assert.assertTrue(olapTable.isEnableFillDataCache(partition2)); @@ -271,7 +271,7 @@ public void testListPartitionSupportPeriodDurationTestSingleDateColumn() throws listPartitionInfo.setMultiValues(1L, multiValuesList1); OlapTable olapTable = new OlapTable(1L, "tb1", partitionColumns, null, (PartitionInfo) listPartitionInfo, null); olapTable.setTableProperty(new TableProperty(new HashMap<>())); - Partition partition1 = new Partition(1L, "p1", null, null); + Partition partition1 = new Partition(1L, 11, "p1", null, null); // Datacache.partition_duration is not set, cache is valid Assert.assertTrue(olapTable.isEnableFillDataCache(partition1)); @@ -284,7 +284,7 @@ public void testListPartitionSupportPeriodDurationTestSingleDateColumn() throws List> multiValuesList2 = new ArrayList<>(Arrays.asList(multiValues2)); listPartitionInfo.setMultiValues(2L, multiValuesList2); olapTable.setDataCachePartitionDuration(TimeUtils.parseHumanReadablePeriodOrDuration("28 hour")); - Partition partition2 = new Partition(2L, "p2", null, null); + Partition partition2 = new Partition(2L, 21, "p2", null, null); // cache is valid Assert.assertTrue(olapTable.isEnableFillDataCache(partition2)); @@ -309,7 +309,7 @@ public void testListPartitionSupportPeriodDurationTestIdToValues() throws Analys listPartitionInfo.setValues(1L, values1); OlapTable olapTable = new OlapTable(1L, "tb1", partitionColumns, null, (PartitionInfo) listPartitionInfo, null); olapTable.setTableProperty(new TableProperty(new HashMap<>())); - Partition partition1 = new Partition(1L, "p1", null, null); + Partition partition1 = new Partition(1L, 11, "p1", null, null); // Datacache.partition_duration is not set, cache is valid Assert.assertTrue(olapTable.isEnableFillDataCache(partition1)); @@ -321,7 +321,7 @@ public void testListPartitionSupportPeriodDurationTestIdToValues() throws Analys List values2 = new ArrayList<>(Arrays.asList(LocalDate.now().toString())); listPartitionInfo.setValues(2L, values2); olapTable.setDataCachePartitionDuration(TimeUtils.parseHumanReadablePeriodOrDuration("28 hour")); - Partition partition2 = new Partition(2L, "p2", null, null); + Partition partition2 = new Partition(2L, 21, "p2", null, null); // cache is valid Assert.assertTrue(olapTable.isEnableFillDataCache(partition2)); @@ -351,7 +351,7 @@ public void testListPartitionSupportPeriodDurationTestDateTimeColumn() throws An listPartitionInfo.setMultiValues(1L, multiValuesList1); OlapTable olapTable = new OlapTable(1L, "tb1", partitionColumns, null, (PartitionInfo) listPartitionInfo, null); olapTable.setTableProperty(new TableProperty(new HashMap<>())); - Partition partition = new Partition(1L, "p1", null, null); + Partition partition = new Partition(1L, 11, "p1", null, null); // Datacache.partition_duration is not set, cache is valid Assert.assertTrue(olapTable.isEnableFillDataCache(partition)); @@ -365,7 +365,7 @@ public void testListPartitionSupportPeriodDurationTestDateTimeColumn() throws An List> multiValuesList2 = new ArrayList<>(Arrays.asList(multiValues2)); listPartitionInfo.setMultiValues(2L, multiValuesList2); olapTable.setDataCachePartitionDuration(TimeUtils.parseHumanReadablePeriodOrDuration("28 hour")); - Partition partition2 = new Partition(2L, "p2", null, null); + Partition partition2 = new Partition(2L, 21, "p2", null, null); // cache is valid Assert.assertTrue(olapTable.isEnableFillDataCache(partition2)); @@ -387,7 +387,7 @@ public void testListPartitionSupportPeriodDurationTestNoneDateTypeColumn() throw listPartitionInfo.setMultiValues(1L, multiValuesList1); OlapTable olapTable = new OlapTable(1L, "tb1", partitionColumns, null, (PartitionInfo) listPartitionInfo, null); olapTable.setTableProperty(new TableProperty(new HashMap<>())); - Partition partition1 = new Partition(1L, "p1", null, null); + Partition partition1 = new Partition(1L, 11, "p1", null, null); olapTable.setDataCachePartitionDuration(TimeUtils.parseHumanReadablePeriodOrDuration("25 hour")); // cache is valid diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/PhysicalPartitionImplTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/PhysicalPartitionImplTest.java index 9ba302b21271be..41f15cdc4538f4 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/PhysicalPartitionImplTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/PhysicalPartitionImplTest.java @@ -105,7 +105,7 @@ public void testPhysicalPartition() throws Exception { Assert.assertTrue(p.visualiseShadowIndex(3, true)); Assert.assertTrue(p.equals(p)); - Assert.assertFalse(p.equals(new Partition(0, "", null, null))); + Assert.assertFalse(p.equals(new Partition(0, 11, "", null, null))); PhysicalPartition p2 = new PhysicalPartition(1, "", 1, 0, new MaterializedIndex()); Assert.assertFalse(p.equals(p2)); diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/ReplaceLakePartitionTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/ReplaceLakePartitionTest.java index 9804d07fd9e504..d6c0cd0e04a258 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/ReplaceLakePartitionTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/ReplaceLakePartitionTest.java @@ -18,16 +18,6 @@ import com.staros.client.StarClientException; import com.staros.proto.FilePathInfo; import com.staros.proto.ShardInfo; -import com.starrocks.catalog.Column; -import com.starrocks.catalog.KeysType; -import com.starrocks.catalog.MaterializedIndex; -import com.starrocks.catalog.Partition; -import com.starrocks.catalog.PartitionInfo; -import com.starrocks.catalog.PartitionKey; -import com.starrocks.catalog.PartitionType; -import com.starrocks.catalog.TabletInvertedIndex; -import com.starrocks.catalog.TabletMeta; -import com.starrocks.catalog.Type; import com.starrocks.common.ExceptionChecker; import com.starrocks.lake.DataCacheInfo; import com.starrocks.lake.LakeTable; @@ -88,8 +78,8 @@ LakeTable buildLakeTableWithTempPartition(PartitionType partitionType) { invertedIndex.addTablet(id, tabletMeta); index.addTablet(new LakeTablet(id), tabletMeta); } - Partition partition = new Partition(partitionId, partitionName, index, null); - Partition tempPartition = new Partition(tempPartitionId, tempPartitionName, index, null); + Partition partition = new Partition(partitionId, partitionId + 100L, partitionName, index, null); + Partition tempPartition = new Partition(tempPartitionId, tempPartitionId + 100L, tempPartitionName, index, null); PartitionInfo partitionInfo = null; if (partitionType == PartitionType.UNPARTITIONED) { @@ -118,7 +108,7 @@ LakeTable buildLakeTableWithTempPartition(PartitionType partitionType) { table.addTempPartition(tempPartition); return table; } - + Partition buildPartitionForTruncateTable() { MaterializedIndex index = new MaterializedIndex(indexId); TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); @@ -128,7 +118,7 @@ Partition buildPartitionForTruncateTable() { index.addTablet(new LakeTablet(id), tabletMeta); } - return new Partition(newPartitionId, partitionName, index, null); + return new Partition(newPartitionId, newPartitionId + 100L, partitionName, index, null); } private void erasePartitionOrTableAndUntilFinished(long id) { @@ -177,7 +167,7 @@ public Warehouse getBackgroundWarehouse() { while (GlobalStateMgr.getCurrentState().getRecycleBin().getRecyclePartitionInfo(id) != null) { ExceptionChecker.expectThrowsNoException(() - -> GlobalStateMgr.getCurrentState().getRecycleBin().erasePartition(Long.MAX_VALUE)); + -> GlobalStateMgr.getCurrentState().getRecycleBin().erasePartition(Long.MAX_VALUE)); try { Thread.sleep(100); } catch (Exception ignore) { @@ -186,7 +176,7 @@ public Warehouse getBackgroundWarehouse() { while (GlobalStateMgr.getCurrentState().getRecycleBin().getRecycleTableInfo(id) != null) { ExceptionChecker.expectThrowsNoException(() - -> GlobalStateMgr.getCurrentState().getRecycleBin().eraseTable(Long.MAX_VALUE)); + -> GlobalStateMgr.getCurrentState().getRecycleBin().eraseTable(Long.MAX_VALUE)); try { Thread.sleep(100); } catch (Exception ignore) { diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/TabletStatMgrTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/TabletStatMgrTest.java index 08bb5ae5b941fd..b68c33292864c1 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/TabletStatMgrTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/TabletStatMgrTest.java @@ -62,6 +62,7 @@ public class TabletStatMgrTest { private static final long TABLE_ID = 2; private static final long PARTITION_ID = 3; private static final long INDEX_ID = 4; + private static final long PH_PARTITION_ID = 5; @Before public void before() { @@ -136,7 +137,7 @@ public void testUpdateLocalTabletStat(@Mocked GlobalStateMgr globalStateMgr, @Mo // Table MaterializedIndex index = new MaterializedIndex(INDEX_ID, MaterializedIndex.IndexState.NORMAL); - Partition partition = new Partition(PARTITION_ID, "p1", index, distributionInfo); + Partition partition = new Partition(PARTITION_ID, PH_PARTITION_ID, "p1", index, distributionInfo); OlapTable table = new OlapTable(TABLE_ID, "t1", columns, KeysType.AGG_KEYS, partitionInfo, distributionInfo); Deencapsulation.setField(table, "baseIndexId", INDEX_ID); table.addPartition(partition); @@ -199,7 +200,7 @@ private LakeTable createLakeTableForTest() { DistributionInfo distributionInfo = new HashDistributionInfo(10, Lists.newArrayList(k1)); PartitionInfo partitionInfo = new SinglePartitionInfo(); partitionInfo.setReplicationNum(PARTITION_ID, (short) 3); - Partition partition = new Partition(PARTITION_ID, "p1", index, distributionInfo); + Partition partition = new Partition(PARTITION_ID, PH_PARTITION_ID, "p1", index, distributionInfo); partition.getDefaultPhysicalPartition().setVisibleVersion(2L, visibleVersionTime); // Lake table diff --git a/fe/fe-core/src/test/java/com/starrocks/clone/DiskAndTabletLoadReBalancerTest.java b/fe/fe-core/src/test/java/com/starrocks/clone/DiskAndTabletLoadReBalancerTest.java index 43cf68a7641493..d2407592b380c1 100644 --- a/fe/fe-core/src/test/java/com/starrocks/clone/DiskAndTabletLoadReBalancerTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/clone/DiskAndTabletLoadReBalancerTest.java @@ -73,6 +73,7 @@ public void testBalance(@Mocked GlobalStateMgr globalStateMgr) { long tableId = 10002L; long partitionId = 10003L; long indexId = 10004L; + long physicalPartitionId = 10005L; long tabletDataSize = 200 * 1024 * 1024L; TStorageMedium medium = TStorageMedium.HDD; long beId1 = 1L; @@ -96,22 +97,22 @@ public void testBalance(@Mocked GlobalStateMgr globalStateMgr) { // tablet inverted index TabletInvertedIndex invertedIndex = new TabletInvertedIndex(); MaterializedIndex materializedIndex = new MaterializedIndex(indexId, IndexState.NORMAL); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20001L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20001L, 30001L, beId1, tabletDataSize, pathHash1); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20002L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20002L, 30002L, beId1, tabletDataSize, pathHash1); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20003L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20003L, 30003L, beId1, tabletDataSize, pathHash1); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20004L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20004L, 30004L, beId2, tabletDataSize, pathHash2); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20005L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20005L, 30005L, beId2, tabletDataSize, pathHash2); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20006L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20006L, 30006L, beId2, tabletDataSize, pathHash2); @@ -122,14 +123,14 @@ public void testBalance(@Mocked GlobalStateMgr globalStateMgr) { DataProperty dataProperty = new DataProperty(medium); partitionInfo.addPartition(partitionId, dataProperty, (short) 1, false); DistributionInfo distributionInfo = new HashDistributionInfo(6, Lists.newArrayList()); - Partition partition = new Partition(partitionId, "partition", materializedIndex, distributionInfo); + Partition partition = new Partition(partitionId, physicalPartitionId, "partition", materializedIndex, distributionInfo); OlapTable table = new OlapTable(tableId, "table", Lists.newArrayList(), KeysType.AGG_KEYS, partitionInfo, distributionInfo); table.addPartition(partition); Database database = new Database(dbId, "database"); database.registerTableUnlocked(table); - PhysicalPartition physicalPartition = new PhysicalPartition(partitionId, "partition", partitionId, + PhysicalPartition physicalPartition = new PhysicalPartition(physicalPartitionId, "partition", partitionId, 0, materializedIndex); new Expectations() { @@ -225,6 +226,8 @@ public void testBalanceWithSameHost(@Mocked GlobalStateMgr globalStateMgr) { long tableId = 10002L; long partitionId = 10003L; long indexId = 10004L; + long physicalPartitionId = 10005L; + long tabletDataSize = 200 * 1024 * 1024L; TStorageMedium medium = TStorageMedium.HDD; long beId1 = 1L; @@ -253,31 +256,31 @@ public void testBalanceWithSameHost(@Mocked GlobalStateMgr globalStateMgr) { // tablet inverted index TabletInvertedIndex invertedIndex = new TabletInvertedIndex(); MaterializedIndex materializedIndex = new MaterializedIndex(indexId, IndexState.NORMAL); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20001L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20001L, 30001L, beId1, tabletDataSize, pathHash1); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20002L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20002L, 30002L, beId1, tabletDataSize, pathHash1); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20003L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20003L, 30003L, beId1, tabletDataSize, pathHash1); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20001L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20001L, 30004L, beId2, tabletDataSize, pathHash2); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20002L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20002L, 30005L, beId2, tabletDataSize, pathHash2); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20003L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20003L, 30006L, beId2, tabletDataSize, pathHash2); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20001L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20001L, 30007L, beId3, tabletDataSize, pathHash3); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20002L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20002L, 30008L, beId3, tabletDataSize, pathHash3); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId, indexId, 20003L, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId, indexId, 20003L, 30009L, beId3, tabletDataSize, pathHash3); @@ -288,14 +291,14 @@ public void testBalanceWithSameHost(@Mocked GlobalStateMgr globalStateMgr) { DataProperty dataProperty = new DataProperty(medium); partitionInfo.addPartition(partitionId, dataProperty, (short) 3, false); DistributionInfo distributionInfo = new HashDistributionInfo(3, Lists.newArrayList()); - Partition partition = new Partition(partitionId, "partition", materializedIndex, distributionInfo); + Partition partition = new Partition(partitionId, physicalPartitionId, "partition", materializedIndex, distributionInfo); OlapTable table = new OlapTable(tableId, "table", Lists.newArrayList(), KeysType.AGG_KEYS, partitionInfo, distributionInfo); table.addPartition(partition); Database database = new Database(dbId, "database"); database.registerTableUnlocked(table); - PhysicalPartition physicalPartition = new PhysicalPartition(partitionId, "partition", partitionId, + PhysicalPartition physicalPartition = new PhysicalPartition(physicalPartitionId, "partition", partitionId, 0, materializedIndex); new Expectations() { @@ -392,6 +395,9 @@ public void testBalanceBackendTablet(@Mocked GlobalStateMgr globalStateMgr) { long partitionId1 = 10010L; long partitionId2 = 10011L; long indexId = 10003L; + long physicalPartitionId1 = 10021L; + long physicalPartitionId2 = 10022L; + long tabletDataSize = 200 * 1024 * 1024L; long beId1 = 1L; long beId2 = 2L; @@ -441,26 +447,26 @@ public void testBalanceBackendTablet(@Mocked GlobalStateMgr globalStateMgr) { // tablet inverted index TabletInvertedIndex invertedIndex = new TabletInvertedIndex(); MaterializedIndex materializedIndex = new MaterializedIndex(indexId, IndexState.NORMAL); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId1, indexId, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId1, indexId, 20001L, 30001L, beId1, tabletDataSize, pathHash10); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId1, indexId, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId1, indexId, 20002L, 30002L, beId1, tabletDataSize, pathHash10); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId1, indexId, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId1, indexId, 20003L, 30003L, beId1, tabletDataSize, pathHash11); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId1, indexId, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId1, indexId, 20004L, 30004L, beId2, tabletDataSize, pathHash20); - addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, partitionId1, indexId, + addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, physicalPartitionId1, indexId, 20005L, 30005L, beId2, tabletDataSize, pathHash20); - addTablet(invertedIndex, materializedIndex, TStorageMedium.SSD, dbId, tableId, partitionId2, indexId, + addTablet(invertedIndex, materializedIndex, TStorageMedium.SSD, dbId, tableId, physicalPartitionId2, indexId, 20006L, 30006L, beId1, tabletDataSize, pathHash13); - addTablet(invertedIndex, materializedIndex, TStorageMedium.SSD, dbId, tableId, partitionId2, indexId, + addTablet(invertedIndex, materializedIndex, TStorageMedium.SSD, dbId, tableId, physicalPartitionId2, indexId, 20007L, 30007L, beId1, tabletDataSize, pathHash13); - addTablet(invertedIndex, materializedIndex, TStorageMedium.SSD, dbId, tableId, partitionId2, indexId, + addTablet(invertedIndex, materializedIndex, TStorageMedium.SSD, dbId, tableId, physicalPartitionId2, indexId, 20008L, 30008L, beId2, tabletDataSize, pathHash21); - addTablet(invertedIndex, materializedIndex, TStorageMedium.SSD, dbId, tableId, partitionId2, indexId, + addTablet(invertedIndex, materializedIndex, TStorageMedium.SSD, dbId, tableId, physicalPartitionId2, indexId, 20009L, 30009L, beId2, tabletDataSize, pathHash21); - addTablet(invertedIndex, materializedIndex, TStorageMedium.SSD, dbId, tableId, partitionId2, indexId, + addTablet(invertedIndex, materializedIndex, TStorageMedium.SSD, dbId, tableId, physicalPartitionId2, indexId, 20010L, 30010L, beId2, tabletDataSize, pathHash21); ClusterLoadStatistic clusterLoadStatistic = new ClusterLoadStatistic(infoService, invertedIndex); @@ -472,8 +478,10 @@ public void testBalanceBackendTablet(@Mocked GlobalStateMgr globalStateMgr) { DataProperty dataProperty2 = new DataProperty(TStorageMedium.SSD); partitionInfo.addPartition(partitionId2, dataProperty2, (short) 1, false); DistributionInfo distributionInfo = new HashDistributionInfo(6, Lists.newArrayList()); - Partition partition1 = new Partition(partitionId1, "partition1", materializedIndex, distributionInfo); - Partition partition2 = new Partition(partitionId2, "partition2", materializedIndex, distributionInfo); + Partition partition1 = new Partition(partitionId1, physicalPartitionId1, + "partition1", materializedIndex, distributionInfo); + Partition partition2 = new Partition(partitionId2, physicalPartitionId2, + "partition2", materializedIndex, distributionInfo); OlapTable table = new OlapTable(tableId, "table", Lists.newArrayList(), KeysType.AGG_KEYS, partitionInfo, distributionInfo); table.addPartition(partition1); @@ -481,9 +489,9 @@ public void testBalanceBackendTablet(@Mocked GlobalStateMgr globalStateMgr) { Database database = new Database(dbId, "database"); database.registerTableUnlocked(table); - PhysicalPartition physicalPartition1 = new PhysicalPartition(partitionId1, "partition1", partitionId1, + PhysicalPartition physicalPartition1 = new PhysicalPartition(physicalPartitionId1, "partition1", partitionId1, 0, materializedIndex); - PhysicalPartition physicalPartition2 = new PhysicalPartition(partitionId2, "partition2", partitionId2, + PhysicalPartition physicalPartition2 = new PhysicalPartition(physicalPartitionId2, "partition2", partitionId2, 0, materializedIndex); new Expectations() { @@ -509,12 +517,12 @@ public void testBalanceBackendTablet(@Mocked GlobalStateMgr globalStateMgr) { minTimes = 0; GlobalStateMgr.getCurrentState().getLocalMetastore() - .getPhysicalPartitionIncludeRecycleBin((OlapTable) any, partitionId1); + .getPhysicalPartitionIncludeRecycleBin((OlapTable) any, physicalPartitionId1); result = physicalPartition1; minTimes = 0; GlobalStateMgr.getCurrentState().getLocalMetastore() - .getPhysicalPartitionIncludeRecycleBin((OlapTable) any, partitionId2); + .getPhysicalPartitionIncludeRecycleBin((OlapTable) any, physicalPartitionId2); result = physicalPartition2; minTimes = 0; @@ -609,9 +617,9 @@ private DiskInfo genDiskInfo(long availableCapB, long dataUsedCapB, long totalCa private void addTablet(TabletInvertedIndex invertedIndex, MaterializedIndex materializedIndex, TStorageMedium medium, - long dbId, long tableId, long partitionId, long indexId, long tabletId, long replicaId, + long dbId, long tableId, long physicalPartitionId, long indexId, long tabletId, long replicaId, long beId, long dataSize, long pathHash) { - TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partitionId, indexId, 1111, medium); + TabletMeta tabletMeta = new TabletMeta(dbId, tableId, physicalPartitionId, indexId, 1111, medium); Replica replica = new Replica(replicaId, beId, 1L, 1111, dataSize, 1000, ReplicaState.NORMAL, -1, 1); invertedIndex.addTablet(tabletId, tabletMeta); @@ -628,6 +636,8 @@ public void testBalanceParallel(@Mocked GlobalStateMgr globalStateMgr) { long tableId = 10002L; long partitionId = 10003L; long indexId = 10004L; + long physicalPartitionId = 10005L; + long tabletDataSize = 200 * 1024 * 1024L; TStorageMedium medium = TStorageMedium.HDD; long beId1 = 1L; @@ -659,9 +669,9 @@ public void testBalanceParallel(@Mocked GlobalStateMgr globalStateMgr) { MaterializedIndex materializedIndex = new MaterializedIndex(indexId, IndexState.NORMAL); for (int i = 1; i <= highTabletCnt; i++) { addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, - partitionId, indexId, 20000 + i, 30000 + i, beId1, tabletDataSize, pathHash1); + physicalPartitionId, indexId, 20000 + i, 30000 + i, beId1, tabletDataSize, pathHash1); addTablet(invertedIndex, materializedIndex, TStorageMedium.HDD, dbId, tableId, - partitionId, indexId, 20000 + highTabletCnt + i, 30000 + highTabletCnt + i, + physicalPartitionId, indexId, 20000 + highTabletCnt + i, 30000 + highTabletCnt + i, beId2, tabletDataSize, pathHash2); } @@ -672,14 +682,14 @@ public void testBalanceParallel(@Mocked GlobalStateMgr globalStateMgr) { DataProperty dataProperty = new DataProperty(medium); partitionInfo.addPartition(partitionId, dataProperty, (short) 1, false); DistributionInfo distributionInfo = new HashDistributionInfo(6, Lists.newArrayList()); - Partition partition = new Partition(partitionId, "partition", materializedIndex, distributionInfo); + Partition partition = new Partition(partitionId, physicalPartitionId, "partition", materializedIndex, distributionInfo); OlapTable table = new OlapTable(tableId, "table", Lists.newArrayList(), KeysType.AGG_KEYS, partitionInfo, distributionInfo); table.addPartition(partition); Database database = new Database(dbId, "database"); database.registerTableUnlocked(table); - PhysicalPartition physicalPartition = new PhysicalPartition(partitionId, "partition", partitionId, + PhysicalPartition physicalPartition = new PhysicalPartition(physicalPartitionId, "partition", partitionId, 0, materializedIndex); new Expectations() { diff --git a/fe/fe-core/src/test/java/com/starrocks/clone/TabletSchedCtxTest.java b/fe/fe-core/src/test/java/com/starrocks/clone/TabletSchedCtxTest.java index cd61c789edd82b..0840704acc1c10 100644 --- a/fe/fe-core/src/test/java/com/starrocks/clone/TabletSchedCtxTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/clone/TabletSchedCtxTest.java @@ -64,6 +64,7 @@ public class TabletSchedCtxTest { private static int PART_ID = 3; private static int INDEX_ID = 4; private static int SCHEMA_HASH = 5; + private static int PH_PART_ID = 6; private static String TB_NAME = "test"; private static List TB_BASE_SCHEMA = Lists.newArrayList(new Column("k1", ScalarType @@ -118,7 +119,7 @@ public void setUp() { // mock catalog MaterializedIndex baseIndex = new MaterializedIndex(TB_ID, MaterializedIndex.IndexState.NORMAL); DistributionInfo distributionInfo = new RandomDistributionInfo(32); - Partition partition = new Partition(PART_ID, TB_NAME, baseIndex, distributionInfo); + Partition partition = new Partition(PART_ID, PH_PART_ID, TB_NAME, baseIndex, distributionInfo); baseIndex.addTablet(tablet, tabletMeta); PartitionInfo partitionInfo = new SinglePartitionInfo(); partitionInfo.setReplicationNum(PART_ID, (short) 3); @@ -162,9 +163,8 @@ public void testSingleReplicaRecover() throws SchedException { LocalTablet missedTablet = new LocalTablet(TABLET_ID_1, GlobalStateMgr.getCurrentState().getTabletInvertedIndex().getReplicasByTabletId(TABLET_ID_1)); - TabletSchedCtx ctx = - new TabletSchedCtx(Type.REPAIR, DB_ID, TB_ID, PART_ID, INDEX_ID, - TABLET_ID_1, System.currentTimeMillis(), GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo()); + TabletSchedCtx ctx = new TabletSchedCtx(Type.REPAIR, DB_ID, TB_ID, PH_PART_ID, INDEX_ID, + TABLET_ID_1, System.currentTimeMillis(), GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo()); ctx.setTablet(missedTablet); ctx.setStorageMedium(TStorageMedium.HDD); @@ -234,7 +234,7 @@ public void testPriorityCompare() { } @Test - public void testChooseDestReplicaForVersionIncomplete() { + public void testChooseDestReplicaForVersionIncomplete() { TabletMeta tabletMeta = new TabletMeta(DB_ID, TB_ID, PART_ID, INDEX_ID, SCHEMA_HASH, TStorageMedium.HDD); GlobalStateMgr.getCurrentState().getTabletInvertedIndex().addTablet(TABLET_ID_2, tabletMeta); Replica replica1 = new Replica(50011, be1.getId(), 0, Replica.ReplicaState.NORMAL); @@ -274,7 +274,7 @@ public void testChooseDestReplicaForVersionIncomplete() { Assert.assertTrue(false); } Assert.assertEquals(be2.getId(), ctx.getDestBackendId()); - + replica2.updateVersionInfo(101, 120, 101); try { ctx.chooseDestReplicaForVersionIncomplete(backendsWorkingSlots); diff --git a/fe/fe-core/src/test/java/com/starrocks/clone/TabletSchedulerTest.java b/fe/fe-core/src/test/java/com/starrocks/clone/TabletSchedulerTest.java index 6c21f0e25551c4..288bfe2bf0a44a 100644 --- a/fe/fe-core/src/test/java/com/starrocks/clone/TabletSchedulerTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/clone/TabletSchedulerTest.java @@ -157,8 +157,8 @@ public void testSubmitBatchTaskIfNotExpired() { Database goodDB = new Database(2, "bueno"); Table badTable = new Table(3, "mal", Table.TableType.OLAP, new ArrayList<>()); Table goodTable = new Table(4, "bueno", Table.TableType.OLAP, new ArrayList<>()); - Partition badPartition = new Partition(5, "mal", null, null); - Partition goodPartition = new Partition(6, "bueno", null, null); + Partition badPartition = new Partition(5, 55, "mal", null, null); + Partition goodPartition = new Partition(6, 66, "bueno", null, null); long now = System.currentTimeMillis(); CatalogRecycleBin recycleBin = new CatalogRecycleBin(); @@ -180,7 +180,7 @@ public void testSubmitBatchTaskIfNotExpired() { TabletSchedCtx.Type.REPAIR, triple.getLeft().getId(), triple.getMiddle().getId(), - triple.getRight().getId(), + triple.getRight().getDefaultPhysicalPartition().getId(), 1, 1, System.currentTimeMillis(), @@ -209,7 +209,7 @@ public void testPendingAddTabletCtx() throws InterruptedException { TabletScheduler tabletScheduler = new TabletScheduler(tabletSchedulerStat); Database goodDB = new Database(2, "bueno"); Table goodTable = new Table(4, "bueno", Table.TableType.OLAP, new ArrayList<>()); - Partition goodPartition = new Partition(6, "bueno", null, null); + Partition goodPartition = new Partition(6, 66, "bueno", null, null); List tabletSchedCtxList = new ArrayList<>(); diff --git a/fe/fe-core/src/test/java/com/starrocks/common/proc/LakeTabletsProcNodeTest.java b/fe/fe-core/src/test/java/com/starrocks/common/proc/LakeTabletsProcNodeTest.java index 1e05a74576bbc1..9f1e9b9fa69416 100644 --- a/fe/fe-core/src/test/java/com/starrocks/common/proc/LakeTabletsProcNodeTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/common/proc/LakeTabletsProcNodeTest.java @@ -64,6 +64,7 @@ public void testFetchResult(@Mocked GlobalStateMgr globalStateMgr, @Mocked Wareh long tableId = 2L; long partitionId = 3L; long indexId = 4L; + long physicalPartitionId = 6L; long tablet1Id = 10L; long tablet2Id = 11L; @@ -101,7 +102,7 @@ public void testFetchResult(@Mocked GlobalStateMgr globalStateMgr, @Mocked Wareh DistributionInfo distributionInfo = new HashDistributionInfo(10, Lists.newArrayList(k1)); PartitionInfo partitionInfo = new SinglePartitionInfo(); partitionInfo.setReplicationNum(partitionId, (short) 3); - Partition partition = new Partition(partitionId, "p1", index, distributionInfo); + Partition partition = new Partition(partitionId, physicalPartitionId, "p1", index, distributionInfo); // Lake table LakeTable table = new LakeTable(tableId, "t1", columns, KeysType.AGG_KEYS, partitionInfo, distributionInfo); diff --git a/fe/fe-core/src/test/java/com/starrocks/common/proc/LocalTabletsProcDirTest.java b/fe/fe-core/src/test/java/com/starrocks/common/proc/LocalTabletsProcDirTest.java index b2e21739494bc0..3689c439870a7a 100644 --- a/fe/fe-core/src/test/java/com/starrocks/common/proc/LocalTabletsProcDirTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/common/proc/LocalTabletsProcDirTest.java @@ -100,6 +100,7 @@ public void testFetchResultWithLocalTablet(@Mocked GlobalStateMgr globalStateMgr long tablet1Id = 5L; long tablet2Id = 6L; long replicaId = 10L; + long physicalPartitionId = 11L; // Columns List columns = new ArrayList(); @@ -135,7 +136,7 @@ public void testFetchResultWithLocalTablet(@Mocked GlobalStateMgr globalStateMgr index.addTablet(tablet2, tabletMeta); // Partition - Partition partition = new Partition(partitionId, "p1", index, distributionInfo); + Partition partition = new Partition(partitionId, physicalPartitionId, "p1", index, distributionInfo); // Table OlapTable table = new OlapTable(tableId, "t1", columns, KeysType.AGG_KEYS, partitionInfo, distributionInfo); diff --git a/fe/fe-core/src/test/java/com/starrocks/common/proc/PartitionsProcDirTest.java b/fe/fe-core/src/test/java/com/starrocks/common/proc/PartitionsProcDirTest.java index 6a4480464e21f3..998151c43fe546 100644 --- a/fe/fe-core/src/test/java/com/starrocks/common/proc/PartitionsProcDirTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/common/proc/PartitionsProcDirTest.java @@ -56,7 +56,8 @@ public void setUp() throws DdlException, AnalysisException { listPartition.setDataCacheInfo(partitionId, dataCache); cloudNativTable = new LakeTable(1024L, "cloud_native_table", col, null, listPartition, null); MaterializedIndex index = new MaterializedIndex(1000L, IndexState.NORMAL); - cloudNativTable.addPartition(new Partition(partitionId, "p1", index, new RandomDistributionInfo(10))); + cloudNativTable.addPartition(new Partition(partitionId, 1035, + "p1", index, new RandomDistributionInfo(10))); db.registerTableUnlocked(cloudNativTable); } @@ -66,7 +67,7 @@ public void testFetchResult() throws AnalysisException { BaseProcResult result = (BaseProcResult) new PartitionsProcDir(db, cloudNativTable, false).fetchResult(); List> rows = result.getRows(); List list1 = rows.get(0); - Assert.assertEquals("1025", list1.get(0)); + Assert.assertEquals("1035", list1.get(0)); Assert.assertEquals("p1", list1.get(1)); Assert.assertEquals("0", list1.get(2)); Assert.assertEquals("1", list1.get(3)); diff --git a/fe/fe-core/src/test/java/com/starrocks/common/proc/TablesProcDirTest.java b/fe/fe-core/src/test/java/com/starrocks/common/proc/TablesProcDirTest.java index 6631919250343a..c1b13474657951 100644 --- a/fe/fe-core/src/test/java/com/starrocks/common/proc/TablesProcDirTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/common/proc/TablesProcDirTest.java @@ -52,7 +52,7 @@ public void setUp() throws DdlException, AnalysisException { Map indexNameToId = Maps.newHashMap(); indexNameToId.put("index1", 1000L); - List p1 = Lists.newArrayList(new Partition(1001L, "p", null, null)); + List p1 = Lists.newArrayList(new Partition(1001L, 1011L, "p", null, null)); List col1 = Lists.newArrayList(new Column("province", Type.VARCHAR)); PartitionInfo pt1 = new ListPartitionInfo(PartitionType.LIST, col1); OlapTable tb1 = new OlapTable(1000L, "tb1", col1, null, pt1, null); @@ -72,7 +72,7 @@ public void setUp() throws DdlException, AnalysisException { } }; - List p2 = Lists.newArrayList(new Partition(20001L, "p", null, null)); + List p2 = Lists.newArrayList(new Partition(20001L, 20011L, "p", null, null)); List col2 = Lists.newArrayList(new Column("dt", Type.DATE)); PartitionInfo pt2 = new RangePartitionInfo(col2); OlapTable tb2 = new OlapTable(2000L, "tb2", col2, null, pt2, null); diff --git a/fe/fe-core/src/test/java/com/starrocks/common/util/UnitTestUtil.java b/fe/fe-core/src/test/java/com/starrocks/common/util/UnitTestUtil.java index b1e1e955ab4fb5..1d60eb6cae6b2d 100644 --- a/fe/fe-core/src/test/java/com/starrocks/common/util/UnitTestUtil.java +++ b/fe/fe-core/src/test/java/com/starrocks/common/util/UnitTestUtil.java @@ -131,7 +131,7 @@ public static OlapTable createOlapTableByName(long dbId, long tableId, long part // partition RandomDistributionInfo distributionInfo = new RandomDistributionInfo(10); - Partition partition = new Partition(partitionId, PARTITION_NAME, index, distributionInfo); + Partition partition = new Partition(partitionId, partitionId + 100, PARTITION_NAME, index, distributionInfo); // columns List columns = new ArrayList(); @@ -208,7 +208,7 @@ public static OlapTable createOlapTableWithName(String tableName, long dbId, lon // partition RandomDistributionInfo distributionInfo = new RandomDistributionInfo(10); - Partition partition = new Partition(partitionId, PARTITION_NAME, index, distributionInfo); + Partition partition = new Partition(partitionId, partitionId + 100, PARTITION_NAME, index, distributionInfo); // columns List columns = new ArrayList(); diff --git a/fe/fe-core/src/test/java/com/starrocks/consistency/ConsistencyCheckerTest.java b/fe/fe-core/src/test/java/com/starrocks/consistency/ConsistencyCheckerTest.java index 0cc4030ad1ec9c..bbd65162aea2f9 100644 --- a/fe/fe-core/src/test/java/com/starrocks/consistency/ConsistencyCheckerTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/consistency/ConsistencyCheckerTest.java @@ -46,6 +46,7 @@ public void testChooseTablets(@Mocked GlobalStateMgr globalStateMgr) { long tabletId = 5L; long replicaId = 6L; long backendId = 7L; + long physicalPartitionId = 8L; TStorageMedium medium = TStorageMedium.HDD; MaterializedIndex materializedIndex = new MaterializedIndex(indexId, MaterializedIndex.IndexState.NORMAL); @@ -59,7 +60,7 @@ public void testChooseTablets(@Mocked GlobalStateMgr globalStateMgr) { DataProperty dataProperty = new DataProperty(medium); partitionInfo.addPartition(partitionId, dataProperty, (short) 3, false); DistributionInfo distributionInfo = new HashDistributionInfo(1, Lists.newArrayList()); - Partition partition = new Partition(partitionId, "partition", materializedIndex, distributionInfo); + Partition partition = new Partition(partitionId, physicalPartitionId, "partition", materializedIndex, distributionInfo); partition.getDefaultPhysicalPartition().setVisibleVersion(2L, System.currentTimeMillis()); OlapTable table = new OlapTable(tableId, "table", Lists.newArrayList(), KeysType.AGG_KEYS, partitionInfo, distributionInfo); diff --git a/fe/fe-core/src/test/java/com/starrocks/consistency/MetaRecoveryDaemonTest.java b/fe/fe-core/src/test/java/com/starrocks/consistency/MetaRecoveryDaemonTest.java index 396c4718790a39..cb2a67cbb88ee7 100644 --- a/fe/fe-core/src/test/java/com/starrocks/consistency/MetaRecoveryDaemonTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/consistency/MetaRecoveryDaemonTest.java @@ -53,16 +53,16 @@ public static void tearDown() throws Exception { public void testRecover() throws Exception { PseudoCluster cluster = PseudoCluster.getInstance(); String sql = "CREATE TABLE test.`tbl_recover` (\n" + - " k1 int,\n" + - " k2 VARCHAR NOT NULL\n" + - ") ENGINE=OLAP\n" + - "DUPLICATE KEY(`k1`)\n" + - "COMMENT \"OLAP\"\n" + - "DISTRIBUTED BY HASH(`k1`) BUCKETS 8\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"3\",\n" + - " \"in_memory\" = \"false\"\n" + - ");"; + " k1 int,\n" + + " k2 VARCHAR NOT NULL\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(`k1`)\n" + + "COMMENT \"OLAP\"\n" + + "DISTRIBUTED BY HASH(`k1`) BUCKETS 8\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"3\",\n" + + " \"in_memory\" = \"false\"\n" + + ");"; cluster.runSql("test", sql); cluster.runSql("test", "insert into test.tbl_recover values (1, 'a'), (2, 'b')"); @@ -71,7 +71,7 @@ public void testRecover() throws Exception { Database database = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore() - .getTable(database.getFullName(), "tbl_recover"); + .getTable(database.getFullName(), "tbl_recover"); Partition partition = table.getPartition("tbl_recover"); MaterializedIndex index = partition.getDefaultPhysicalPartition() .getMaterializedIndices(MaterializedIndex.IndexExtState.ALL).get(0); @@ -90,19 +90,20 @@ public void testRecover() throws Exception { for (Backend backend : GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().getBackends()) { backend.getBackendStatus().lastSuccessReportTabletsTime = TimeUtils - .longToTimeString(System.currentTimeMillis()); + .longToTimeString(System.currentTimeMillis()); } // add a committed txn TransactionState transactionState = new TransactionState(database.getId(), Lists.newArrayList(table.getId()), - 11111, "xxxx", null, TransactionState.LoadJobSourceType.FRONTEND, null, 2222, 100000); + 11111, "xxxx", null, TransactionState.LoadJobSourceType.FRONTEND, null, 2222, 100000); TableCommitInfo tableCommitInfo = new TableCommitInfo(table.getId()); - PartitionCommitInfo partitionCommitInfo = new PartitionCommitInfo(partition.getId(), 4, -1L); + PartitionCommitInfo partitionCommitInfo = new PartitionCommitInfo(partition.getDefaultPhysicalPartition().getId(), + 4, -1L); tableCommitInfo.addPartitionCommitInfo(partitionCommitInfo); transactionState.putIdToTableCommitInfo(table.getId(), tableCommitInfo); transactionState.setTransactionStatus(TransactionStatus.COMMITTED); GlobalStateMgr.getCurrentState().getGlobalTransactionMgr() - .getDatabaseTransactionMgr(database.getId()).replayUpsertTransactionState(transactionState); + .getDatabaseTransactionMgr(database.getId()).replayUpsertTransactionState(transactionState); // recover will fail, because there is a committed txn on that partition MetaRecoveryDaemon recovery = new MetaRecoveryDaemon(); @@ -124,14 +125,14 @@ public void testRecover() throws Exception { // change replica version LocalTablet localTablet = (LocalTablet) partition.getDefaultPhysicalPartition() .getMaterializedIndices(MaterializedIndex.IndexExtState.ALL) - .get(0).getTablets().get(0); + .get(0).getTablets().get(0); long version = 3; for (Replica replica : localTablet.getAllReplicas()) { replica.updateForRestore(++version, 10, 10); } LocalTablet localTablet2 = (LocalTablet) partition.getDefaultPhysicalPartition() .getMaterializedIndices(MaterializedIndex.IndexExtState.ALL) - .get(0).getTablets().get(0); + .get(0).getTablets().get(0); for (Replica replica : localTablet2.getAllReplicas()) { replica.updateForRestore(4, 10, 10); } @@ -176,7 +177,7 @@ public void testCheckTabletReportCacheUp() { MetaRecoveryDaemon metaRecoveryDaemon = new MetaRecoveryDaemon(); for (Backend backend : GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().getBackends()) { backend.getBackendStatus().lastSuccessReportTabletsTime = TimeUtils - .longToTimeString(timeMs); + .longToTimeString(timeMs); } Assert.assertFalse(metaRecoveryDaemon.checkTabletReportCacheUp(timeMs + 1000L)); Assert.assertTrue(metaRecoveryDaemon.checkTabletReportCacheUp(timeMs - 1000L)); diff --git a/fe/fe-core/src/test/java/com/starrocks/http/StarRocksHttpTestCase.java b/fe/fe-core/src/test/java/com/starrocks/http/StarRocksHttpTestCase.java index 51e51744c38ff3..cba70fc2fcba5b 100644 --- a/fe/fe-core/src/test/java/com/starrocks/http/StarRocksHttpTestCase.java +++ b/fe/fe-core/src/test/java/com/starrocks/http/StarRocksHttpTestCase.java @@ -129,6 +129,7 @@ public abstract class StarRocksHttpTestCase { protected static long testDbId = 100L; protected static long testTableId = 200L; protected static long testPartitionId = 201L; + protected static long testPhysicalPartitionId = 202L; public static long testIndexId = testTableId; // the base indexid == tableid protected static long tabletId = 400L; @@ -171,7 +172,8 @@ public static OlapTable newEmptyTable(String name) { // partition HashDistributionInfo distributionInfo = new HashDistributionInfo(10, Lists.newArrayList(k1)); - Partition partition = new Partition(testPartitionId, "testPartition", baseIndex, distributionInfo); + Partition partition = new Partition(testPartitionId, testPhysicalPartitionId, + "testPartition", baseIndex, distributionInfo); partition.getDefaultPhysicalPartition().updateVisibleVersion(testStartVersion); partition.getDefaultPhysicalPartition().setNextVersion(testStartVersion + 1); @@ -228,7 +230,8 @@ public static OlapTable newTable(String name, long replicaDataSize) { // partition HashDistributionInfo distributionInfo = new HashDistributionInfo(10, Lists.newArrayList(k1)); - Partition partition = new Partition(testPartitionId, "testPartition", baseIndex, distributionInfo); + Partition partition = new Partition(testPartitionId, testPhysicalPartitionId, + "testPartition", baseIndex, distributionInfo); partition.getDefaultPhysicalPartition().updateVisibleVersion(testStartVersion); partition.getDefaultPhysicalPartition().setNextVersion(testStartVersion + 1); diff --git a/fe/fe-core/src/test/java/com/starrocks/http/rest/v2/TablePartitionActionTest.java b/fe/fe-core/src/test/java/com/starrocks/http/rest/v2/TablePartitionActionTest.java index 35ac7e2805fa6a..63b256c8cc70e6 100644 --- a/fe/fe-core/src/test/java/com/starrocks/http/rest/v2/TablePartitionActionTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/http/rest/v2/TablePartitionActionTest.java @@ -368,7 +368,9 @@ private static OlapTable newOlapTable(Long tableId, String tableName, int partit DistributionInfo distributionInfo = new HashDistributionInfo(8, Lists.newArrayList(c1)); long partitionId = BASE_PARTITION_ID + i; - Partition partition = new Partition(partitionId, "testPartition_" + i, baseIndex, distributionInfo); + long physicalPartitionId = partitionId + partitionSize; + Partition partition = new Partition(partitionId, physicalPartitionId, + "testPartition_" + i, baseIndex, distributionInfo); partition.getDefaultPhysicalPartition().setVisibleVersion(testStartVersion, System.currentTimeMillis()); partition.getDefaultPhysicalPartition().setNextVersion(testStartVersion + 1); @@ -458,7 +460,9 @@ private static LakeTable newLakeTable(Long tableId, String tableName, int partit DistributionInfo distributionInfo = new HashDistributionInfo(8, Lists.newArrayList(c1)); long partitionId = BASE_PARTITION_ID + i; - Partition partition = new Partition(partitionId, "testPartition_" + i, baseIndex, distributionInfo); + long physicalPartitionId = partitionId + partitionSize; + Partition partition = new Partition(partitionId, physicalPartitionId, + "testPartition_" + i, baseIndex, distributionInfo); partition.getDefaultPhysicalPartition().setVisibleVersion(testStartVersion, System.currentTimeMillis()); partition.getDefaultPhysicalPartition().setNextVersion(testStartVersion + 1); diff --git a/fe/fe-core/src/test/java/com/starrocks/lake/LakeMaterializedViewTest.java b/fe/fe-core/src/test/java/com/starrocks/lake/LakeMaterializedViewTest.java index e7425d1143dbf3..42ee013ee9bfae 100644 --- a/fe/fe-core/src/test/java/com/starrocks/lake/LakeMaterializedViewTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/lake/LakeMaterializedViewTest.java @@ -125,6 +125,7 @@ int getCurrentStateJournalVersion() { long mvId = 2L; long partitionId = 3L; long indexId = 4L; + long physicalPartitionId = 6L; long tablet1Id = 10L; long tablet2Id = 11L; @@ -149,7 +150,7 @@ int getCurrentStateJournalVersion() { DistributionInfo distributionInfo = new HashDistributionInfo(10, Lists.newArrayList(k1)); PartitionInfo partitionInfo = new SinglePartitionInfo(); partitionInfo.setReplicationNum(partitionId, (short) 3); - Partition partition = new Partition(partitionId, "p1", index, distributionInfo); + Partition partition = new Partition(partitionId, physicalPartitionId, "p1", index, distributionInfo); // refresh scheme MvRefreshScheme mvRefreshScheme = new MvRefreshScheme(); @@ -391,6 +392,9 @@ int getCurrentStateJournalVersion() { long indexId = 3L; long partition1Id = 20L; long partition2Id = 21L; + long physicalPartitionId1 = 22L; + long physicalPartitionId2 = 23L; + long tablet1Id = 10L; long tablet2Id = 11L; @@ -417,7 +421,7 @@ int getCurrentStateJournalVersion() { TabletMeta tabletMeta1 = new TabletMeta(dbId, mvId, partition1Id, indexId, 0, TStorageMedium.HDD, true); Tablet tablet1 = new LakeTablet(tablet1Id); index1.addTablet(tablet1, tabletMeta1); - Partition partition1 = new Partition(partition1Id, "p1", index1, distributionInfo); + Partition partition1 = new Partition(partition1Id, physicalPartitionId1, "p1", index1, distributionInfo); LocalDate upper1 = LocalDate.now().minus(duration); LocalDate lower1 = upper1.minus(duration); @@ -430,7 +434,7 @@ int getCurrentStateJournalVersion() { TabletMeta tabletMeta2 = new TabletMeta(dbId, mvId, partition2Id, indexId, 0, TStorageMedium.HDD, true); Tablet tablet2 = new LakeTablet(tablet2Id); index2.addTablet(tablet2, tabletMeta2); - Partition partition2 = new Partition(partition2Id, "p2", index1, distributionInfo); + Partition partition2 = new Partition(partition2Id, physicalPartitionId2, "p2", index1, distributionInfo); LocalDate upper2 = LocalDate.now(); LocalDate lower2 = upper2.minus(duration); diff --git a/fe/fe-core/src/test/java/com/starrocks/lake/StarMgrMetaSyncerTest.java b/fe/fe-core/src/test/java/com/starrocks/lake/StarMgrMetaSyncerTest.java index caa73068ad85fb..d47609a077b9e6 100644 --- a/fe/fe-core/src/test/java/com/starrocks/lake/StarMgrMetaSyncerTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/lake/StarMgrMetaSyncerTest.java @@ -106,7 +106,7 @@ public void setUp() throws Exception { long dbId = 1L; long tableId = 2L; long partitionId = 3L; - + long physicalPartitionId = 4L; new Expectations() { @@ -202,7 +202,8 @@ public List getTablesIncludeRecycleBin(Database db) { public Collection getAllPartitionsIncludeRecycleBin(OlapTable tbl) { MaterializedIndex baseIndex = new MaterializedIndex(); DistributionInfo distributionInfo = new HashDistributionInfo(); - return Lists.newArrayList(new Partition(partitionId, "p1", baseIndex, distributionInfo, shardGroupId)); + return Lists.newArrayList(new Partition(partitionId, physicalPartitionId, + "p1", baseIndex, distributionInfo, shardGroupId)); } }; diff --git a/fe/fe-core/src/test/java/com/starrocks/lake/delete/DeleteTest.java b/fe/fe-core/src/test/java/com/starrocks/lake/delete/DeleteTest.java index 289b4f01775bd3..8cde84cf33c64d 100644 --- a/fe/fe-core/src/test/java/com/starrocks/lake/delete/DeleteTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/lake/delete/DeleteTest.java @@ -84,6 +84,7 @@ public class DeleteTest { private final long tableId = 2L; private final long partitionId = 3L; private final long indexId = 4L; + private final long physicalPartitionId = 6L; private final long tablet1Id = 10L; private final long tablet2Id = 11L; private final long backendId = 20L; @@ -130,7 +131,7 @@ private Database createDb() { DistributionInfo distributionInfo = new HashDistributionInfo(10, Lists.newArrayList(k1)); PartitionInfo partitionInfo = new SinglePartitionInfo(); partitionInfo.setReplicationNum(partitionId, (short) 3); - Partition partition = new Partition(partitionId, partitionName, index, distributionInfo); + Partition partition = new Partition(partitionId, physicalPartitionId, partitionName, index, distributionInfo); // Lake table LakeTable table = new LakeTable(tableId, tableName, columns, KeysType.DUP_KEYS, partitionInfo, distributionInfo); diff --git a/fe/fe-core/src/test/java/com/starrocks/load/DeleteHandlerTest.java b/fe/fe-core/src/test/java/com/starrocks/load/DeleteHandlerTest.java index 58f0dcbd63ba5d..32c1d62b5b6aab 100644 --- a/fe/fe-core/src/test/java/com/starrocks/load/DeleteHandlerTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/load/DeleteHandlerTest.java @@ -87,6 +87,7 @@ public class DeleteHandlerTest { private static final long REPLICA_ID_3 = 70002L; private static final long TABLET_ID = 60000L; private static final long PARTITION_ID = 40000L; + private static final long PH_PARTITION_ID = 40011L; private static final long TBL_ID = 30000L; private static final long DB_ID = 20000L; @@ -120,7 +121,7 @@ public void setUp() { e.printStackTrace(); Assert.fail(); } - TabletMeta tabletMeta = new TabletMeta(DB_ID, TBL_ID, PARTITION_ID, TBL_ID, 0, null); + TabletMeta tabletMeta = new TabletMeta(DB_ID, TBL_ID, PH_PARTITION_ID, TBL_ID, 0, null); invertedIndex.addTablet(TABLET_ID, tabletMeta); invertedIndex.addReplica(TABLET_ID, new Replica(REPLICA_ID_1, BACKEND_ID_1, 0, Replica.ReplicaState.NORMAL)); invertedIndex.addReplica(TABLET_ID, new Replica(REPLICA_ID_2, BACKEND_ID_2, 0, Replica.ReplicaState.NORMAL)); @@ -256,7 +257,7 @@ public void testQuorumTimeout() throws DdlException, QueryStateException { Set finishedReplica = Sets.newHashSet(); finishedReplica.add(new Replica(REPLICA_ID_1, BACKEND_ID_1, 0, Replica.ReplicaState.NORMAL)); finishedReplica.add(new Replica(REPLICA_ID_2, BACKEND_ID_2, 0, Replica.ReplicaState.NORMAL)); - TabletDeleteInfo tabletDeleteInfo = new TabletDeleteInfo(PARTITION_ID, TABLET_ID); + TabletDeleteInfo tabletDeleteInfo = new TabletDeleteInfo(PH_PARTITION_ID, TABLET_ID); tabletDeleteInfo.getFinishedReplicas().addAll(finishedReplica); new MockUp() { @@ -305,7 +306,7 @@ public void testNormalTimeout() throws DdlException, QueryStateException { finishedReplica.add(new Replica(REPLICA_ID_1, BACKEND_ID_1, 0, Replica.ReplicaState.NORMAL)); finishedReplica.add(new Replica(REPLICA_ID_2, BACKEND_ID_2, 0, Replica.ReplicaState.NORMAL)); finishedReplica.add(new Replica(REPLICA_ID_3, BACKEND_ID_3, 0, Replica.ReplicaState.NORMAL)); - TabletDeleteInfo tabletDeleteInfo = new TabletDeleteInfo(PARTITION_ID, TABLET_ID); + TabletDeleteInfo tabletDeleteInfo = new TabletDeleteInfo(PH_PARTITION_ID, TABLET_ID); tabletDeleteInfo.getFinishedReplicas().addAll(finishedReplica); new MockUp() { @@ -355,7 +356,7 @@ public void testCommitFail(@Mocked MarkedCountDownLatch countDownLatch) throws D finishedReplica.add(new Replica(REPLICA_ID_1, BACKEND_ID_1, 0, Replica.ReplicaState.NORMAL)); finishedReplica.add(new Replica(REPLICA_ID_2, BACKEND_ID_2, 0, Replica.ReplicaState.NORMAL)); finishedReplica.add(new Replica(REPLICA_ID_3, BACKEND_ID_3, 0, Replica.ReplicaState.NORMAL)); - TabletDeleteInfo tabletDeleteInfo = new TabletDeleteInfo(PARTITION_ID, TABLET_ID); + TabletDeleteInfo tabletDeleteInfo = new TabletDeleteInfo(PH_PARTITION_ID, TABLET_ID); tabletDeleteInfo.getFinishedReplicas().addAll(finishedReplica); new MockUp() { @@ -420,7 +421,7 @@ public void testPublishFail(@Mocked MarkedCountDownLatch countDownLatch, @Mocked finishedReplica.add(new Replica(REPLICA_ID_1, BACKEND_ID_1, 0, Replica.ReplicaState.NORMAL)); finishedReplica.add(new Replica(REPLICA_ID_2, BACKEND_ID_2, 0, Replica.ReplicaState.NORMAL)); finishedReplica.add(new Replica(REPLICA_ID_3, BACKEND_ID_3, 0, Replica.ReplicaState.NORMAL)); - TabletDeleteInfo tabletDeleteInfo = new TabletDeleteInfo(PARTITION_ID, TABLET_ID); + TabletDeleteInfo tabletDeleteInfo = new TabletDeleteInfo(PH_PARTITION_ID, TABLET_ID); tabletDeleteInfo.getFinishedReplicas().addAll(finishedReplica); new MockUp() { @@ -477,7 +478,7 @@ public void testNormal(@Mocked MarkedCountDownLatch countDownLatch) throws DdlEx finishedReplica.add(new Replica(REPLICA_ID_1, BACKEND_ID_1, 0, Replica.ReplicaState.NORMAL)); finishedReplica.add(new Replica(REPLICA_ID_2, BACKEND_ID_2, 0, Replica.ReplicaState.NORMAL)); finishedReplica.add(new Replica(REPLICA_ID_3, BACKEND_ID_3, 0, Replica.ReplicaState.NORMAL)); - TabletDeleteInfo tabletDeleteInfo = new TabletDeleteInfo(PARTITION_ID, TABLET_ID); + TabletDeleteInfo tabletDeleteInfo = new TabletDeleteInfo(PH_PARTITION_ID, TABLET_ID); tabletDeleteInfo.getFinishedReplicas().addAll(finishedReplica); new MockUp() { @@ -524,7 +525,7 @@ public void testLockTimeout(@Mocked MarkedCountDownLatch countDownLatch) throws finishedReplica.add(new Replica(REPLICA_ID_1, BACKEND_ID_1, 0, Replica.ReplicaState.NORMAL)); finishedReplica.add(new Replica(REPLICA_ID_2, BACKEND_ID_2, 0, Replica.ReplicaState.NORMAL)); finishedReplica.add(new Replica(REPLICA_ID_3, BACKEND_ID_3, 0, Replica.ReplicaState.NORMAL)); - TabletDeleteInfo tabletDeleteInfo = new TabletDeleteInfo(PARTITION_ID, TABLET_ID); + TabletDeleteInfo tabletDeleteInfo = new TabletDeleteInfo(PH_PARTITION_ID, TABLET_ID); tabletDeleteInfo.getFinishedReplicas().addAll(finishedReplica); new MockUp() { diff --git a/fe/fe-core/src/test/java/com/starrocks/load/PartitionUtilsTest.java b/fe/fe-core/src/test/java/com/starrocks/load/PartitionUtilsTest.java index 75a1491fda07b1..7d7e5eea4f7cab 100644 --- a/fe/fe-core/src/test/java/com/starrocks/load/PartitionUtilsTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/load/PartitionUtilsTest.java @@ -48,7 +48,7 @@ public void testClearTabletsFromInvertedIndex() throws Exception { HashDistributionInfo distributionInfo = new HashDistributionInfo(1, Lists.newArrayList(new Column("id", Type.BIGINT))); - Partition p1 = new Partition(10001L, "p1", materializedIndex, distributionInfo); + Partition p1 = new Partition(10001L, 10002L, "p1", materializedIndex, distributionInfo); partitions.add(p1); PartitionUtils.clearTabletsFromInvertedIndex(partitions); } diff --git a/fe/fe-core/src/test/java/com/starrocks/load/loadv2/SparkLoadPendingTaskTest.java b/fe/fe-core/src/test/java/com/starrocks/load/loadv2/SparkLoadPendingTaskTest.java index ce5c02c2bcf848..06a69c4ad4d303 100644 --- a/fe/fe-core/src/test/java/com/starrocks/load/loadv2/SparkLoadPendingTaskTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/load/loadv2/SparkLoadPendingTaskTest.java @@ -110,7 +110,7 @@ public void testExecuteTask(@Injectable SparkLoadJob sparkLoadJob, long partitionId = 2L; DistributionInfo distributionInfo = new HashDistributionInfo(2, Lists.newArrayList(columns.get(0))); PartitionInfo partitionInfo = new SinglePartitionInfo(); - Partition partition = new Partition(partitionId, "p1", null, distributionInfo); + Partition partition = new Partition(partitionId, 21, "p1", null, distributionInfo); List partitions = Lists.newArrayList(partition); // file group @@ -258,11 +258,11 @@ public void testRangePartitionHashDistribution(@Injectable SparkLoadJob sparkLoa int distributionColumnIndex = 1; DistributionInfo distributionInfo = new HashDistributionInfo(3, Lists.newArrayList(columns.get(distributionColumnIndex))); - Partition partition1 = new Partition(partition1Id, "p1", null, + Partition partition1 = new Partition(partition1Id, 21, "p1", null, distributionInfo); - Partition partition2 = new Partition(partition2Id, "p2", null, + Partition partition2 = new Partition(partition2Id, 51, "p2", null, new HashDistributionInfo(4, Lists.newArrayList(columns.get(distributionColumnIndex)))); - Partition partition3 = new Partition(partition3Id, "tp3", null, + Partition partition3 = new Partition(partition3Id, 61, "tp3", null, distributionInfo); int partitionColumnIndex = 0; List partitions = Lists.newArrayList(partition1, partition2); diff --git a/fe/fe-core/src/test/java/com/starrocks/persist/RecyclePartitionInfoTest.java b/fe/fe-core/src/test/java/com/starrocks/persist/RecyclePartitionInfoTest.java index 3529cf661b3a46..0a79b65fb71fba 100644 --- a/fe/fe-core/src/test/java/com/starrocks/persist/RecyclePartitionInfoTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/persist/RecyclePartitionInfoTest.java @@ -53,7 +53,7 @@ public void testRangeSerialization() throws Exception { PartitionKey.createPartitionKey(Lists.newArrayList(new PartitionValue("3")), columns), BoundType.CLOSED); DataProperty dataProperty = new DataProperty(TStorageMedium.HDD); - Partition partition = new Partition(1L, "p1", new MaterializedIndex(), null); + Partition partition = new Partition(1L, 11L, "p1", new MaterializedIndex(), null); RecycleRangePartitionInfo info1 = new RecycleRangePartitionInfo(11L, 22L, partition, range, dataProperty, (short) 1, false, null); diff --git a/fe/fe-core/src/test/java/com/starrocks/planner/OlapTableSinkTest.java b/fe/fe-core/src/test/java/com/starrocks/planner/OlapTableSinkTest.java index 931be92193606f..0087f437618a28 100644 --- a/fe/fe-core/src/test/java/com/starrocks/planner/OlapTableSinkTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/planner/OlapTableSinkTest.java @@ -151,7 +151,7 @@ public void testSinglePartition() throws UserException { MaterializedIndex index = new MaterializedIndex(2, MaterializedIndex.IndexState.NORMAL); HashDistributionInfo distInfo = new HashDistributionInfo( 2, Lists.newArrayList(new Column("k1", Type.BIGINT))); - Partition partition = new Partition(2, "p1", index, distInfo); + Partition partition = new Partition(2, 22, "p1", index, distInfo); new Expectations() { { @@ -186,8 +186,8 @@ public void testRangePartition( Column partKey = new Column("k2", Type.VARCHAR); PartitionKey key = PartitionKey .createPartitionKey(Lists.newArrayList(new PartitionValue("123")), Lists.newArrayList(partKey)); - Partition p1 = new Partition(1, "p1", index, distInfo); - Partition p2 = new Partition(2, "p2", index, distInfo); + Partition p1 = new Partition(1, 21, "p1", index, distInfo); + Partition p2 = new Partition(2, 22, "p2", index, distInfo); new Expectations() { { @@ -248,6 +248,7 @@ public void testCreateLocationWithLocalTablet(@Mocked GlobalStateMgr globalState long partitionId = 3L; long indexId = 4L; long tabletId = 5L; + long physicalPartitionId = 6L; long replicaId = 10L; long backendId = 20L; @@ -279,11 +280,11 @@ public void testCreateLocationWithLocalTablet(@Mocked GlobalStateMgr globalState // Index MaterializedIndex index = new MaterializedIndex(indexId, MaterializedIndex.IndexState.NORMAL); - TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partitionId, indexId, 0, TStorageMedium.SSD); + TabletMeta tabletMeta = new TabletMeta(dbId, tableId, physicalPartitionId, indexId, 0, TStorageMedium.SSD); index.addTablet(tablet, tabletMeta); // Partition - Partition partition = new Partition(partitionId, "p1", index, distributionInfo); + Partition partition = new Partition(partitionId, physicalPartitionId, "p1", index, distributionInfo); // Table OlapTable table = new OlapTable(tableId, "t1", columns, KeysType.AGG_KEYS, partitionInfo, distributionInfo); @@ -308,7 +309,7 @@ public void testCreateLocationWithLocalTablet(@Mocked GlobalStateMgr globalState TOlapTablePartitionParam partitionParam = new TOlapTablePartitionParam(); TOlapTablePartition tPartition = new TOlapTablePartition(); - tPartition.setId(partitionId); + tPartition.setId(physicalPartitionId); partitionParam.addToPartitions(tPartition); TOlapTableLocationParam param = OlapTableSink.createLocation( table, partitionParam, false); @@ -332,6 +333,7 @@ public void testReplicatedStorageWithLocalTablet(@Mocked GlobalStateMgr globalSt long partitionId = 3L; long indexId = 4L; long tabletId = 5L; + long physicalPartitionId = 6L; long replicaId = 10L; long backendId = 20L; @@ -357,7 +359,7 @@ public void testReplicatedStorageWithLocalTablet(@Mocked GlobalStateMgr globalSt tablet.addReplica(replica3); // Index - TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partitionId, indexId, 0, TStorageMedium.SSD); + TabletMeta tabletMeta = new TabletMeta(dbId, tableId, physicalPartitionId, indexId, 0, TStorageMedium.SSD); index.addTablet(tablet, tabletMeta); } @@ -370,7 +372,7 @@ public void testReplicatedStorageWithLocalTablet(@Mocked GlobalStateMgr globalSt partitionInfo.setReplicationNum(partitionId, (short) 3); // Partition - Partition partition = new Partition(partitionId, "p1", index, distributionInfo); + Partition partition = new Partition(partitionId, physicalPartitionId, "p1", index, distributionInfo); // Table OlapTable table = new OlapTable(tableId, "t1", columns, KeysType.AGG_KEYS, partitionInfo, distributionInfo); @@ -395,7 +397,7 @@ public void testReplicatedStorageWithLocalTablet(@Mocked GlobalStateMgr globalSt TOlapTablePartitionParam partitionParam = new TOlapTablePartitionParam(); TOlapTablePartition tPartition = new TOlapTablePartition(); - tPartition.setId(partitionId); + tPartition.setId(physicalPartitionId); partitionParam.addToPartitions(tPartition); TOlapTableLocationParam param = OlapTableSink.createLocation( table, partitionParam, true); @@ -428,7 +430,7 @@ public void testSingleListPartition() throws UserException { MaterializedIndex index = new MaterializedIndex(1, MaterializedIndex.IndexState.NORMAL); HashDistributionInfo distInfo = new HashDistributionInfo( 3, Lists.newArrayList(new Column("id", Type.BIGINT))); - Partition partition = new Partition(1, "p1", index, distInfo); + Partition partition = new Partition(1, 11, "p1", index, distInfo); Map idToColumn = Maps.newTreeMap(ColumnId.CASE_INSENSITIVE_ORDER); idToColumn.put(ColumnId.create("province"), new Column("province", Type.STRING)); @@ -463,7 +465,7 @@ public void testImmutablePartition() throws UserException { partInfo.setReplicationNum(2, (short) 3); MaterializedIndex index = new MaterializedIndex(2, MaterializedIndex.IndexState.NORMAL); RandomDistributionInfo distInfo = new RandomDistributionInfo(3); - Partition partition = new Partition(2, "p1", index, distInfo); + Partition partition = new Partition(2, 22, "p1", index, distInfo); PhysicalPartition physicalPartition = new PhysicalPartition(3, "", 2, 0, index); partition.addSubPartition(physicalPartition); @@ -503,7 +505,7 @@ public void testInitialOpenPartition() throws UserException { partInfo.setReplicationNum(2, (short) 3); MaterializedIndex index = new MaterializedIndex(2, MaterializedIndex.IndexState.NORMAL); RandomDistributionInfo distInfo = new RandomDistributionInfo(3); - Partition partition = new Partition(2, "p1", index, distInfo); + Partition partition = new Partition(2, 22, "p1", index, distInfo); PhysicalPartition physicalPartition = new PhysicalPartition(3, "", 2, 0, index); partition.addSubPartition(physicalPartition); diff --git a/fe/fe-core/src/test/java/com/starrocks/planner/OlapTableSinkTest2.java b/fe/fe-core/src/test/java/com/starrocks/planner/OlapTableSinkTest2.java index ec268ea4493acc..2ca24d9e079330 100644 --- a/fe/fe-core/src/test/java/com/starrocks/planner/OlapTableSinkTest2.java +++ b/fe/fe-core/src/test/java/com/starrocks/planner/OlapTableSinkTest2.java @@ -16,7 +16,9 @@ import com.starrocks.catalog.Database; import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; import com.starrocks.catalog.PartitionInfo; +import com.starrocks.catalog.PhysicalPartition; import com.starrocks.common.UserException; import com.starrocks.qe.ConnectContext; import com.starrocks.server.GlobalStateMgr; @@ -32,8 +34,6 @@ import org.junit.BeforeClass; import org.junit.Test; -import java.util.List; - public class OlapTableSinkTest2 { private static StarRocksAssert starRocksAssert; private static ConnectContext connectContext; @@ -61,14 +61,15 @@ public int getQuorumNum(long partitionId, TWriteQuorumType writeQuorum) { Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("db2"); OlapTable olapTable = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "tbl1"); - List partitionIds = olapTable.getAllPartitionIds(); - TOlapTablePartitionParam partitionParam = new TOlapTablePartitionParam(); TOlapTablePartition tPartition = new TOlapTablePartition(); - for (Long partitionId : partitionIds) { - tPartition.setId(partitionId); - partitionParam.addToPartitions(tPartition); + for (Partition partition : olapTable.getPartitions()) { + for (PhysicalPartition physicalPartition : partition.getSubPartitions()) { + tPartition.setId(physicalPartition.getId()); + partitionParam.addToPartitions(tPartition); + } } + try { OlapTableSink.createLocation(olapTable, partitionParam, false); } catch (UserException e) { diff --git a/fe/fe-core/src/test/java/com/starrocks/server/LocalMetaStoreTest.java b/fe/fe-core/src/test/java/com/starrocks/server/LocalMetaStoreTest.java index 69304487d244f0..2e2c5e99e74651 100644 --- a/fe/fe-core/src/test/java/com/starrocks/server/LocalMetaStoreTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/server/LocalMetaStoreTest.java @@ -180,10 +180,10 @@ public void testReplayAddSubPartition() throws DdlException { int schemaHash = table.getSchemaHashByIndexId(p.getBaseIndex().getId()); MaterializedIndex index = new MaterializedIndex(); TabletMeta tabletMeta = new TabletMeta(db.getId(), table.getId(), p.getId(), - index.getId(), schemaHash, table.getPartitionInfo().getDataProperty(p.getId()).getStorageMedium()); + index.getId(), schemaHash, table.getPartitionInfo().getDataProperty(p.getParentId()).getStorageMedium()); index.addTablet(new LocalTablet(0), tabletMeta); PhysicalPartitionPersistInfoV2 info = new PhysicalPartitionPersistInfoV2( - db.getId(), table.getId(), p.getId(), new PhysicalPartition(123, "", p.getId(), 0, index)); + db.getId(), table.getId(), p.getParentId(), new PhysicalPartition(123, "", p.getId(), 0, index)); LocalMetastore localMetastore = connectContext.getGlobalStateMgr().getLocalMetastore(); localMetastore.replayAddSubPartition(info); diff --git a/fe/fe-core/src/test/java/com/starrocks/server/WarehouseManagerTest.java b/fe/fe-core/src/test/java/com/starrocks/server/WarehouseManagerTest.java index a15ab2cdcbed60..e32431f0243e14 100644 --- a/fe/fe-core/src/test/java/com/starrocks/server/WarehouseManagerTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/server/WarehouseManagerTest.java @@ -276,7 +276,7 @@ public RunMode getCurrentRunMode() { }; OlapScanNode scanNode = newOlapScanNode(); - Partition partition = new Partition(123, "aaa", null, null); + Partition partition = new Partition(123, 456, "aaa", null, null); MaterializedIndex index = new MaterializedIndex(1, MaterializedIndex.IndexState.NORMAL); ErrorReportException ex = Assert.assertThrows(ErrorReportException.class, () -> scanNode.addScanRangeLocations(partition, partition.getDefaultPhysicalPartition(), @@ -353,7 +353,7 @@ public RunMode getCurrentRunMode() { }; OlapScanNode scanNode = newOlapScanNode(); - Partition partition = new Partition(123, "aaa", null, null); + Partition partition = new Partition(123, 456, "aaa", null, null); MaterializedIndex index = new MaterializedIndex(1, MaterializedIndex.IndexState.NORMAL); scanNode.addScanRangeLocations(partition, partition.getDefaultPhysicalPartition(), index, Collections.emptyList(), 1); // Since this is the second call to addScanRangeLocations on the same OlapScanNode, we do not expect another call to diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/OptimizerTaskTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/OptimizerTaskTest.java index 855d3aeb7f6c49..6c3a03c486476f 100644 --- a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/OptimizerTaskTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/OptimizerTaskTest.java @@ -446,21 +446,25 @@ public void testFourJoin(@Mocked OlapTable olapTable1, result = new ArrayList<>(scanColumnMap.values()); minTimes = 0; } + { olapTable2.getBaseSchema(); result = new ArrayList<>(scanColumnMap.values()); minTimes = 0; } + { olapTable3.getBaseSchema(); result = new ArrayList<>(scanColumnMap.values()); minTimes = 0; } + { olapTable4.getBaseSchema(); result = new ArrayList<>(scanColumnMap.values()); minTimes = 0; } + { olapTable5.getBaseSchema(); result = new ArrayList<>(scanColumnMap.values()); @@ -1945,11 +1949,11 @@ public void testBroadcastExceedRowLimitWithHugeGapInRowCount(@Mocked OlapTable o MaterializedIndex m1 = new MaterializedIndex(); m1.setRowCount(100000000); - Partition p1 = new Partition(0, "p1", m1, hashDistributionInfo1); + Partition p1 = new Partition(0, 10, "p1", m1, hashDistributionInfo1); MaterializedIndex m2 = new MaterializedIndex(); m2.setRowCount(20000000); - Partition p2 = new Partition(1, "p2", m2, hashDistributionInfo2); + Partition p2 = new Partition(1, 11, "p2", m2, hashDistributionInfo2); new Expectations() { { olapTable1.getId(); @@ -2053,11 +2057,11 @@ public void testBroadcastExceedRowLimitWithoutHugeGapInRowCount(@Mocked OlapTabl MaterializedIndex m1 = new MaterializedIndex(); m1.setRowCount(100000000); - Partition p1 = new Partition(0, "p1", m1, hashDistributionInfo1); + Partition p1 = new Partition(0, 10, "p1", m1, hashDistributionInfo1); MaterializedIndex m2 = new MaterializedIndex(); m2.setRowCount(20000000); - Partition p2 = new Partition(1, "p2", m2, hashDistributionInfo2); + Partition p2 = new Partition(1, 11, "p2", m2, hashDistributionInfo2); new Expectations() { { @@ -2170,11 +2174,11 @@ public void testOlapTablePartitionRowCount(@Mocked OlapTable olapTable1, MaterializedIndex m1 = new MaterializedIndex(); m1.setRowCount(1000000); - Partition p1 = new Partition(0, "p1", m1, hashDistributionInfo1); + Partition p1 = new Partition(0, 10, "p1", m1, hashDistributionInfo1); MaterializedIndex m2 = new MaterializedIndex(); m2.setRowCount(2000000); - Partition p2 = new Partition(1, "p2", m2, hashDistributionInfo2); + Partition p2 = new Partition(1, 11, "p2", m2, hashDistributionInfo2); Map idToColumn = Maps.newTreeMap(ColumnId.CASE_INSENSITIVE_ORDER); idToColumn.put(column2.getColumnId(), column2); diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/DistributionPrunerRuleTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/DistributionPrunerRuleTest.java index b3e35721683599..c7f286f4c2a183 100644 --- a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/DistributionPrunerRuleTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/DistributionPrunerRuleTest.java @@ -28,6 +28,7 @@ import com.starrocks.catalog.MaterializedIndex; import com.starrocks.catalog.OlapTable; import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PhysicalPartition; import com.starrocks.catalog.ScalarType; import com.starrocks.catalog.Type; import com.starrocks.planner.PartitionColumnFilter; @@ -57,7 +58,8 @@ public class DistributionPrunerRuleTest { @Test - public void transform(@Mocked OlapTable olapTable, @Mocked Partition partition, @Mocked MaterializedIndex index, + public void transform(@Mocked OlapTable olapTable, @Mocked Partition partition, @Mocked PhysicalPartition physicalPartition, + @Mocked MaterializedIndex index, @Mocked HashDistributionInfo distributionInfo) { List tabletIds = Lists.newArrayListWithExpectedSize(300); for (long i = 0; i < 300; i++) { @@ -169,9 +171,9 @@ public void transform(@Mocked OlapTable olapTable, @Mocked Partition partition, result = idToColumn; partition.getSubPartitions(); - result = Arrays.asList(partition); + result = Arrays.asList(physicalPartition); - partition.getDefaultPhysicalPartition().getIndex(anyLong); + physicalPartition.getIndex(anyLong); result = index; partition.getDistributionInfo(); diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/PartitionPruneRuleTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/PartitionPruneRuleTest.java index 3770aada1b1e2f..ec775c1c4c2cb2 100644 --- a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/PartitionPruneRuleTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/PartitionPruneRuleTest.java @@ -62,11 +62,11 @@ public class PartitionPruneRuleTest { @Test public void transform1(@Mocked OlapTable olapTable, @Mocked RangePartitionInfo partitionInfo) { FeConstants.runningUnitTest = true; - Partition part1 = new Partition(1, "p1", null, null); - Partition part2 = new Partition(2, "p2", null, null); - Partition part3 = new Partition(3, "p3", null, null); - Partition part4 = new Partition(4, "p4", null, null); - Partition part5 = new Partition(5, "p5", null, null); + Partition part1 = new Partition(1, 11, "p1", null, null); + Partition part2 = new Partition(2, 22, "p2", null, null); + Partition part3 = new Partition(3, 33, "p3", null, null); + Partition part4 = new Partition(4, 44, "p4", null, null); + Partition part5 = new Partition(5, 55, "p5", null, null); List columns = Lists.newArrayList( new Column("dealDate", Type.DATE, false) @@ -165,11 +165,11 @@ public void transform1(@Mocked OlapTable olapTable, @Mocked RangePartitionInfo p @Test public void transform2(@Mocked OlapTable olapTable, @Mocked RangePartitionInfo partitionInfo) { FeConstants.runningUnitTest = true; - Partition part1 = new Partition(1, "p1", null, null); - Partition part2 = new Partition(2, "p2", null, null); - Partition part3 = new Partition(3, "p3", null, null); - Partition part4 = new Partition(4, "p4", null, null); - Partition part5 = new Partition(5, "p5", null, null); + Partition part1 = new Partition(1, 11, "p1", null, null); + Partition part2 = new Partition(2, 22, "p2", null, null); + Partition part3 = new Partition(3, 33, "p3", null, null); + Partition part4 = new Partition(4, 44, "p4", null, null); + Partition part5 = new Partition(5, 55, "p5", null, null); List columns = Lists.newArrayList( new Column("dealDate", Type.DATE, false), @@ -304,8 +304,8 @@ public void transformForSingleItemListPartition(@Mocked OlapTable olapTable, LogicalOlapScanOperator operator = new LogicalOlapScanOperator(olapTable, scanColumnMap, columnMetaToColRefMap, null, -1, predicate); - Partition part1 = new Partition(10001L, "p1", null, null); - Partition part2 = new Partition(10002L, "p2", null, null); + Partition part1 = new Partition(10001L, 10003L, "p1", null, null); + Partition part2 = new Partition(10002L, 10004L, "p2", null, null); List p1 = Lists.newArrayList( new PartitionValue("guangdong").getValue(Type.STRING), @@ -383,8 +383,8 @@ public void transformForSingleItemListPartitionWithTemp(@Mocked OlapTable olapTa null, -1, null, olapTable.getBaseIndexId(), null, partitionNames, false, Lists.newArrayList(), Lists.newArrayList(), null, false); - Partition part1 = new Partition(10001L, "p1", null, null); - Partition part2 = new Partition(10002L, "p2", null, null); + Partition part1 = new Partition(10001L, 10003L, "p1", null, null); + Partition part2 = new Partition(10002L, 10004L, "p2", null, null); List p1 = Lists.newArrayList( new PartitionValue("guangdong").getValue(Type.STRING), diff --git a/fe/fe-core/src/test/java/com/starrocks/statistic/StatisticsCollectionTriggerTest.java b/fe/fe-core/src/test/java/com/starrocks/statistic/StatisticsCollectionTriggerTest.java index 934c261f943af0..3594982a441b20 100644 --- a/fe/fe-core/src/test/java/com/starrocks/statistic/StatisticsCollectionTriggerTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/statistic/StatisticsCollectionTriggerTest.java @@ -73,7 +73,7 @@ public void triggerOnLoad() throws Exception { TransactionState transactionState = new TransactionState(); TableCommitInfo commitInfo = new TableCommitInfo(table.getId()); - commitInfo.addPartitionCommitInfo(new PartitionCommitInfo(partition.getId(), 2, 1)); + commitInfo.addPartitionCommitInfo(new PartitionCommitInfo(partition.getDefaultPhysicalPartition().getId(), 2, 1)); transactionState.putIdToTableCommitInfo(table.getId(), commitInfo); setPartitionStatistics((OlapTable) table, "p1", 1000); @@ -129,7 +129,7 @@ public void triggerOnInsertOverwrite() throws Exception { InsertOverwriteJobStats stats = new InsertOverwriteJobStats( List.of(sourceId), List.of(targetId), 1000, 1001); StatisticsCollectionTrigger.triggerOnInsertOverwrite(stats, db, table, true, true); - Partition targetPartition = new Partition(targetId, "p1", null, null); + Partition targetPartition = new Partition(targetId, targetId + 100, "p1", null, null); Map> tableStats = storage.getTableStatistics(table.getId(), List.of(targetPartition)); Assert.assertEquals(Map.of(targetId, Optional.of(1000L)), tableStats); diff --git a/fe/fe-core/src/test/java/com/starrocks/transaction/DatabaseTransactionMgrTest.java b/fe/fe-core/src/test/java/com/starrocks/transaction/DatabaseTransactionMgrTest.java index ec1536c03697f3..eb4df188e4ac04 100644 --- a/fe/fe-core/src/test/java/com/starrocks/transaction/DatabaseTransactionMgrTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/transaction/DatabaseTransactionMgrTest.java @@ -441,7 +441,7 @@ public void testGetTableTransInfo() throws AnalysisException { List tableTransInfo = tableTransInfos.get(0); assertEquals(2, tableTransInfo.size()); assertEquals(2L, tableTransInfo.get(0)); - assertEquals("3", tableTransInfo.get(1)); + assertEquals("103", tableTransInfo.get(1)); } @Test @@ -454,7 +454,7 @@ public void testGetPartitionTransInfo() throws AnalysisException { assertEquals(1, partitionTransInfos.size()); List partitionTransInfo = partitionTransInfos.get(0); assertEquals(2, partitionTransInfo.size()); - assertEquals(3L, partitionTransInfo.get(0)); + assertEquals(103L, partitionTransInfo.get(0)); assertEquals(13L, partitionTransInfo.get(1)); } diff --git a/fe/fe-core/src/test/java/com/starrocks/transaction/LakeTableTestHelper.java b/fe/fe-core/src/test/java/com/starrocks/transaction/LakeTableTestHelper.java index 1412583d88e5c9..7b19c35d7442d2 100644 --- a/fe/fe-core/src/test/java/com/starrocks/transaction/LakeTableTestHelper.java +++ b/fe/fe-core/src/test/java/com/starrocks/transaction/LakeTableTestHelper.java @@ -37,17 +37,18 @@ public class LakeTableTestHelper { long partitionId = 9002; long indexId = 9003; long[] tabletId = {9004, 90005}; + long physicalPartitionId = 90006; long nextTxnId = 10000; LakeTable buildLakeTable() { MaterializedIndex index = new MaterializedIndex(indexId); TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); for (long id : tabletId) { - TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partitionId, 0, 0, TStorageMedium.HDD, true); + TabletMeta tabletMeta = new TabletMeta(dbId, tableId, physicalPartitionId, 0, 0, TStorageMedium.HDD, true); invertedIndex.addTablet(id, tabletMeta); index.addTablet(new LakeTablet(id), tabletMeta); } - Partition partition = new Partition(partitionId, "p0", index, null); + Partition partition = new Partition(partitionId, physicalPartitionId, "p0", index, null); LakeTable table = new LakeTable( tableId, "t0", Lists.newArrayList(new Column("c0", Type.BIGINT)), diff --git a/fe/fe-core/src/test/java/com/starrocks/transaction/LakeTableTxnLogApplierTest.java b/fe/fe-core/src/test/java/com/starrocks/transaction/LakeTableTxnLogApplierTest.java index 58d27805758be9..8dbf83f4400afe 100644 --- a/fe/fe-core/src/test/java/com/starrocks/transaction/LakeTableTxnLogApplierTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/transaction/LakeTableTxnLogApplierTest.java @@ -25,7 +25,7 @@ public void testCommitAndApply() { LakeTableTxnLogApplier applier = new LakeTableTxnLogApplier(table); TransactionState state = newTransactionState(); state.setTransactionStatus(TransactionStatus.COMMITTED); - PartitionCommitInfo partitionCommitInfo = new PartitionCommitInfo(partitionId, 2, 0); + PartitionCommitInfo partitionCommitInfo = new PartitionCommitInfo(physicalPartitionId, 2, 0); TableCommitInfo tableCommitInfo = new TableCommitInfo(tableId); tableCommitInfo.addPartitionCommitInfo(partitionCommitInfo);