J-HowHuang commented on code in PR #15891:
URL: https://github.com/apache/pinot/pull/15891#discussion_r2138522771
##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/rebalance/tenant/DefaultTenantRebalancer.java:
##########
@@ -49,6 +56,80 @@ public DefaultTenantRebalancer(PinotHelixResourceManager
pinotHelixResourceManag
@Override
public TenantRebalanceResult rebalance(TenantRebalanceConfig config) {
+ Map<String, RebalanceResult> dryRunResults = new HashMap<>();
+ Set<String> tables = getTenantTables(config.getTenantName());
+ Set<String> allowTables = config.getAllowTables();
+ if (!allowTables.isEmpty()) {
+ tables.retainAll(allowTables);
+ }
+ tables.removeAll(config.getBlockTables());
+ tables.forEach(table -> {
+ try {
+ RebalanceConfig rebalanceConfig = RebalanceConfig.copy(config);
+ rebalanceConfig.setDryRun(true);
+ dryRunResults.put(table,
+ _pinotHelixResourceManager.rebalanceTable(table, rebalanceConfig,
createUniqueRebalanceJobIdentifier(),
+ false));
+ } catch (TableNotFoundException exception) {
+ dryRunResults.put(table, new RebalanceResult(null,
RebalanceResult.Status.FAILED, exception.getMessage(),
+ null, null, null, null, null));
+ }
+ });
+ if (config.isDryRun()) {
+ return new TenantRebalanceResult(null, dryRunResults,
config.isVerboseResult());
+ }
+
+ String tenantRebalanceJobId = createUniqueRebalanceJobIdentifier();
+ TenantRebalanceObserver observer = new
ZkBasedTenantRebalanceObserver(tenantRebalanceJobId, config.getTenantName(),
+ tables, _pinotHelixResourceManager);
+ observer.onTrigger(TenantRebalanceObserver.Trigger.START_TRIGGER, null,
null);
+ ConcurrentLinkedQueue<String> parallelQueue = createTableQueue(config,
dryRunResults);
+ // ensure atleast 1 thread is created to run the sequential table
rebalance operations
+ int parallelism = Math.max(config.getDegreeOfParallelism(), 1);
+ try {
+ for (int i = 0; i < parallelism; i++) {
+ _executorService.submit(() -> {
+ while (true) {
+ String table = parallelQueue.poll();
+ if (table == null) {
+ break;
+ }
+ RebalanceConfig rebalanceConfig = RebalanceConfig.copy(config);
+ rebalanceConfig.setDryRun(false);
+ if (dryRunResults.get(table)
+ .getRebalanceSummaryResult()
+ .getSegmentInfo()
+ .getReplicationFactor()
+ .getExpectedValueAfterRebalance() == 1) {
+ rebalanceConfig.setMinAvailableReplicas(0);
+ }
+ rebalanceTable(table, rebalanceConfig,
dryRunResults.get(table).getJobId(), observer);
+ }
+ observer.onSuccess(String.format("Successfully rebalanced tenant
%s.", config.getTenantName()));
+ });
+ }
+ } catch (Exception exception) {
+ observer.onError(String.format("Failed to rebalance the tenant %s.
Cause: %s", config.getTenantName(),
+ exception.getMessage()));
+ }
+
+ // Prepare tenant rebalance result to return
+ Map<String, RebalanceResult> rebalanceResults = new HashMap<>();
+ for (String table : dryRunResults.keySet()) {
+ RebalanceResult result = dryRunResults.get(table);
+ if (result.getStatus() == RebalanceResult.Status.DONE) {
+ rebalanceResults.put(table, new RebalanceResult(result.getJobId(),
RebalanceResult.Status.IN_PROGRESS,
Review Comment:
Yes, this is the same design as table rebalance api.
As for the failed tables, there will be an aggregated view of all failed
tables in the new `TenantRebalanceResult` in the later output improvement PR
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]