KAFKA-14591 DeleteRecordsCommand moved to tools#13278
KAFKA-14591 DeleteRecordsCommand moved to tools#13278mimaison merged 19 commits intoapache:trunkfrom
Conversation
|
This PR is being marked as stale since it has not had any activity in 90 days. If you would like to keep this PR alive, please ask a committer for review. If the PR has merge conflicts, please update it with the latest from trunk (or appropriate release branch) |
There was a problem hiding this comment.
Hi @nizhikov, thanks for working on this. Nice job.
It looks like the scala version does not work well when you pass a non existent partition, while yours works as expected.
# trunk
Executing records delete operation
Records delete operation completed:
partition: mytest-0 error: org.apache.kafka.common.errors.TimeoutException: Call(callName=deleteRecords(api=METADATA), deadlineMs=1688824453413, tries=332909, nextAllowedTryMs=1688824453523) timed out at 1688824453423 after 332909 attempt(s)
# this PR
Executing records delete operation
Records delete operation completed:
partition: mytest-0 error: org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server does not host this topic-partition.Have you considered using the Jackson wrapper proposed here?
Can you please rebase and add a unit test for the various options, like we have for the other tools?
|
Hello, @fvaleri Thanks for the feedback.
Yes. I merged trunk to the PR and added tests for rewritten command.
I'm not aware of this work. |
fvaleri
left a comment
There was a problem hiding this comment.
@nizhikov there are a couple of checkstyle errors.
Please, take a look into DeleteRecordsCommandTest.
Done. Left some comments.
I'm not aware of this work.
It seems right now command works like expected :)
The JSON parsing logic is shared between multiple tools, including this one, so it seems a nice optimization to have. If we can get #13585 merged soon, then you can also use it here. Let's also see what other think about this.
Sure. We must keep codebase consistent. |
|
Yep, LGTM. Just waiting for #13585. |
|
#13585 has been merged. Can you please rebase and integrate with this PR? |
|
@fvaleri Done. New tests passed locally. |
mimaison
left a comment
There was a problem hiding this comment.
Thanks for the PR! I left a few comments.
| assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\"}]}"); | ||
| assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"partition\": \"\"}]}"); | ||
| assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"partition\": 0}]}"); | ||
| assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"offset\":0}]}"); |
There was a problem hiding this comment.
What happens if there are extra fields?
There was a problem hiding this comment.
Extra fields will be ignored.
There was a problem hiding this comment.
Success parsing test case extended like this:
@Test
public void testParse() throws Exception {
Collection<DeleteRecordsCommand.Tuple<TopicPartition, Long>> res = DeleteRecordsCommand.parseOffsetJsonStringWithoutDedup(
"{\"partitions\":[" +
"{\"topic\":\"t\", \"partition\":0, \"offset\":0}," +
"{\"topic\":\"t\", \"partition\":1, \"offset\":1, \"ignored\":\"field\"}," +
"{\"topic\":\"t\", \"partition\":0, \"offset\":2}," +
"{\"topic\":\"t\", \"partition\":0, \"offset\":0}" +
"]}"
);
|
@mimaison Thanks for the review. It seems I addressed all your comments. Please, take a look one more time. |
|
Tests failures unrelated. |
mimaison
left a comment
There was a problem hiding this comment.
Thanks for the updates, I made another pass and left a few more comments
| * 2. It is the most general possible utility, not just the thing you needed in one particular place | ||
| * 3. You have tests for it if it is nontrivial in any way | ||
| */ | ||
| public class CoreUtils { |
There was a problem hiding this comment.
Maybe ToolsUtils would be a better name?
There was a problem hiding this comment.
We already have ToolsUtils in server-common, and maybe we should think about moving it to the tools module in a separate PR.
| } | ||
| } | ||
|
|
||
| public static final class Tuple<V1, V2> { |
There was a problem hiding this comment.
Do we really need this class? Why can't we use something like Map?
There was a problem hiding this comment.
Regular Map implementations can have only one key -> value mapping.
But parseOffsetJsonStringWithoutDedup collects data with possible duplicates.
I can rework command logic and throw on first TopicPartition duplicate, but, it seems out of scope of "just rewrite command in java without changing anything".
Do we have some multi map implementation that can be used here?
There was a problem hiding this comment.
We have similar clases like
org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.TaskPairs.Pairorg.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.ConsumerGenerationPairorg.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.ConsumerPair
So may be it will be better to keep generic class and reuse it in other places?
There was a problem hiding this comment.
I was just asking, if there's a good reason to keep Tuple, we should keep it and not change the tool's semantics.
There was a problem hiding this comment.
It seems tool semantic not changed with Map<TopicPartition, List<Long>>.
So current changes conform both semantic requirements and your suggestion.
| assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\"}]}"); | ||
| assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"partition\": \"\"}]}"); | ||
| assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"partition\": 0}]}"); | ||
| assertThrowsAdminOperationException("{\"partitions\":[{\"topic\":\"t\", \"offset\":0}]}"); |
|
@mimaison all your comments addressed except the one with the Tuple. Please, review. |
|
@mimaison I reworked |
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
This PR contains changes to move
DeleteRecordsCommandto java codeCommitter Checklist (excluded from commit message)