Join GitHub today
GitHub is home to over 20 million developers working together to host and review code, manage projects, and build software together.
Removing the "has_nulls" option from to_parquet, or set default to True? #2177
Comments
|
Bumping this issue. If files are being written with nonzero probability of being read by Spark, Hive, Drill Impala, or any other SQL-on-Hadoop system then the default options are producing files whose results in those systems may be incorrect. If using |
|
I don’t see how it can be incorrect to save the value NaN as opposed to NULL in a parquet file, when the input is a pandas float column in which, as far as I know, there is no difference between the two. |
|
The NaN values in pandas are semantically NULL; when you write them as NaN they are no longer considered to be NULL by other systems. So
will yield different results from |
|
"The NaN values in pandas are semantically NULL" - so how does one achieve a semantic NaN? |
|
You cannot -- NaN is null:
|
|
I think that the following is a simple and motivational use case in favor of treating nans/nulls dilligently (what @wesm describes). I haven't thought about this topic enough to have a strong opinion, but I think that this example might be useful. Imagine you read the following CSV file
Pandas treats the missing values here properly as missing values >>> df = pd.read_csv('myfile.csv')
>>> df.b.sum()
7But now we move the data to parquet and then read with Spark and try the same thing: >>> df.agg({'b': 'sum'})
NaNSo either Spark would need to interpret NaN's as Nulls (which I suspect is unlikely and probably incorrect from their perspective) or we are transforming the data so that it produces different results on different tools. |
|
Overmore, I believe that RepetitionType::REQUIRED should be avoided altogether, since if you append data to a dataset written initially by Dask, you may have a schema conflict (many systems use OPTIONAL exclusively). |
wesm
referenced
this issue
in pandas-dev/pandas
Apr 9, 2017
Open
ENH: add to/from_parquet with pyarrow & fastparquet #15838
|
Here's an example of writing a DataFrame with nulls with the default options and then querying it with Impala: Write the file: import dask.dataframe as dd, numpy as np, pandas as pd
df = pd.DataFrame({'key': [u'a', u'b', u'c'] * 100,
'value': np.random.randn(300)})
df['value'][::4] = np.nan
ddf = dd.from_pandas(df, npartitions=1)
ddf.to_parquet('proper_foo.parquet', write_index=False)Do a groupby with pandas df.groupby('key').value.sum()
Out[11]:
key
a 13.728379
b -0.028474
c -2.610579
Name: value, dtype: float64Create an EXTERNAL TABLE in Impala referencing the new Parquet file: import ibis
import pandas as pd
hdfs = ibis.hdfs_connect('localhost', port=5070)
con = ibis.impala.connect('localhost', port=21050, hdfs_client=hdfs)
ibis.options.verbose = True
path = '/tmp/fastparquet-test'
hdfs.mkdir(path)
hdfs.put('/tmp/fastparquet-test/0.parq', 'proper_foo.parquet/part.0.parquet')Create a table referencing the file: >>> pf = con.parquet_file(path)
SHOW DATABASES LIKE '__ibis_tmp'
CREATE EXTERNAL TABLE __ibis_tmp.`__ibis_tmp_524b11f116a04d29add65f1bd9bfee8c`
LIKE PARQUET '/tmp/fastparquet-test/0.parq'
STORED AS PARQUET
LOCATION '/tmp/fastparquet-test'Now do the same groupby, which yields NaN results: cur = con.con.execute('select key, sum(value) from __ibis_tmp.`__ibis_tmp_524b11f116a04d29add65f1bd9bfee8c` group by key')
cur.fetchall()
select key, sum(value) from __ibis_tmp.`__ibis_tmp_524b11f116a04d29add65f1bd9bfee8c` group by key
Out[18]:
[('b', nan), ('a', nan), ('c', nan)]For the sake of compatibility, in Dask I strongly recommend:
I think it's OK if you want to have a special option for fastparquet users to disable compatibility but give better performance, but it should not be the default. |
|
@cpcloud your example has a mistake because the type of |
|
Removed, the example works with proper types. |
|
Checking in on the status here. @martindurant did you have any thoughts? |
This was referenced Apr 14, 2017
wesm
added a commit
to wesm/dask
that referenced
this issue
Apr 17, 2017
|
|
wesm |
b5d551e
|
wesm
added a commit
to wesm/dask
that referenced
this issue
Apr 17, 2017
|
|
wesm |
a1370e3
|
|
Sorry for my relative silence on this issue. Here are a couple of further thoughts:
I am OK to say that performance comes second to compatibility in general, as long as the loss is not too big, if we are conscious that that is the decision being made. On the point that all schema fields should be OPTIONAL: this sounds pretty odd to me, no designer of a database table would agree that the primary key should be optional, even if in practice it never is. |
Mathematically, you are right, but pandas is not a library for scientific computing; it's for business analytics and data analysis (where people don't seem to care about NaNs). In 2008 there was substantial precedent in the financial analytics world for using NaN for performance and convenience (null propagation in hardware, also in operations involving floats and ints together). In pandas 2.0, like in databases and Spark / other big datasystems, there will be a semantic distinction between NaN and null, but that is still a good ways off. If you have an application where NaN is a meaningful value, I don't recommend using pandas at all -- better to use NumPy directly.
This is a practical argument based on empirical Parquet use, where the columns are always nullable. Hive and Impala do not have
then some downstream readers of those datasets may raise errors due to some of the files having "different" schemas from the others. |
|
Ah, I misunderstood your earlier statement about appends, I had honestly not thought of people using different tools to append to the same dataset. Does that happen? |
|
In the case of Hive or Impala, the Hive metastore allows you to create partitions that can point to any directory in HDFS -- it's not unusual for different ETL processes to dump data into a given logical dataset. See e.g. https://blog.cloudera.com/blog/2015/11/how-to-ingest-and-query-fast-data-with-impala-without-kudu/ |
|
@wesm' arguments sound reasonable to me. If it's important that people be able to choose performance presets easily, perhaps this should be handled by a dialect or compatibility setting in fastparquet, so that users don't have to bother about individual configuration options such as |
|
OK, I'm convinced, I'll make the change. This will be for has_nulls=True for both fastparquet and dask's to_parquet. What would be a reasonable name for the current behaviour (which is no longer default, and so should not be None): "fast_but_incompatible" ? |
|
I recommend naming after the semantics like nan_as_null or has_nulls,
rather than performance adjectives like fast.
…
|
|
It seems like this is a candidate for a pass-through parameter to the engine. I'd like to provide the option to write files with pyarrow also |
wesm
added a commit
to wesm/dask
that referenced
this issue
Apr 22, 2017
|
|
wesm |
6fb0294
|
mrocklin
added a commit
that referenced
this issue
Apr 28, 2017
|
|
wesm + mrocklin |
68f9e41
|
|
Can we change this default option in Dask? I tried in 68f9e41 but it broke the test suite, so more work is required |
|
Yes, agree that this can be True (but please do not remove). |
|
Has this been resolved? |
|
This is also in #2365 , which I now see has a py2 issue yet to be resolved; I'll get on that. |
wesm commentedApr 5, 2017
•
edited
In dask.dataframe.io.to_parquet:
I am looking in the logic in:
https://github.com/dask/fastparquet/blob/master/fastparquet/writer.py#L626
My understand of this is that the default behavior is to not write nulls in the Parquet definition levels (i.e.
RepetitionType:REQUIRED) for float32/64 types. I'm concerned this is a dangerous default setting because the Parquet files written will not be readable in general by other systems that use Parquet files. Would you be open to removing this option altogether (respecting pandas's NULL semantics) or changing its default value toTrue?cc @martindurant