Skip to content
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

KAFKA-14591 DeleteRecordsCommand moved to tools #13278

Merged
merged 19 commits into from Jul 21, 2023

Conversation

nizhikov
Copy link
Contributor

This PR contains changes to move DeleteRecordsCommand to java code

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@nizhikov nizhikov changed the title KAFKA-14591 DeleteRecordsCommand moved to java KAFKA-14591 DeleteRecordsCommand moved to tools Feb 19, 2023
@github-actions
Copy link

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)
If this PR is no longer valid or desired, please feel free to close it. If no activity occurrs in the next 30 days, it will be automatically closed.

@github-actions github-actions bot added the stale Stale PRs label Jun 10, 2023
@github-actions github-actions bot removed the stale Stale PRs label Jun 21, 2023
Copy link
Collaborator

@fvaleri fvaleri left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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?

@nizhikov
Copy link
Contributor Author

Hello, @fvaleri

Thanks for the feedback.

Can you please rebase and add a unit test for the various options, like we have for the other tools?

Yes. I merged trunk to the PR and added tests for rewritten command.
Please, take a look into DeleteRecordsCommandTest.

Have you considered using the Jackson wrapper proposed here?

I'm not aware of this work.
It seems right now command works like expected :)

Copy link
Collaborator

@fvaleri fvaleri left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@nizhikov there are a couple of checkstyle errors.

https://ci-builds.apache.org/blue/rest/organizations/jenkins/pipelines/Kafka/pipelines/kafka-pr/branches/PR-13278/runs/3/nodes/10/log/?start=0

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.

@nizhikov
Copy link
Contributor Author

The JSON parsing logic is shared between multiple tools, including this one, so it seems a nice optimization to have.

Sure. We must keep codebase consistent.
Anyway, in the moment my PR seems to be complete, isn't it?

@fvaleri
Copy link
Collaborator

fvaleri commented Jul 10, 2023

Yep, LGTM. Just waiting for #13585.

@fvaleri
Copy link
Collaborator

fvaleri commented Jul 18, 2023

#13585 has been merged. Can you please rebase and integrate with this PR?

@nizhikov
Copy link
Contributor Author

@fvaleri Done. New tests passed locally.

Copy link
Collaborator

@fvaleri fvaleri left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. Thanks!

Build failure is due to #14037.

@fvaleri fvaleri requested review from showuon and removed request for mimaison July 18, 2023 12:03
Copy link
Member

@mimaison mimaison left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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}]}");
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What happens if there are extra fields?

Copy link
Contributor Author

@nizhikov nizhikov Jul 18, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Extra fields will be ignored.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we have a test for it?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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}" +
            "]}"
        );

checkstyle/import-control.xml Outdated Show resolved Hide resolved
@nizhikov
Copy link
Contributor Author

@mimaison Thanks for the review. It seems I addressed all your comments. Please, take a look one more time.

@nizhikov
Copy link
Contributor Author

Tests failures unrelated.

Copy link
Member

@mimaison mimaison left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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 {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe ToolsUtils would be a better name?

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We already have ToolsUtils in server-common, and maybe we should think about moving it to the tools module in a separate PR.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Renamed

}
}

public static final class Tuple<V1, V2> {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we really need this class? Why can't we use something like Map?

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 on this

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We have similar clases like

  • org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.TaskPairs.Pair
  • org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.ConsumerGenerationPair
  • org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.ConsumerPair

So may be it will be better to keep generic class and reuse it in other places?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I was just asking, if there's a good reason to keep Tuple, we should keep it and not change the tool's semantics.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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}]}");
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we have a test for it?

@nizhikov
Copy link
Contributor Author

@mimaison all your comments addressed except the one with the Tuple. Please, review.

@nizhikov
Copy link
Contributor Author

@mimaison I reworked parseOffsetJsonStringWithoutDedup to return Map<TopicPartition, List<Long>>.
Now, Tuple class eliminated from PR. Please, review.

Copy link
Member

@mimaison mimaison left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for all the updates!

@mimaison mimaison merged commit 4bba2c8 into apache:trunk Jul 21, 2023
1 check failed
@nizhikov
Copy link
Contributor Author

@fvaleri @mimaison Thank you for the review and merge!

Cerchie pushed a commit to Cerchie/kafka that referenced this pull request Jul 25, 2023
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
jeqo pushed a commit to aiven/kafka that referenced this pull request Aug 15, 2023
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
4 participants