gitzwz commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2601733467
> Here's what I did in my particular case to give the general idea:
Thanks to [DieHertz](https://github.com/DieHertz), I tried the demo code
with CPython 3.12 & Debian
jpugliesi commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2557778425
Just to contribute some findings: We also encountered this case where
`pyiceberg`'s scanning`plan_files` was surprisingly slow reading manifest
files from GCS. Switching t
11xor6 commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2489912105
I'm encountering this as well, specifically with methods that rely on
`plan_files`. If there's anything I can do to help or move this forward please
let me know.
--
This is
DieHertz commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2430363061
> If the work is done in Cython avro decoder -> pyarrow recordbatches using
PyArrow Cython API, then that also leaves room to release the GIL for
meaningful threaded concurr
corleyma commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2430215685
>IMO it makes sense to wait until it gets implemented or contribute there,
rather than writing our own general-purpose Avro-To-Arrow code.
We don't need general-purpos
DieHertz commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2429473805
Here I have extracted the code returning `list[dict]` of entries for each
`Manifest` and run it inside the `ThreadPoolExecutor` provided by the
`pyiceberg.utils.concurrent.E
DieHertz commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2428572896
By the way, there is a related goal in Arrow:
https://github.com/apache/arrow/issues/16991
IMO it makes sense to wait until it gets implemented or contribute there,
r
DieHertz commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2428470415
> Had to solve it with cloudpickle.
With `functors.partial` it doesn't seem to be necessary, as I've shown in my
earlier messages.
> yep, optimistically build
DieHertz commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2428451067
So I haven't tried any actual changes yet, but decided to collect some
baseline measurements with py-spy.
First there's pyiceberg 7.0.1 `.inspect.files()` on my big ph
kevinjqliu commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2423385861
> pyiceberg implements its own avro reader/writer using Cython
yep, optimistically build avro decoder, fall back to pure python.
See
https://github.com/apache/ice
corleyma commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2422931184
> Most of the time is spent processing the manifests record-by-record and
converting each record to a dict
I haven't looked at this closely, but if memory serves, pyic
kevinjqliu commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2414941022
BTW I'm not opposed to using `ProcessPoolExecutor`. I'm using curious why
`ThreadPoolExecutor` cant hit the same performance profile
--
This is an automated message fro
kevinjqliu commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2414939042
> Most of the time is spent processing the manifests record-by-record and
converting each record to a dict
Heres a snippet using threads to parallelize both reading
DieHertz commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2414915918
Indeed it is good enough for I/O-bound tasks, but in my understanding this
part is CPU-bound.
I think so because I'm observing close to 100% CPU usage when inside
`pl
kevinjqliu commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2414816052
That's interesting. I thought the `ThreadPoolExexutor` is good for I/O bound
tasks such as reading from the avro manifest files.
If you have a PoC, its something I'd wa
DieHertz commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2414395991
> There's already an ExecutorFactory, do you think we can use that instead
of ProcessPoolExecutor?
The issue with the `ExecutorFactory` is it's using a `ThreadPool
kevinjqliu commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2414173534
As an aside, I think parallelly reading multiple manifests is something we'd
want to reuse at other parts of the program
--
This is an automated message from the Apache
DieHertz commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2414054569
> Would you be interested in working on this issue?
Yes, I'd be happy to contribute back
--
This is an automated message from the Apache Git Service.
To respond to t
sungwy commented on issue #1229:
URL:
https://github.com/apache/iceberg-python/issues/1229#issuecomment-2413901958
Hi @DieHertz - thank you for raising this issue, and for sharing your
benchmarks. I think this is a great idea, that I think we should also consider
applying to other `Inspect
19 matches
Mail list logo