@@ -652,7 +652,7 @@ def read_partition(
652
652
# to categorigal manually for integer types.
653
653
if partitions and isinstance (partitions , list ):
654
654
for partition in partitions :
655
- if df [partition .name ].dtype .name != "category" :
655
+ if len ( partition . keys ) and df [partition .name ].dtype .name != "category" :
656
656
# We read directly from fragments, so the partition
657
657
# columns are already in our dataframe. We just
658
658
# need to convert non-categorical types.
@@ -1123,15 +1123,23 @@ def _collect_dataset_info(
1123
1123
# names of partitioned columns.
1124
1124
#
1125
1125
partition_obj , partition_names = [], []
1126
- if (
1127
- ds .partitioning
1128
- and ds .partitioning .dictionaries
1129
- and all (arr is not None for arr in ds .partitioning .dictionaries )
1130
- ):
1126
+ if ds .partitioning and ds .partitioning .schema :
1131
1127
partition_names = list (ds .partitioning .schema .names )
1132
1128
for i , name in enumerate (partition_names ):
1129
+ dictionary = (
1130
+ ds .partitioning .dictionaries [i ]
1131
+ if ds .partitioning .dictionaries
1132
+ else None
1133
+ )
1133
1134
partition_obj .append (
1134
- PartitionObj (name , ds .partitioning .dictionaries [i ].to_pandas ())
1135
+ PartitionObj (
1136
+ name ,
1137
+ (
1138
+ pd .Series ([], dtype = "object" )
1139
+ if dictionary is None
1140
+ else dictionary .to_pandas ()
1141
+ ),
1142
+ )
1135
1143
)
1136
1144
1137
1145
# Check the `aggregate_files` setting
@@ -1229,7 +1237,7 @@ def _create_dd_meta(cls, dataset_info):
1229
1237
_partitions = [p for p in partitions if p not in physical_column_names ]
1230
1238
if not _partitions :
1231
1239
partitions = []
1232
- dataset_info ["partitions" ] = None
1240
+ dataset_info ["partitions" ] = []
1233
1241
dataset_info ["partition_keys" ] = {}
1234
1242
dataset_info ["partition_names" ] = partitions
1235
1243
elif len (_partitions ) != len (partitions ):
@@ -1266,6 +1274,8 @@ def _create_dd_meta(cls, dataset_info):
1266
1274
if partition_obj :
1267
1275
# Update meta dtypes for partitioned columns
1268
1276
for partition in partition_obj :
1277
+ if not len (partition .keys ):
1278
+ continue
1269
1279
if isinstance (index , list ) and partition .name == index [0 ]:
1270
1280
# Index from directory structure
1271
1281
meta .index = pd .CategoricalIndex (
@@ -1766,14 +1776,17 @@ def _read_table(
1766
1776
for partition in partitions :
1767
1777
if partition .name not in arrow_table .schema .names :
1768
1778
# We read from file paths, so the partition
1769
- # columns are NOT in our table yet.
1779
+ # columns may NOT be in our table yet.
1770
1780
cat = keys_dict .get (partition .name , None )
1771
- cat_ind = np .full (
1772
- len (arrow_table ), partition .keys .get_loc (cat ), dtype = "i4"
1773
- )
1774
- arr = pa .DictionaryArray .from_arrays (
1775
- cat_ind , pa .array (partition .keys )
1776
- )
1781
+ if not len (partition .keys ):
1782
+ arr = pa .array (np .full (len (arrow_table ), cat ))
1783
+ else :
1784
+ cat_ind = np .full (
1785
+ len (arrow_table ), partition .keys .get_loc (cat ), dtype = "i4"
1786
+ )
1787
+ arr = pa .DictionaryArray .from_arrays (
1788
+ cat_ind , pa .array (partition .keys )
1789
+ )
1777
1790
arrow_table = arrow_table .append_column (partition .name , arr )
1778
1791
1779
1792
return arrow_table
0 commit comments