You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@arrow.apache.org by "neerajd12 (via GitHub)" <gi...@apache.org> on 2023/06/26 12:14:01 UTC

[GitHub] [arrow] neerajd12 opened a new issue, #36302: dataset filter doesn't apply custom schema to parquet file if the file has schema with metadata

neerajd12 opened a new issue, #36302:
URL: https://github.com/apache/arrow/issues/36302

   ### Describe the bug, including details regarding any error messages, version, and platform.
   
   ## Pyarrow version
   
   ```
   import pyarrow as pa
   import pyarrow.compute as pc
   import pyarrow.dataset as ds
   
   pa.__version__
   '12.0.0'
   ```
   
   dataset.filter fails with below error when filtering on timestamp. when the schema provided doesn't match schema in parquet file
   
   ```
   ---------------------------------------------------------------------------
   ArrowNotImplementedError                  Traceback (most recent call last)
   Cell In[24], line 2
         1 dataset = ds.dataset('./yellow_tripdata_2009-01.parquet', schema=schema)
   ----> 2 dataset.filter((pc.field("Trip_Pickup_DateTime") <= pc.strptime('2009-01-02', format='%Y-%m-%d', unit='s'))).head(10).to_pandas()
   
   File ~/.local/lib/python3.10/site-packages/pyarrow/_dataset.pyx:702, in pyarrow._dataset.Dataset.head()
   
   File ~/.local/lib/python3.10/site-packages/pyarrow/_dataset.pyx:3495, in pyarrow._dataset.Scanner.head()
   
   File ~/.local/lib/python3.10/site-packages/pyarrow/error.pxi:144, in pyarrow.lib.pyarrow_internal_check_status()
   
   File ~/.local/lib/python3.10/site-packages/pyarrow/error.pxi:121, in pyarrow.lib.check_status()
   
   ArrowNotImplementedError: Function 'equal' has no kernel matching input types (timestamp[s], string)
   
   ```
   ### Sample data
   
   https://d37ci6vzurychx.cloudfront.net/trip-data/yellow_tripdata_2009-01.parquet
   
   ### Failing code
   ```
   schema = pa.schema(
       pa.struct({
           "vendor_name": pa.string(),
           "Trip_Pickup_DateTime": pa.timestamp('s'),
           "Trip_Dropoff_DateTime": pa.timestamp('s'),
           "Passenger_Count": pa.int64(),
           "Trip_Distance": pa.float64(),
           "Start_Lon": pa.float64(),
           "Start_Lat": pa.float64(),
           "Rate_Code": pa.float64(),
           "store_and_forward": pa.float64(),
           "End_Lon": pa.float64(),
           "End_Lat": pa.float64(),
           "Payment_Type": pa.string(),
           "Fare_Amt": pa.float64(),
           "surcharge": pa.float64(),
           "mta_tax": pa.float64(),
           "Tip_Amt": pa.float64(),
           "Tolls_Amt": pa.float64(),
           "Total_Amt": pa.float64()
   }))
   
   dataset = ds.dataset('./yellow_tripdata_2009-01.parquet', schema=schema)
   
   dataset.filter(
   (
      pc.field("Trip_Pickup_DateTime") <= pc.strptime('2009-01-02', format='%Y-%m-%d', unit='s'))
   
   ).head(10).to_pandas()
   
   # check schema
   
   dataset.schema
   
   vendor_name: string
   Trip_Pickup_DateTime: timestamp[s]
   Trip_Dropoff_DateTime: timestamp[s]
   Passenger_Count: int64
   Trip_Distance: double
   Start_Lon: double
   Start_Lat: double
   Rate_Code: double
   store_and_forward: double
   End_Lon: double
   End_Lat: double
   Payment_Type: string
   Fare_Amt: double
   surcharge: double
   mta_tax: double
   Tip_Amt: double
   Tolls_Amt: double
   Total_Amt: double
   
   # check metadata 
   dataset.schema.metadata
   '  '
   ```
   ### Remove the schema and the filter works as string.
   ```
   dataset = ds.dataset('./yellow_tripdata_2009-01.parquet')
   dataset.filter((pc.field("Trip_Pickup_DateTime") <= '2009-01-02')).head(10).to_pandas()
   ```
   ```
   dataset.schema
   
   vendor_name: string
   Trip_Pickup_DateTime: string
   Trip_Dropoff_DateTime: string
   Passenger_Count: int64
   Trip_Distance: double
   Start_Lon: double
   Start_Lat: double
   Rate_Code: double
   store_and_forward: double
   End_Lon: double
   End_Lat: double
   Payment_Type: string
   Fare_Amt: double
   surcharge: double
   mta_tax: double
   Tip_Amt: double
   Tolls_Amt: double
   Total_Amt: double
   -- schema metadata --
   pandas: '{"index_columns": [{"kind": "range", "name": null, "start": 0, "' + 2473
   
   ```
   ```
   dataset.schema.metadata
   
   {b'pandas': b'{"index_columns": [{"kind": "range", "name": null, "start": 0, "stop": 14092413, "step": 1}], "column_indexes": [{"name": null, "field_name": null, "pandas_type": "unicode", "numpy_type": "object", "metadata": {"encoding": "UTF-8"}}], "columns": [{"name": "vendor_name", "field_name": "vendor_name", "pandas_type": "unicode", "numpy_type": "object", "metadata": null}, {"name": "Trip_Pickup_DateTime", "field_name": "Trip_Pickup_DateTime", "pandas_type": "unicode", "numpy_type": "object", "metadata": null}, {"name": "Trip_Dropoff_DateTime", "field_name": "Trip_Dropoff_DateTime", "pandas_type": "unicode", "numpy_type": "object", "metadata": null}, {"name": "Passenger_Count", "field_name": "Passenger_Count", "pandas_type": "int64", "numpy_type": "int64", "metadata": null}, {"name": "Trip_Distance", "field_name": "Trip_Distance", "pandas_type": "float64", "numpy_type": "float64", "metadata": null}, {"name": "Start_Lon", "field_name": "Start_Lon", "pandas_type": "float64", "
 numpy_type": "float64", "metadata": null}, {"name": "Start_Lat", "field_name": "Start_Lat", "pandas_type": "float64", "numpy_type": "float64", "metadata": null}, {"name": "Rate_Code", "field_name": "Rate_Code", "pandas_type": "float64", "numpy_type": "float64", "metadata": null}, {"name": "store_and_forward", "field_name": "store_and_forward", "pandas_type": "float64", "numpy_type": "float64", "metadata": null}, {"name": "End_Lon", "field_name": "End_Lon", "pandas_type": "float64", "numpy_type": "float64", "metadata": null}, {"name": "End_Lat", "field_name": "End_Lat", "pandas_type": "float64", "numpy_type": "float64", "metadata": null}, {"name": "Payment_Type", "field_name": "Payment_Type", "pandas_type": "unicode", "numpy_type": "object", "metadata": null}, {"name": "Fare_Amt", "field_name": "Fare_Amt", "pandas_type": "float64", "numpy_type": "float64", "metadata": null}, {"name": "surcharge", "field_name": "surcharge", "pandas_type": "float64", "numpy_type": "float64", "metadata"
 : null}, {"name": "mta_tax", "field_name": "mta_tax", "pandas_type": "float64", "numpy_type": "float64", "metadata": null}, {"name": "Tip_Amt", "field_name": "Tip_Amt", "pandas_type": "float64", "numpy_type": "float64", "metadata": null}, {"name": "Tolls_Amt", "field_name": "Tolls_Amt", "pandas_type": "float64", "numpy_type": "float64", "metadata": null}, {"name": "Total_Amt", "field_name": "Total_Amt", "pandas_type": "float64", "numpy_type": "float64", "metadata": null}], "creator": {"library": "pyarrow", "version": "8.0.0"}, "pandas_version": "1.2.3"}'}
   
   ```
   
   ### Filter works with explicit schema if applied on table.
   ```
   dataset.head(10).filter(
   (
      pc.field("Trip_Pickup_DateTime") <= pc.strptime('2009-01-02', format='%Y-%m-%d', unit='s')
   )
   ).to_pandas()
   ```
   
   ### Filter also works if I load the file with schema and save it to new parquet file and then load again with the same schema. 
   
   1. load file with custom schema and save to Parquet
   ````
   dataset = ds.dataset('./yellow_tripdata_2009-01.parquet', schema=schema)
   
   import pyarrow.parquet as pp
   pp.write_to_dataset(dataset, root_path='./test')
   ````
   2. Load new file with same schema and filter
   
   ````
   dataset = ds.dataset('./test/8c0673b61cc34b4e8094dc1cb11534bd-0.parquet', schema=schema)
   
   dataset.filter(
              (
               pc.field("Trip_Pickup_DateTime") <= pc.strptime('2009-01-02', format='%Y-%m-%d', unit='s')
             )
   ).head(10).to_pandas()
   
   ````
   
   Dataset.filter should work if custom schema is provided with data types different than parquet metadata
   
   
   
   
   ### Component(s)
   
   Parquet, Python


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@arrow.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] neerajd12 commented on issue #36302: [Python][Parquet] dataset filter doesn't apply custom schema to parquet file if the file has schema with metadata

Posted by "neerajd12 (via GitHub)" <gi...@apache.org>.
neerajd12 commented on issue #36302:
URL: https://github.com/apache/arrow/issues/36302#issuecomment-1631905055

   thanks @westonpace thats exactly what it is. parquet metadata has the field as string while the schema and filter have timestamp. The difference in filter behaviour between dataset and table is what got me questioning. For table it seems the casting happens before filter is applied while for dataset it still depends on parquet metadata as you said. I might be wrong though 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] mapleFU commented on issue #36302: [Python][Parquet] dataset filter doesn't apply custom schema to parquet file if the file has schema with metadata

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on issue #36302:
URL: https://github.com/apache/arrow/issues/36302#issuecomment-1616410058

   Hmmm parquet internal would tent to use ARROW SCHEMA in metadata, so you want to overwrite with own schema here?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on issue #36302: [Python][Parquet] dataset filter doesn't apply custom schema to parquet file if the file has schema with metadata

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on issue #36302:
URL: https://github.com/apache/arrow/issues/36302#issuecomment-1648014338

   I'll leave it open, though I might reword the title a bit.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on issue #36302: [Python][Parquet] dataset filter doesn't apply custom schema to parquet file if the file has schema with metadata

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on issue #36302:
URL: https://github.com/apache/arrow/issues/36302#issuecomment-1631568352

   What is the type of the field in the parquet file itself?  A string?  If so, then the parquet metadata is going to be storing min/max using lexical comparison which might not be the same min/max you'd get doing temporal comparison.  So I don't know if it would be valid to use these statistics for filtering.
   
   I think I might not be understanding though.  A few questions:
   
   What is the type of the column in the file itself (e.g. in the parquet metadata, is this string?)
   What is the type of the column in the dataset schema (looks like timestamp)
   What is the type of literal in the filter (looks like timestamp)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on issue #36302: [Python][Parquet] dataset filter doesn't apply custom schema to parquet file if the file has schema with metadata

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on issue #36302:
URL: https://github.com/apache/arrow/issues/36302#issuecomment-1632861575

   Yes, the casting will happen after we read the column into memory.  Something like (this is just pseudocode)...
   
   ```
   # column is a string array
   column = read_from_parquet_file(col_index)
   desired_type = dataset_schema.types[col_index]
   if column.type != desired_type:
     # now column is a timestamp array
     column = cast(column, desired_type)
   ...
   table = build_table_from_columns(...)
   ...
   # filter happens down here
   ```
   
   However, if you apply the filter to a dataset, then we are going to try and use it for pushdown filtering.  So if we zoom out a little on the above pseudocode...
   
   ```
   metadata = get_parquet_metadata()
   for simple_filter_clause in filter: # e.g. things like x > 0
     for row_group in metadata.row_groups:
       row_group_stats = row_group.statistics
       # Casting error is being thrown here
       if simple_filter_clause.cannot_match(row_group_stats):
         skip_row_group()
   
   # column is a string array
   column = read_from_parquet_file(col_index)
   desired_type = dataset_schema.types[col_index]
   if column.type != desired_type:
     # now column is a timestamp array
     column = cast(column, desired_type)
   ...
   table = build_table_from_columns(...)
   ...
   # filter happens down here
   ```
   
   So we cannot use the filter for pushdown directly.  I don't think we can safely cast it.  We _could_ just skip this filter (exclude it from pushdown) and then allow it to be applied later on.  So I think it is possible to get better behavior here.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] neerajd12 commented on issue #36302: [Python][Parquet] dataset filter doesn't apply custom schema to parquet file if the file has schema with metadata

Posted by "neerajd12 (via GitHub)" <gi...@apache.org>.
neerajd12 commented on issue #36302:
URL: https://github.com/apache/arrow/issues/36302#issuecomment-1640354140

   thanks for the clarification @westonpace. I'll let you decide if you want to keep this bug or close it


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org