You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
I have created a random classification dataset with 100,000 rows and 30 columns and I am training the distributed xgboost on this dataset. My system has 8 workers, therefore I created 8 partitions of my dataset. While running dask.xgboost only once on this dataset, each of the 8 workers get one part of the DaskDmatrix (total it has 8 parts). But when I submit multiple dask.xgboost calls, then the 8 partitions get randomly divided among a subset of the workers. Therefore, only those workers end up utilizing CPU and runtime becomes very high.
Here is a reproducible example,
from dask.distributed import Client
import xgboost as xgb
from sklearn.datasets import make_classification
import dask.array as da
from dask.distributed import get_client
import dask
n_samples, n_features, num_test_samples = 100000, 30, 100
dask.config.set({'distributed.worker.daemon': False})
def invoke_dis_xgboost(X, y, number_of_classes, number_of_estimators):
client_xgboost = get_client()
dtrain = xgb.dask.DaskDMatrix(client_xgboost, X, y)
xgb_params = {
'n_estimators': number_of_estimators,
'num_class': number_of_classes,
## all other xgb parameter
}
output = xgb.dask.train(client_xgboost, xgb_params, dtrain, num_boost_round=100)
return
def main(client):
print(f'n_samples={n_samples}, n_features={n_features}')
X_local, y_local = make_classification(n_samples=n_samples, n_features=n_features, random_state=12345)
number_of_classes = len(set(y_local))
X = da.from_array(X_local, chunks=(n_samples//8,n_features), name='train_feature')
y = da.from_array(y_local, chunks=(n_samples//8,), name='train_label')
futures = []
results = []
for i in range(100, 105):
f1 = client.submit(invoke_dis_xgboost, X, y, number_of_classes, i)
futures.append(f1)
for i, f in enumerate(futures):
results.append(f.result())
return
if __name__ == '__main__':
client = Client('127.0.0.1:8786')
main(client)
Looking at this part of the source code of xgboost.dask.train
key_to_partition = {part.key: part for part in parts}
who_has = await client.scheduler.who_has(keys=[part.key for part in parts])
worker_map: Dict[str, "distributed.Future"] = defaultdict(list)
for key, workers in who_has.items():
worker_map[next(iter(workers))].append(key_to_partition[key])
I was hoping to see each part will be distributed to one worker only. However, I am getting the following output,
Since there are only 3 unique workers getting all the parititions, number of dispatched_train is 3, therefore only 3 workers are utilizing CPU according to the dask dashboard.
The text was updated successfully, but these errors were encountered:
Thanks for raising an issue @rudra0713. It's not clear to me if this behavior is due to Dask or XGBoost -- cc'ing @trivialfis for thoughts on the XGBoost side of things
Thanks for the ping. This is from dmlc/xgboost#7544 . The background is dask distributes the data to only a limited set of workers (3 in the issue), as a result XGBoost runs training only on these workers instead of utilizing all available resources. The solution is to have something similar to client.rebalance that distributes data to workers in a more uniform way. But I think this function is still being worked on, so redirected the discussion from xgboost to distributed.
Is it possible to reproduce this problem without xgboost? Can you create a similar situation where, just submitting some dummy futures, and then submitting more futures, the second batch doesn't get distributed to all workers as expected? It sounds to me like it's a more general issue with scheduling policy than something xgboost is doing, but I may be wrong.
I have created a random classification dataset with 100,000 rows and 30 columns and I am training the distributed xgboost on this dataset. My system has 8 workers, therefore I created 8 partitions of my dataset. While running dask.xgboost only once on this dataset, each of the 8 workers get one part of the DaskDmatrix (total it has 8 parts). But when I submit multiple dask.xgboost calls, then the 8 partitions get randomly divided among a subset of the workers. Therefore, only those workers end up utilizing CPU and runtime becomes very high.
Here is a reproducible example,
Looking at this part of the source code of xgboost.dask.train
I was hoping to see each part will be distributed to one worker only. However, I am getting the following output,
Since there are only 3 unique workers getting all the parititions, number of
dispatched_train
is 3, therefore only 3 workers are utilizing CPU according to the dask dashboard.The text was updated successfully, but these errors were encountered: