-
Notifications
You must be signed in to change notification settings - Fork 2.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SUPPORT] Deltastreamer errors ingesting kafka-streams topics (transactional / exactly_once producers) #8258
Comments
More observations on this
As seen in the logs attached above, we suspect that the underlying Spark KafkaRDD library is confused about the transaction marker when it is the last record in the topic |
I am able to reproduce your issue with HoodieDeltaStreamer. We are looking into this issue. meanwhile you can use spark structured streaming to read the data from transactional topic and write to Hudi. I am not seeing any error while using the structured streaming. Below is the code you can refer. |
Thanks @ad1happy2go We are also leaning towards using either Spark Structured streaming or Batch processing as a workaround. We have some working examples for both cases now, thanks again for sharing your example code too Could you confirm if we need to have a separate process for compaction when using Spark Structured streaming for writing to Hudi? |
@WarFox |
@danielfordfc JIRA created for the tracking the fix - https://issues.apache.org/jira/browse/HUDI-6297 |
@ad1happy2go @nsivabalan is this absolutely not going to happen anytime soon? It's preventing us from directly ingesting a large majority of our Kafka topics in our organisation and i'm very surprised it's not a more widely experienced issue, given its a common feature of topics produced through kafka-streams applications |
Describe the problem you faced
Deltastreamer, when being run on a transactional topic (one being produced to by a transactional producer, like kafka-streams) is unable to be read.
Full stack traces will be linked below.
Our configuration is as follows:
We've identified that this is because the
get()
/compactedNext()
method used by theInternalKafkaDataConsumer
is failing to poll records/batches for these transactional topics..If we create a simple non-compacted topic that we'll be writing to non-transactionally, and one transactionally:
Produced 16 message non-transactionally -- as you can see the end/"next available" offset is the one after the last offset containing data in each partition
Produced 16 message transactionally -- as you can see the end/"next available" offset is 2 more than the last offset containing data in each partition, because the end of that batch of write placed a commit marker/offset message in each partition
And we see the stack traces mentioned at the bottom:
hoodie-allow-consecutive-off-false.log
hoodie-allow-consecutive-off-true.log
Notably
Extra Information gathered from running this locally
Dive into our local example showing how we get the poll of [topic-partition] 5, followed by a poll of [] 0, followed by the crash when AllowNonConsecutiveOffsets=true
Interestingly, in the below, when setting AllowNonConsecutiveOffsets=False, we see that the initial poll for the partition 0 (which from the above screenshot, showed offset 0->4 being valid messages, offset 5 being the commit marker, has it poll those first 5 messages, then fail on the next poll.
If we create another topic with one partition and write a single batch of 16 records transactionally (so 0->15 is data, 16 is commit marker, end of topic is 17), we see similar behaviour.
my-transactional-topic-single-partition.log
If we remove the possibility that it might be crashing because the endOffset is the "invisible" marker that it can't read, by adding another 16 records (putting 17->32 as data, 33 as the marker and 34 as endOffset), we see a similar issue with the following:
my-transactional-topic-single-partition-32-msgs.log
Changing to
AllowNonConsecutiveOffsets=true
on the above topic yields the following:Stack trace for the above:
my-transactional-topic-single-partition-allownonconsecutiveoffsetsTrue.log
Answers Required
So we know what the problem is, we are just unsure on how to fix.
We've taken this to the hudi office hours before and the host suggested to ask @yihua for advice.
Usual Environment in Production, but all of this has been reproduced locally
Hudi version : Deltastreamer on EMR 6.8.0 running Hudi 0.11.1-amzn-0
Spark version : 3.3.0
Hive version : 3.1.3
Hadoop version : Amazon 3.2.1
Storage (HDFS/S3/GCS..) : S3
Running on Docker? (yes/no) : No
Additional context
hudi 0.12.1 used for local testing
Can add more details if required.
Stacktrace
Stacktraces have been littered throughout but pasted here again:
my-transactional-topic-single-partition-32-msgs.log
my-transactional-topic-single-partition-allownonconsecutiveoffsetsTrue.log
hoodie-allow-consecutive-off-false.log
hoodie-allow-consecutive-off-true.log
my-transactional-topic-single-partition.log
The text was updated successfully, but these errors were encountered: