NIFI-14123: support kafka tombstone with USE_WRAPPER strategy#9604
NIFI-14123: support kafka tombstone with USE_WRAPPER strategy#9604greyp9 merged 5 commits intoapache:mainfrom babubabu:NIFI-14123
Conversation
|
Hi @babubabu; thanks for the contribution! Looks like the "integration-tests" CI failed on this PR for the new test that was added. You can easily run this single test (after a full build) with a command line something like this: |
|
Hi @greyp9 , |
|
Hello @greyp9, |
Add possibility to leave out the "value" attribute in flowfile wrapper to support publishing kafka tombstones. With this solution, we support a wrapper either completely without the "value" attribute or with "value: null". The JSON Reader and AVRO Set writer support null values out-of-the-box
|
Fixed checkstyle violations and rebased to current main |
I just started the automation. It looks like there is a bit of additional nuance related to the It makes sense that Additionally, Thanks for your continued attention to this! |
Consume: Kafka tombstones are consumed by writing an empty content and setting the flowfile attribute kafka.tombstone=true Publish: An empty flowfile-content with attribute kafka.tombstone=true results in an published kafka.tombstone
|
To keep the changes in one place, I commited the changes for ConsumeKafka and PublishKafka to this PR. I hope, it's still fine :-) |
|
Hi @greyp9 : the checkstyle violations have been fixed. Can you please start again the actions? |
greyp9
left a comment
There was a problem hiding this comment.
Please consider my suggestion about coverage for the Publish empty string test case. A couple of couple comments as well.
Thanks for your continued attention!
| @@ -0,0 +1,14 @@ | |||
| { | |||
There was a problem hiding this comment.
I think you were on to something with the previous version of this test resource. Three test cases should provide good coverage:
- value is empty string
- value is null
- omit value from the JSON representation of the record
(The test code would then need to be adjusted to expect three records to be emitted from the publish part.)
There was a problem hiding this comment.
I just overthought the empty string case: This can be a valid "filled" value for String values, without sending tombstones at all.
| try (KafkaConsumer<String, String> consumer = new KafkaConsumer<>(getKafkaConsumerProperties())) { | ||
| consumer.subscribe(Arrays.asList(TEST_TOPIC, OVERRIDE_TOPIC)); | ||
| final ConsumerRecords<String, String> records = consumer.poll(DURATION_POLL); | ||
| assertEquals(1, records.count()); |
There was a problem hiding this comment.
Here is where we would expect three, after the adjustment of the test resource.
| final byte[] value = converter.toBytes(WrapperRecord.VALUE, writerFactory); | ||
| ProducerUtils.checkMessageSize(maxMessageSize, value.length); | ||
|
|
||
| if (value != null) { |
There was a problem hiding this comment.
For the RecordWrapper strategy, there needs to be some logic at this point to do the evaluation of value and the FlowFile attribute "kafka.tombstone". One option is to implement a utility method to do things inline at line 109, something like:
private byte[] toValue(final byte[] bytes, final FlowFile flowFile) {
final String tombstone = flowFile.getAttribute(KafkaFlowFileAttribute.KAFKA_TOMBSTONE);
final boolean isTombstone = (Boolean.TRUE.toString().equals(tombstone) && (flowFile.getSize() == 0));
return isTombstone ? null : bytes;
}
This is surfaced by adding the zero byte value test case described in this review.
There was a problem hiding this comment.
Are you sure to depend a tombstone produce on both value=null and the flowfile attribute? I know, on the one hand, the attribute is a part of the "contract" of the nifi-kafka-interface, but on the other hand as a user, I would not assume to also set the flowfile attribute after adjusting the content to wrapper structure (e.g. jolt-transfom or others). Additionally, we can keep the code cleaner ;-)
I would follow your intention in this point. If you want to, I'll add the dependency!
There was a problem hiding this comment.
Yes, there is some ambiguity here.
Given the other code touch, it seems like it makes sense to let the "wrapper record" value pass through, which would be implicitly interpreted by Kafka as a tombstone. The confusion is that for the FlowFile strategy, it is not possible to set the value to null, so we need to flex based on the FlowFile attribute. But not for this strategy.
I think this change is good...
| + "may be record-oriented data that can be read by the configured Record Reader. " | ||
| + "The complementary NiFi processor for fetching messages is ConsumeKafka.") | ||
| + "The complementary NiFi processor for fetching messages is ConsumeKafka. " | ||
| + "To produce a kafka tombstone message while using PublishStrategy.USE_WRAPPER, simply set the value of a record to 'null'.") |
There was a problem hiding this comment.
The previous implementation seems to suggest the expectation that value=null should only be sent when both conditions are true (zero byte, tombstone attribute). But then it would be nice to be able to send n wrapper records, where some are tombstone records and some are not. So this should be ok.
There was a problem hiding this comment.
Sorry, I don't understand this point! Is this related to the Wrapper change above (adding the tombstone attribute)?
There was a problem hiding this comment.
I was talking myself through the tradeoffs. For USE_WRAPPER, this documentation seems fine. No request here.
| recordHeaders.add(recordHeader); | ||
| }); | ||
|
|
||
| // NIFI-14122: Support Kafka tombstones |
There was a problem hiding this comment.
Code comments like this (with JIRA numbers) in the production code are generally disfavored (though they do exist). Maybe just:
Support Kafka tombstones
greyp9
left a comment
There was a problem hiding this comment.
Almost there. Thanks for the patience while I reasoned through the intersection of RecordWrapperStrategy and the tombstone behavior.
There is a problem with the behavior of maven-failsafe-plugin regarding PublishKafkaWrapperRecordTombstoneIT2. I think it just needs a rename to PublishKafkaWrapperRecordTombstoneIT so it will be seen and the test run in the context of -P integration-tests.
|
Thanks for your patience, too! |
greyp9
left a comment
There was a problem hiding this comment.
Thanks for this. I verified that the renamed IT now runs during the integration tests. I'm approving the PR, and plan to merge once the automation completes.
|
Thank you for reviewing and gate-keeping :-) |
Add possibility to leave out the "value" attribute in flowfile wrapper to support publishing kafka tombstones. With this solution, we support a wrapper either completely without the "value" attribute or with "value: null". The JSON Reader and AVRO Set writer support null values out-of-the-box
Summary
NIFI-14123
NIFI-14122
Tracking
Please complete the following tracking steps prior to pull request creation.
Issue Tracking
Pull Request Tracking
NIFI-00000NIFI-00000Pull Request Formatting
mainbranchVerification
Please indicate the verification steps performed prior to pull request creation.
Build
mvn clean install -P contrib-checkLicensing
LICENSEandNOTICEfilesDocumentation