Open svenatarms opened 1 year ago
This was introduced by the solution for https://github.com/apache/arrow/issues/33448. It looks like we made a backwards incompatible change here which is unfortunate.
NOTICE: the Equal Sign = is URL encoded for the request, but won't become %3D on S3 filesystem. That means, the URL encoded equal sign = seems to be interpreted correctly
I'm not sure it's relevant to my greater point but I don't think the Equal Sign is encoded in the request:
\
product=My%20Fancy%20Product/date=2023-01-10/a724b93c251a486b897eb7b151c622bd-0.parquet\
Unfortunately, it is a tricky problem. The encoding here is not to support HTTP requests (in S3 all these paths go into the HTTP body and are not part of the URI) but instead to support two different problems.
First, we need to support the concept of hive partitioning. In hive partitioning there is a special meaning behind the =
and /
characters because {x:3, y:7}
gets encoded as x=3/y=7
. This caused issues if the hive keys or hive values had /
or =
and so the solution was to encode the value (in retrospect I suppose we should be encoding the keys as well).
Second, most filesystems only support a reserved set of characters. Note that even S3 doesn't fully support space:
Space – Significant sequences of spaces might be lost in some uses (especially multiple spaces)
To solve this problem we are now using uriparser's RFC3986 encode function. This is an imprecise approach. It is converting more characters than strictly needed in all cases. However, there is some precedence for this (Spark) and I fear that anything more narrow would be too complex and/or unintuitive.
I'd support a PR to turn encoding on and off entirely (either as an argument to a partitioning object or part of the write_dataset options). The default could be on and then users could choose to disable this feature. Users are then responsible for ensuring their partitioning values consist of legal characters for their filesystem.
Thanks for looking into the issue and tracking down the cause. I like the idea, to be able to turn encoding off for backwards compatibility. On our side we'll change the behavior of our application to ensure that partitioning values won't have characters like whitespace anymore.
I've labeled this good-first-issue
in case anyone wants to take a look at it. I'm happy to provide more context. The steps we need would be:
HivePartitioningOptions
to disable / enable URI escaping (in src/arrow/dataset/partition.h)HivePartitioning::FormatValues
in src/arrow/dataset/partition.cc)HivePartitioning
class in dataset.pyx, just look for null_fallback
which is a similar option)I took a look at this a bit, and this is not a good fit for me to dive into right now. Maybe some other issue in the future.
I am interested to work on this issue. Can you provide some guidance and assistance in resolving this.
@jainamshah102 that's great. You will first want to get a C++ development environment setup for Arrow and make sure you can build and run the tests (this is a complex task). The C++ development guide should help. In addition you might want to look at the first PR guide if you have not made a PR for Arrow before.
Once everything is building correctly you will want to create a unit test that reproduces this issue. This would probably be in cpp/src/arrow/dataset/partition_test.cc
. Some general context:
The class arrow::dataset::Partitioning
is a pure virtual class (e.g. an interface) that turns paths into expressions and back. For example, a directory partitioning could turn the path /7/12
into the expression x==7 && y == 12
. The hive partitioning would turn that same expression into the path /x=7/y=12
(hive partitioning is key=value
style and directory partitioning omits the keys).
This is done with two methods Format
and Parse
. The problem here is with the HivePartitioning
class. Currently, in Format
, we url encode the path. Then, in parse, we url decode the path. The ask is to add a new option to HivePartitioning
(perhaps named escape_paths
which, if set to true, will use the current behavior and, if set to false, will skip the url encoding/decoding.
Let me know if you run into more problems.
Hi @jainamshah102 might you be still interesting in tackling this issue?
take
This is the case for GCS as well as S3. we just encountered this when updating from pyarrow 10.0.1 to 14.0.2 but is present in all versions from 11.0.0 onwards. it is present for both the GCSFS library and the pyarrow.fs.GcsFileSystem example code:
import gcsfs
import pyarrow as pa
import pyarrow.fs as pafs
import pyarrow.dataset as ds
import datetime
fs = gcsfs.GCSFileSystem()
data = {
"some_timestamp": [datetime.datetime.now(tz=datetime.timezone.utc) - datetime.timedelta(days=1),
datetime.datetime.now(tz=datetime.timezone.utc) - datetime.timedelta(days=2),
datetime.datetime.now(tz=datetime.timezone.utc) - datetime.timedelta(days=3)],
"value1": ["hello", "world", "foo"],
"value2": [123, 456, 789]
}
schema = pa.schema([
pa.field("some_timestamp", pa.timestamp("ms")),
pa.field("value1", pa.string()),
pa.field("value2", pa.int64())
])
result_pya_table = pa.Table.from_pydict(data, schema=schema)
ds.write_dataset(
data=result_pya_table,
base_dir=f"adam_ryota_data/pyarrowfstest/2023.12.2.post1-10.0.1/",
format='parquet',
partitioning=["some_timestamp"],
partitioning_flavor='hive',
existing_data_behavior='overwrite_or_ignore',
basename_template="data-{i}.parquet",
filesystem=fs
)
10.0.1 results in:
11.0.0 or higher results in:
note that it is not part of the overall uri being encoded. only the data within the dataset is affected by this. when using the hive partition as part of the path:
import gcsfs
import pyarrow as pa
import pyarrow.fs as pafs
import pyarrow.dataset as ds
import datetime
fs = gcsfs.GCSFileSystem()
data = {
# "some_timestamp": [datetime.datetime.now(tz=datetime.timezone.utc) - datetime.timedelta(days=1),
# datetime.datetime.now(tz=datetime.timezone.utc) - datetime.timedelta(days=2),
# datetime.datetime.now(tz=datetime.timezone.utc) - datetime.timedelta(days=3)],
"value1": ["hello", "world", "foo"],
"value2": [123, 456, 789]
}
schema = pa.schema([
# pa.field("some_timestamp", pa.timestamp("ms")),
pa.field("value1", pa.string()),
pa.field("value2", pa.int64())
])
result_pya_table = pa.Table.from_pydict(data, schema=schema)
ds.write_dataset(
data=result_pya_table,
base_dir=f"adam_ryota_data/manualhive/2023.12.2.post1-10.0.1/some_timestamp=2024-04-07 11:13:27.169/",
format='parquet',
# partitioning=["some_timestamp"],
# partitioning_flavor='hive',
existing_data_behavior='overwrite_or_ignore',
basename_template="data-{i}.parquet",
filesystem=fs
)
even in 11.0.0+ the data is written as expected.
Describe the bug, including details regarding any error messages, version, and platform.
Environment
OS: Windows/Linux Python: 3.10.10 s3fs: from 2022.7.1 to 2023.3.0 (doesn't matter) S3 Backend: MinIO / Ceph (doesn't matter)
Description
Version 11.0.0 of pyarrow introduced an unexpected behavior when uploading Parquet Files to an S3 Bucket (using s3fs.S3FileSystem), if the Path to the Parque File contains white spaces. White Spaces will be replaced by URL encoded Syntax %20 e.g: A Directory Name like:
becomes
on S3 filesystem. NOTICE: the Equal Sign
=
is URL encoded for the request, but won't become %3D on S3 filesystem. That means, the URL encoded equal sign=
seems to be interpreted correctlyExample Code
Example Result
Expected Result (using pyarrow 10.0.1)
Debug output
Actual result (using pyarrow 11.0.0)
Debug output
The difference in the debug output is the line starting with botocore.parsers - DEBUG - Response body:. In the XML Part, the Node
<Key></Key>
contains an URL Encoded string (pyarrow 11.0.0) vs. "human readable" String (pyarrow 10.0.1). But the URL encoded string is not URL encoded at all, as mentioned before e.g. the equal sign=
is intepreted as expected.It seems, that the URL encode/decode(?) isn't done correctly at all?
Wild guess of mine: This behavior might be introduced by: #33598 and/or #33468
Thanks, Sven
Component(s)
Python