fastparquet icon indicating copy to clipboard operation
fastparquet copied to clipboard

append corrupts _metadata and _common_metadata files

Open zdomokos opened this issue 3 years ago • 6 comments

What happened:

saving dataframe chunks as rowgroups with the code below:

def save_parquet(df, file_name: Path, write_index: bool = True):
    append = file_name.exists() and any(file_name.glob("*.parquet"))
    if append:
        fastparquet.write(str(file_name), df, write_index=write_index, compression='SNAPPY', file_scheme="hive", append=True)
    else:
        fastparquet.write(str(file_name), df, write_index=write_index, compression='SNAPPY', file_scheme="hive")

after 215 appends, the append write corrupts the _metadata and _common_metadata files and cannot append any more.

INFO:root:Processing: L101
ERROR:root:Error at: 216
Traceback (most recent call last):
  File "C:\AVMSDK\Projects\iii_ch\iiich\apps\data_management\etl\adhoc\refinitiv\l1_sampled\a1.py", line 54, in split_file
    save_parquet(df, file_name=out_file, write_index=False)
  File "C:\AVMSDK\Projects\iii_ch\iiich\apps\data_management\etl\adhoc\refinitiv\l1_sampled\a1.py", line 24, in save_parquet
    fastparquet.write(str(file_name), df, write_index=write_index, compression='SNAPPY', file_scheme="hive", append=True)
  File "C:\Users\zdomokos\Anaconda3\envs\iii_ch\lib\site-packages\fastparquet\writer.py", line 1172, in write
    pf = ParquetFile(filename, open_with=open_with)
  File "C:\Users\zdomokos\Anaconda3\envs\iii_ch\lib\site-packages\fastparquet\api.py", line 146, in __init__
    self._parse_header(f, verify)
  File "C:\Users\zdomokos\Anaconda3\envs\iii_ch\lib\site-packages\fastparquet\api.py", line 216, in _parse_header
    for rg in fmd[4]:
TypeError: 'NoneType' object is not iterable

Environment:

  • fastparquet version: 0.8.3
  • Dask version: 2022.7.0
  • Python version: 3.10 anaconda forge
  • Operating System: windows 11
  • Install method (conda, pip, source): conda

zdomokos avatar Sep 15 '22 17:09 zdomokos

Quick question: how do you know that the _common_metadata file is also corrupted? There isn't really any reason for an append operation to touch this file at all (but I haven't checked whether it does or not).

martindurant avatar Sep 19 '22 13:09 martindurant

I should have mentioned an obvious workaround, if you have not time to debug further: just write separate data files in the same directory without bothering to use append on the whole dataset. Without (_common)_metadata, a dataset is just the total of the .parquet files in a directory.

However, was there no error/warning during append before the data became unreadable?

martindurant avatar Sep 26 '22 14:09 martindurant

@yohplala , it's probably time we follow suit from dask, spark, arrow... and explicitly allow for write and append without creating the global _metadata. It could even be the default, although fastparquet is more likely to be able to fit the whole dataset

martindurant avatar Sep 26 '22 15:09 martindurant

how do you know that the _common_metadata file is also corrupted? I compared the content with a before corruption (size ~1.2kb) and after corruption (size ~30b)
was there no error/warning during append before the data became unreadable? The error is pasted into the ticket https://github.com/dask/fastparquet/issues/807#issue-1374878982 I am very busy with a different project, will get back to this issue in a week time.

zdomokos avatar Sep 27 '22 19:09 zdomokos

I am very busy with a different project, will get back to this issue in a week time.

no worries, I appreciate your taking the time to report.

was there no error/warning during append before the data became unreadable?

The error is pasted into the ticket

By the time of the error, the metadata file is apparently already corrupt. I was wondering if there was any warning on the previous iteration.

The next step would be to try to make a reproducer on fake/public data, since, as @yohplala points out in the partner issue, many-iteration appends do normally succeed.

martindurant avatar Sep 27 '22 19:09 martindurant

@yohplala , it's probably time we follow suit from dask, spark, arrow... and explicitly allow for write and append without creating the global _metadata. It could even be the default, although fastparquet is more likely to be able to fit the whole dataset

Hi @martindurant , Ok, I will follow :) Maybe we can discuss this in a specific thread? If dask already does this by default, you likely already have a clear view about it.

My 1st reaction is when ParquetFile instance is created from a list of files: we probably need a new helper to sort parquet file names to feed ParquetFile.__init__(), to circumvent lexicographic filename sorting. and effectively use fastparquet's numbering logic in filename. Something based on row 23 in api.py.

PART_ID = re.compile(r'.*part.(?P<i>[\d]+).parquet$')

Other than that, I am guessing changes are mostly in writer.py, in routines wrriting the _metadata file.

At write time, given a "classical" append does not update metadata in existing files, we could consider by convention that updated metadata have to be read from the last file. Hence the interest in fastparquet's parquet file naming convention, that keeps file order with filenames. (as opposed to pyarrow naming convention I believe)

yohplala avatar Sep 28 '22 06:09 yohplala