Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2024-01-26 Thread via GitHub
Fokko merged PR #8980: URL: https://github.com/apache/iceberg/pull/8980 -- 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-unsubscr...@iceberg.apach

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2024-01-08 Thread via GitHub
singhpk234 commented on PR #8980: URL: https://github.com/apache/iceberg/pull/8980#issuecomment-1881801609 > I'm also very skeptical of counting the added files, and I think we might want to remove that piece of logic (in a separate PR). +1 on this @Fokko other than this no further co

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-12-27 Thread via GitHub
cccs-jc commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1437294256 ## core/src/main/java/org/apache/iceberg/MicroBatches.java: ## @@ -92,7 +92,7 @@ private static List> indexManifests( for (ManifestFile manifest : manifestFiles

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-12-22 Thread via GitHub
Fokko commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1435350938 ## core/src/main/java/org/apache/iceberg/MicroBatches.java: ## @@ -92,7 +92,7 @@ private static List> indexManifests( for (ManifestFile manifest : manifestFiles)

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-12-22 Thread via GitHub
cccs-jc commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1435311105 ## core/src/main/java/org/apache/iceberg/MicroBatches.java: ## @@ -92,7 +92,7 @@ private static List> indexManifests( for (ManifestFile manifest : manifestFiles

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-12-22 Thread via GitHub
Fokko commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1435150752 ## core/src/main/java/org/apache/iceberg/MicroBatches.java: ## @@ -92,7 +92,7 @@ private static List> indexManifests( for (ManifestFile manifest : manifestFiles)

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-12-13 Thread via GitHub
cccs-jc commented on PR #8980: URL: https://github.com/apache/iceberg/pull/8980#issuecomment-1853964957 so I did more digging. On our production tables I search for all manifests which have a `existing_data_files_count > 0` and `added_data_files_count > 0` and I find none. This leads me to

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-12-11 Thread via GitHub
singhpk234 commented on PR #8980: URL: https://github.com/apache/iceberg/pull/8980#issuecomment-1850795022 > Can you explain what is the purpose of using existingFilesCount here ? I am not fully aware of this logically i totally agree with you it makes no sense to keep it but what i a

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-12-07 Thread via GitHub
cccs-jc commented on PR #8980: URL: https://github.com/apache/iceberg/pull/8980#issuecomment-1845659813 > @cccs-jc i mean let's have changes for 3.5 with it's test only in 3.5 and we can backport the change with it's test in lower spark version like 3.4 and 3.3, 3.4 test failures are expect

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-12-05 Thread via GitHub
singhpk234 commented on PR #8980: URL: https://github.com/apache/iceberg/pull/8980#issuecomment-1841776563 @cccs-jc i mean let's have changes for 3.5 with it's test only in 3.5 and we can backport the change with it's test in lower spark version like 3.4 and 3.3, 3.4 test failures are expec

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-28 Thread via GitHub
cccs-jc commented on PR #8980: URL: https://github.com/apache/iceberg/pull/8980#issuecomment-1829796066 > @singhpk234 As you recommended I removed the 3.4 implementation and only kept one version 3.5. > > However, now the test cases for 3.4 are failing. Any idea how to fix this. Shou

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-23 Thread via GitHub
cccs-jc commented on PR #8980: URL: https://github.com/apache/iceberg/pull/8980#issuecomment-1824466349 @singhpk234 As you recommended I removed the 3.4 implementation and only kept one version 3.5. However, now the test cases for 3.4 are failing. Any idea how to fix this. Should I

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-22 Thread via GitHub
cccs-jc commented on PR #8980: URL: https://github.com/apache/iceberg/pull/8980#issuecomment-1823493615 > @cccs-jc i would recommend to make the changes 1 spark version at a time and then create back-port pr, i am not sure what is the preferred though but checking it 1 version at a time hel

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-22 Thread via GitHub
cccs-jc commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1402704727 ## spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java: ## @@ -392,8 +392,15 @@ public Offset latestOffset(Offset startOffset, Rea

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-22 Thread via GitHub
cccs-jc commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1402703648 ## spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java: ## @@ -406,6 +413,30 @@ public Offset latestOffset(Offset startOffset, Rea

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-14 Thread via GitHub
singhpk234 commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1393421486 ## core/src/main/java/org/apache/iceberg/MicroBatches.java: ## @@ -92,7 +92,7 @@ private static List> indexManifests( for (ManifestFile manifest : manifestFi

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-14 Thread via GitHub
singhpk234 commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1393422572 ## spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java: ## @@ -392,8 +392,15 @@ public Offset latestOffset(Offset startOffset,

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-08 Thread via GitHub
cccs-jc commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1386580961 ## core/src/main/java/org/apache/iceberg/MicroBatches.java: ## @@ -92,7 +92,7 @@ private static List> indexManifests( for (ManifestFile manifest : manifestFiles

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-07 Thread via GitHub
singhpk234 commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1385993967 ## spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java: ## @@ -309,6 +309,19 @@ private static StreamingOffset determineStarti

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-07 Thread via GitHub
singhpk234 commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1385993310 ## core/src/main/java/org/apache/iceberg/MicroBatches.java: ## @@ -92,7 +92,7 @@ private static List> indexManifests( for (ManifestFile manifest : manifestFi

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-07 Thread via GitHub
cccs-jc commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1384895852 ## spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java: ## @@ -309,6 +309,19 @@ private static StreamingOffset determineStartingO

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-06 Thread via GitHub
nastra commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1383492951 ## spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java: ## @@ -497,6 +500,67 @@ public void testReadStreamWithSnapshotTypeO

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-06 Thread via GitHub
cccs-jc commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1383313970 ## core/src/main/java/org/apache/iceberg/MicroBatches.java: ## @@ -92,7 +92,7 @@ private static List> indexManifests( for (ManifestFile manifest : manifestFiles

Re: [PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-03 Thread via GitHub
singhpk234 commented on code in PR #8980: URL: https://github.com/apache/iceberg/pull/8980#discussion_r1382019127 ## spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java: ## @@ -392,8 +405,15 @@ public Offset latestOffset(Offset startOffset,

[PR] Core: Iceberg streaming streaming-skip-overwrite-snapshots SparkMicroBatchStream only skips over one file per trigger [iceberg]

2023-11-03 Thread via GitHub
cccs-jc opened a new pull request, #8980: URL: https://github.com/apache/iceberg/pull/8980 Closes #8902 @singhpk234 I have fixed the issue https://github.com/apache/iceberg/issues/8902. Could you have a look at it. -- This is an automated message from the Apache Git Service. To re