Re: How and where iceberg spark streaming determines latest StreamingOffset upon trigger

2024-04-15 Thread Nirav Patel
pr 14, 2024 at 11:56 AM Prashant Singh wrote: > Hi Nirav, > > > in our case streaming job was stuck for over 3 days > > 3 days seems too much, what exactly were the read limits and how many > files before and after compaction ? Can you describe a bit more on your > ingest

Re: How and where iceberg spark streaming determines latest StreamingOffset upon trigger

2024-04-10 Thread Nirav Patel
(all the files) in that commit to position ourselves for the > next commit." > > for more details please ref : > https://github.com/apache/iceberg/issues/8902 > > Regards, > > Prashant Singh > > On Wed, Apr 10, 2024 at 8:46 AM Nirav Patel wrote: > >> &g

Fwd: How and where iceberg spark streaming determines latest StreamingOffset upon trigger

2024-04-10 Thread Nirav Patel
We are encountering the following issue where spark streaming read job from iceberg table stays stuck after some maintenance jobs (rewrite_data_files and rewrite_manifests) has been ran on parallel on same table. https://github.com/apache/iceberg/issues/10117 I'm trying to understand what creates

Re: Iceberg 1.4/spark3.5 seem to have some breaking issue with spark-connect

2024-02-28 Thread Nirav Patel
r Java environments and Iceberg itself > doesn't do anything fancy around classloading. > > > On Thu, Feb 22, 2024 at 11:15 PM Nirav Patel wrote: > >> Hi Ryan, >> >> I updated the spark-jira I opened with more information I found after >> taking heapdump: >

Re: Iceberg 1.4/spark3.5 seem to have some breaking issue with spark-connect

2024-02-22 Thread Nirav Patel
anks Nirav On Thu, Jan 18, 2024 at 9:46 AM Nirav Patel wrote: > Classloading does seem like an issue while using it with Spark Connect 3.5 > and iceberg >= 1.4 version only though. > > It's weird as I also mentioned in previous email that after adding spark > property (spa

Re: Iceberg 1.4/spark3.5 seem to have some breaking issue with spark-connect

2024-01-18 Thread Nirav Patel
, depending on where a class was loaded for the first time. > > On Fri, Jan 12, 2024 at 5:30 PM Nirav Patel wrote: > >> It seem to happening on executor of SC server as I see the error in >> executor logs. We did verify that there was only one version of >> iceberg-spark-

Re: Iceberg 1.4/spark3.5 seem to have some breaking issue with spark-connect

2024-01-12 Thread Nirav Patel
eTableWithSize` is not a subclass of `Table`, > but it definitely should be. That sort of problem is usually caused by > class loading issues. Can you double-check that you have only one Iceberg > runtime in the Environment tab of your Spark cluster? > > On Tue, Jan 9, 2024 at 4:57 PM Nir

Re: Iceberg 1.4/spark3.5 seem to have some breaking issue with spark-connect

2024-01-09 Thread Nirav Patel
PS - issue doesn't happen if we don't use spark-connect and instead just use spark-shell or pyspark as OP in github said as well. however stacktrace desont seem to point any of the class from spark-connect jar (org.apache.spark:spark-connect_2.12:3.5.0). On Tue, Jan 9, 2024 at 4:52 PM N

Iceberg 1.4/spark3.5 seem to have some breaking issue with spark-connect

2024-01-09 Thread Nirav Patel
Hi, We are testing spark-connect with iceberg. We tried spark 3.5, iceberg 1.4.x versions (all of iceberg-spark-runtime-3.5_2.12-1.4.x.jar) with all the 1.4.x jars we are having following issue when running iceberg queries from sparkSession created using spark-connect (--remote "sc://remote-master

Re: Which time based partition column to choose for streaming

2023-09-08 Thread Nirav Patel
> less sophisticated) don't have trouble querying the table. > > On Fri, Sep 8, 2023 at 8:30 AM Nirav Patel wrote: > >> I am using spark-streaming to ingest live event streams every 5 minutes >> and append into iceberg table. This table can be ingested and processed b

Which time based partition column to choose for streaming

2023-09-08 Thread Nirav Patel
I am using spark-streaming to ingest live event streams every 5 minutes and append into iceberg table. This table can be ingested and processed by a downstream data pipeline or it can be directly used by the end consumer to do data analysis. Every event based datapoint has publish_time, event_tim

Re: do spark structured streaming writer options works with foreachBatch?

2023-09-08 Thread Nirav Patel
ER BY` statement > to your SQL. > > Ryan > > On Wed, Aug 30, 2023 at 10:36 AM Nirav Patel wrote: > >> Should I try "fanout-enabled" option within foreachBatch method where I >> do dataframe.write ? >> >> On Wed, Aug 30, 2023 at 10:29 AM Nirav Pate

Re: do spark structured streaming writer options works with foreachBatch?

2023-08-30 Thread Nirav Patel
Should I try "fanout-enabled" option within foreachBatch method where I do dataframe.write ? On Wed, Aug 30, 2023 at 10:29 AM Nirav Patel wrote: > Hi, > > I am using spark structured streaming and using foreachBatch sink to > append to iceberg dual hidden partition

do spark structured streaming writer options works with foreachBatch?

2023-08-30 Thread Nirav Patel
Hi, I am using spark structured streaming and using foreachBatch sink to append to iceberg dual hidden partitioned table. I got this infamous error about input dataframe or partition needing to be clustered: *Incoming records violate the writer assumption that records are clustered by spec and by

Spark + Iceberg ; How to ensure idempotent updates and deduplication

2023-07-20 Thread Nirav Patel
Hi, I'm using spark structured streaming to append to iceberg partitioned table. I am using custom iceberg catalog (gCP biglake iceberg catalog) to upsert data into iceberg tables that are backed by gcp biglake metastore. There are multiple ways to append streaming data into partition table. One

Re: tradeoffs between serializable vs snapshot isolation for single writer

2023-05-04 Thread Nirav Patel
ritten by another operation), > will error-handling be required. > > Hope that helps > Szehon > > On May 4, 2023, at 12:19 PM, Nirav Patel wrote: > > I am trying to ingest data into iceberg table using spark streaming. There > are no multiple writers to same data at the moment.

tradeoffs between serializable vs snapshot isolation for single writer

2023-05-04 Thread Nirav Patel
I am trying to ingest data into iceberg table using spark streaming. There are no multiple writers to same data at the moment. According to iceberg api