dask
dask copied to clipboard
printing dd.concat([]).reset_index().set_index() raises IndexError
- python: Python 3.13.9 | packaged by Anaconda, Inc. | (main, Oct 21 2025, 19:16:10) [GCC 11.2.0] on linux
- dask: 2025.11.0 (conda-forge)
import dask
print(dask.__version__)
import dask.dataframe as dd
import pandas as pd
df1 = pd.DataFrame({'trid': [0, 1], 'ts': [100, 101]}).set_index('trid')
df1 = dd.from_pandas(df1, npartitions=1)
df2 = pd.DataFrame({'trid': [2, 3], 'ts': [102, 103]}).set_index('trid')
df2 = dd.from_pandas(df2, npartitions=1)
df3 = dd.concat([df1, df2], axis=0).reset_index().set_index('trid')
print(df3)
raises
---------------------------------------------------------------------------
IndexError Traceback (most recent call last)
Cell In[20], line 6
4 df2 = dd.from_pandas(df2, npartitions=1)
5 df3 = dd.concat([df1, df2], axis=0).reset_index().set_index('trid')
----> 6 print(df3)
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/dataframe/dask_expr/_collection.py:431], in FrameBase.__repr__(self)
430 def __repr__(self):
--> 431 data = self._repr_data().to_string(max_rows=5)
432 _str_fmt = """Dask {klass} Structure:
433 {data}
434 Dask Name: {name}, {n_expr}
435 Expr={expr}"""
436 if not isinstance(self, Series) and not len(self.columns):
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/dataframe/dask_expr/_collection.py:4072], in DataFrame._repr_data(self)
4070 def _repr_data(self):
4071 meta = self._meta
-> 4072 index = self._repr_divisions
4073 cols = meta.columns
4074 if len(cols) == 0:
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/dataframe/dask_expr/_collection.py:2625], in FrameBase._repr_divisions(self)
2623 @property
2624 def _repr_divisions(self):
-> 2625 name = f"npartitions={self.npartitions}"
2626 if self.known_divisions:
2627 divisions = pd.Index(self.divisions, name=name)
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/dataframe/dask_expr/_collection.py:354], in FrameBase.npartitions(self)
351 @property
352 def npartitions(self):
353 """Return number of partitions"""
--> 354 return self.expr.npartitions
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/dataframe/dask_expr/_shuffle.py:803], in BaseSetIndexSortValues.npartitions(self)
801 @property
802 def npartitions(self):
--> 803 return self.operand("npartitions") or len(self._divisions()) - 1
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/dataframe/dask_expr/_shuffle.py:786], in BaseSetIndexSortValues._divisions(self)
779 if (
780 is_index_like(self._divisions_column._meta)
781 and self._divisions_column.known_divisions
782 and self._divisions_column.npartitions == self.frame.npartitions
783 ):
784 return self.other.divisions
--> 786 divisions, mins, maxes, presorted = _get_divisions(
787 self.frame,
788 self._divisions_column,
789 self._npartitions_input,
790 self.ascending,
791 upsample=self.upsample,
792 )
793 if presorted and len(mins) == self._npartitions_input:
794 divisions = mins.copy() + [maxes[-1]]
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/dataframe/dask_expr/_shuffle.py:1333], in _get_divisions(frame, other, npartitions, ascending, partition_size, upsample)
1331 if key in divisions_lru:
1332 return divisions_lru[key]
-> 1333 result = _calculate_divisions(
1334 frame, other, npartitions, ascending, partition_size, upsample
1335 )
1336 divisions_lru[key] = result
1337 return result
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/dataframe/dask_expr/_shuffle.py:1357], in _calculate_divisions(frame, other, npartitions, ascending, partition_size, upsample)
1354 other = new_collection(other).cat.as_ordered()._expr
1356 try:
-> 1357 divisions, mins, maxes = compute(
1358 new_collection(RepartitionQuantiles(other, npartitions, upsample=upsample)),
1359 new_collection(other).map_partitions(M.min),
1360 new_collection(other).map_partitions(M.max),
1361 )
1362 except TypeError as e:
1363 # When there are nulls and a column is non-numeric, a TypeError is sometimes raised as a result of
1364 # 1) computing mins/maxes above, 2) every null being switched to NaN, and 3) NaN being a float.
1365 # Also, Pandas ExtensionDtypes may cause TypeErrors when dealing with special nulls such as pd.NaT or pd.NA.
1366 # If this happens, we hint the user about eliminating nulls beforehand.
1367 if not pd.api.types.is_numeric_dtype(other._meta.dtype):
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/base.py:678], in compute(traverse, optimize_graph, scheduler, get, *args, **kwargs)
659 expr = FinalizeCompute(expr)
661 with shorten_traceback():
662 # The high level optimize will have to be called client side (for now)
663 # The optimize can internally trigger already a computation
(...) 675 # change the graph submission to a handshake which introduces all sorts
676 # of concurrency control issues)
--> 678 expr = expr.optimize()
679 keys = list(flatten(expr.__dask_keys__()))
681 results = schedule(expr, keys, **kwargs)
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/_expr.py:440], in Expr.optimize(self, fuse)
437 def optimize(self, fuse: bool = False) -> Expr:
438 stage: OptimizerStage = "fused" if fuse else "simplified-physical"
--> 440 return optimize_until(self, stage)
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/_expr.py:934], in optimize_until(expr, stage)
931 return result
933 # Simplify
--> 934 expr = result.simplify()
935 if stage == "simplified-logical":
936 return expr
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/_expr.py:450], in Expr.simplify(self)
448 while True:
449 dependents = collect_dependents(expr)
--> 450 new = expr.simplify_once(dependents=dependents, simplified={})
451 if new._name == expr._name:
452 break
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/_expr.py:420], in Expr.simplify_once(self, dependents, simplified)
417 if isinstance(operand, Expr):
418 # Bandaid for now, waiting for Singleton
419 dependents[operand._name].append(weakref.ref(expr))
--> 420 new = operand.simplify_once(
421 dependents=dependents, simplified=simplified
422 )
423 simplified[operand._name] = new
424 if new._name != operand._name:
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/_expr.py:420], in Expr.simplify_once(self, dependents, simplified)
417 if isinstance(operand, Expr):
418 # Bandaid for now, waiting for Singleton
419 dependents[operand._name].append(weakref.ref(expr))
--> 420 new = operand.simplify_once(
421 dependents=dependents, simplified=simplified
422 )
423 simplified[operand._name] = new
424 if new._name != operand._name:
[... skipping similar frames: Expr.simplify_once at line 420 (2 times)]
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/_expr.py:420], in Expr.simplify_once(self, dependents, simplified)
417 if isinstance(operand, Expr):
418 # Bandaid for now, waiting for Singleton
419 dependents[operand._name].append(weakref.ref(expr))
--> 420 new = operand.simplify_once(
421 dependents=dependents, simplified=simplified
422 )
423 simplified[operand._name] = new
424 if new._name != operand._name:
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/_expr.py:403], in Expr.simplify_once(self, dependents, simplified)
401 # Allow children to simplify their parents
402 for child in expr.dependencies():
--> 403 out = child._simplify_up(expr, dependents)
404 if out is None:
405 out = expr
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/dataframe/dask_expr/_concat.py:284], in Concat._simplify_up(self, parent, dependents)
264 frames = [
265 (
266 frame[cols]
(...) 272 if len(cols) > 0
273 ]
274 result = type(self)(
275 self.join,
276 self.ignore_order,
(...) 281 *frames,
282 )
--> 284 if result.columns == _convert_to_list(parent.operand("columns")):
285 if result.ndim == parent.ndim:
286 return result
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/dataframe/dask_expr/_expr.py:452], in Expr.columns(self)
449 @property
450 def columns(self) -> list:
451 try:
--> 452 return list(self._meta.columns)
453 except AttributeError:
454 if self.ndim == 1:
File [~/miniconda3/envs/py313forge/lib/python3.13/functools.py:1026], in cached_property.__get__(self, instance, owner)
1024 val = cache.get(self.attrname, _NOT_FOUND)
1025 if val is _NOT_FOUND:
-> 1026 val = self.func(instance)
1027 try:
1028 cache[self.attrname] = val
File [~/miniconda3/envs/py313forge/lib/python3.13/site-packages/dask/dataframe/dask_expr/_concat.py:67], in Concat._meta(self)
61 filtered = [
62 meta_nonempty(df._meta)
63 for df in self._frames
64 if df.ndim < 2 or len(df._meta.columns) > 0
65 ]
66 if len(filtered) == 0:
---> 67 return make_meta(meta_nonempty(self._frames[0]._meta))
68 else:
69 return make_meta(
70 methods.concat(
71 filtered,
(...) 77 )
78 )
IndexError: list index out of range
According to git bisect this regression was introduced in #11736 (2025.3.0).