Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[fix](cloud) Fix constant folding for requests sent to BE without dis… #42865

Merged
merged 4 commits into from
Nov 1, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -270,7 +270,7 @@ private static Map<String, Expression> evalOnBE(Map<String, Map<String, TExpr>>
Map<String, Expression> constMap, ConnectContext context) {
Map<String, Expression> resultMap = new HashMap<>();
try {
List<Long> backendIds = Env.getCurrentSystemInfo().getAllBackendIds(true);
List<Long> backendIds = Env.getCurrentSystemInfo().getAllBackendByCurrentCluster(true);
if (backendIds.isEmpty()) {
throw new UserException("No alive backends");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -361,7 +361,7 @@ private Map<String, Map<String, Expr>> calcConstExpr(Map<String, Map<String, TEx
TNetworkAddress brpcAddress = null;
Map<String, Map<String, Expr>> resultMap = new HashMap<>();
try {
List<Long> backendIds = Env.getCurrentSystemInfo().getAllBackendIds(true);
List<Long> backendIds = Env.getCurrentSystemInfo().getAllBackendByCurrentCluster(true);
if (backendIds.isEmpty()) {
throw new LoadException("Failed to get all partitions. No alive backends");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3495,7 +3495,7 @@ public TCreatePartitionResult createPartition(TCreatePartitionRequest request) t
// build nodes
List<TNodeInfo> nodeInfos = Lists.newArrayList();
SystemInfoService systemInfoService = Env.getCurrentSystemInfo();
for (Long id : systemInfoService.getAllBackendIds(false)) {
for (Long id : systemInfoService.getAllBackendByCurrentCluster(false)) {
Backend backend = systemInfoService.getBackend(id);
nodeInfos.add(new TNodeInfo(backend.getId(), 0, backend.getHost(), backend.getBrpcPort()));
}
Expand Down Expand Up @@ -3704,7 +3704,7 @@ public TReplacePartitionResult replacePartition(TReplacePartitionRequest request
// build nodes
List<TNodeInfo> nodeInfos = Lists.newArrayList();
SystemInfoService systemInfoService = Env.getCurrentSystemInfo();
for (Long id : systemInfoService.getAllBackendIds(false)) {
for (Long id : systemInfoService.getAllBackendByCurrentCluster(false)) {
Backend backend = systemInfoService.getBackend(id);
nodeInfos.add(new TNodeInfo(backend.getId(), 0, backend.getHost(), backend.getBrpcPort()));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@ public int compare(List<Backend> list1, List<Backend> list2) {
public static TPaloNodesInfo createAliveNodesInfo() {
TPaloNodesInfo nodesInfo = new TPaloNodesInfo();
SystemInfoService systemInfoService = Env.getCurrentSystemInfo();
for (Long id : systemInfoService.getAllBackendIds(true /*need alive*/)) {
for (Long id : systemInfoService.getAllBackendByCurrentCluster(true)) {
Backend backend = systemInfoService.getBackend(id);
nodesInfo.addToNodes(new TNodeInfo(backend.getId(), 0, backend.getHost(), backend.getBrpcPort()));
}
Expand Down Expand Up @@ -363,11 +363,22 @@ public List<Long> getAllBackendIds() {
public int getBackendsNumber(boolean needAlive) {
int beNumber = ConnectContext.get().getSessionVariable().getBeNumberForTest();
if (beNumber < 0) {
beNumber = getAllBackendIds(needAlive).size();
beNumber = getAllBackendByCurrentCluster(needAlive).size();
}
return beNumber;
}

public List<Long> getAllBackendByCurrentCluster(boolean needAlive) {
try {
return getBackendsByCurrentCluster()
.values().stream().filter(be -> !needAlive || be.isAlive())
.map(Backend::getId).collect(Collectors.toList());
} catch (AnalysisException e) {
LOG.warn("failed to get backends by Current Cluster", e);
return Lists.newArrayList();
}
}

public List<Long> getAllBackendIds(boolean needAlive) {
ImmutableMap<Long, Backend> idToBackend = getAllClusterBackendsNoException();
List<Long> backendIds = Lists.newArrayList(idToBackend.keySet());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,7 @@ private void getFileListFromBackend() throws AnalysisException {
backendIdForRequest = backendId;
} else {
Preconditions.checkState(sharedStorage);
List<Long> beIds = Env.getCurrentSystemInfo().getAllBackendIds(true);
List<Long> beIds = Env.getCurrentSystemInfo().getAllBackendByCurrentCluster(true);
if (beIds.isEmpty()) {
throw new AnalysisException("No available backend");
}
Expand Down
Loading