Skip to content

KAFKA-10740; Replace OffsetsForLeaderEpochRequest.PartitionData with automated protocol#9689

Merged
dajac merged 4 commits intoapache:trunkfrom
dajac:KAFKA-10740
Dec 17, 2020
Merged

KAFKA-10740; Replace OffsetsForLeaderEpochRequest.PartitionData with automated protocol#9689
dajac merged 4 commits intoapache:trunkfrom
dajac:KAFKA-10740

Conversation

@dajac
Copy link
Copy Markdown
Member

@dajac dajac commented Dec 4, 2020

This patch follows up #9547. It refactors AbstractFetcherThread and its descendants to use OffsetForLeaderEpochRequestData.OffsetForLeaderPartition instead of OffsetsForLeaderEpochRequest.PartitionData. The patch relies on existing tests.

Committer Checklist (excluded from commit message)

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

@dajac dajac marked this pull request as ready for review December 7, 2020 12:46
@dajac
Copy link
Copy Markdown
Member Author

dajac commented Dec 9, 2020

cc @chia7712 @hachikuji

Copy link
Copy Markdown
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

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

@dajac Nice refactor !

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Should it call setPartition?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Good catch! Let me fix this.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

I have updated tests to catch this.

Copy link
Copy Markdown
Contributor

@hachikuji hachikuji left a comment

Choose a reason for hiding this comment

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

Thanks, left a couple small suggestions.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Could we use RequestUtils.getLeaderEpoch?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Yes, let me change this.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Maybe we can use scala.compat.java8.OptionConverters._?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

We could but that would require two conversions instead of one:

val currentLeaderEpoch = latestEpochsForPartitions.get(tp).flatMap { p =>
    RequestUtils.getLeaderEpoch(p.currentLeaderEpoch).asScala
}.asJava

I have a small preference for the current approach to avoid this.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Hmm.. It's a little curious that we need the call to getLeaderEpoch here. Tracing this back to fetchTruncatingPartitions, it looks like we could not have a negative epoch here.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Yeah, that's because onPartitionFenced takes an Optional actually.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

As onPartitionFenced is used in other code paths, I did not refactor it. We could have another variant which does not take an Optional. Is it worth doing it?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

@hachikuji Ah.. I see what you mean now. I have removed the call to getLeaderEpoch.

@dajac
Copy link
Copy Markdown
Member Author

dajac commented Dec 10, 2020

@chia7712 @hachikuji Thanks for your feedback. I have addressed your comments. Could you take another look please?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

@chia7712 Thanks for pointing this out. I think that we should tackle this in follow-up PRs as this is not strictly related to this change. Ok for you?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

It is ok to me :)

Copy link
Copy Markdown
Contributor

@hachikuji hachikuji left a comment

Choose a reason for hiding this comment

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

Thanks, LGTM. I wonder in general if we should hesitate about using the request objects directly in internal code. This patch seems fine because we are just replacing the old request object with the generated one. But in general, using request objects seem to introduce some representation awkwardness.

For example, EpochData has a field for the partition index, but no field for the topic. This makes it awkward for use in contexts outside of the request. You feel obligated to populate the partition index, but that alone is not useful unless you are keeping track of the topic somewhere else.

We are also forced into the representation of the protocol. If we have to make a field optional for compatibility, it's difficult to write internal code which requires it presence. Take the currentLeaderEpoch field for example. We will always have this available in the replica fetcher thread even though it is considered optional from the perspective of the API.

Basically it feels like request/response details end up leaking into the implementation. I'm interested if you have any thoughts about this. I could see one extreme side of this would be to say that generated classes should only survive in the context of a specific request. But perhaps we would like to relax that in some cases to avoid conversion costs.

@dajac
Copy link
Copy Markdown
Member Author

dajac commented Dec 17, 2020

@hachikuji I totally agree with your comment. I have been pursuing this migration for this API to better understand the issues that we would face if we really want to directly use the auto-generated requests and responses in the near future.

If we want to proceed on that path, I think that we should evolved the auto-generated protocol. I envision the following improvements:

  1. The first area is about the Map. We made the choice to not use Map in the auto-generated classes in favour of using our internal map-like data structure which is more efficient. The down side of this is that it always requires to convert our internal Map to that data structure. It would be great if we could just use the internal Map directly. That would reduce the boiler plate code to do the conversion.

  2. Similarly, it seems that could make TopicPartition a first class citizen in the auto-generated protocol. It would be so nice if we would get Map with TopicPartition as the key when a request or a response represents topics/partitions. I have put some details here: https://issues.apache.org/jira/browse/KAFKA-10795. This would remove the awkwardness that you are talking about as, in our particular case, it would remove the partition index from EpochData.

  3. I believe that we should add support for Optional in the auto-generated protocol. The idea is to get an empty option when the value is equals to the sentinel value. I prototyped this a while ago. The PR is a bit outdated but the overall idea remains. It would be great it you could take a look. MINOR: Support java.util.Optional in the auto-generated protocol #9085

  4. Another area would be to have the ability to define common structs in the auto-generated protocol. That would allow to share them between requests/responses. This is not strictly necessary but that would be nice to have.

Overall, I believe that we could make it happen if we improve the auto-generated protocol. The substitution reads pretty well when the auto-generated data structure is a one-to-one replacement. This PR is a good example: #9746.

Assuming that we can it make feel more natural with the suggested improvement, do you believe that we should try to continue on trying to use the auto-generate requests and responses directly?

@chia7712
Copy link
Copy Markdown
Member

Assuming that we can it make feel more natural with the suggested improvement, do you believe that we should try to continue on trying to use the auto-generate requests and responses directly?

I'm +1000 to @dajac as there is a great benefit from using auto-generated code.

  1. avoid extra conversion
  2. remove duplicate code from code base
  3. builder pattern (it is especially good to the classes having a bunch of fields)

Also, it is ok to use extra request/response wrap to offer some helper methods when they are NOT supported by auto-generated code (optional and TopicPartition, for example).

@dajac
Copy link
Copy Markdown
Member Author

dajac commented Dec 17, 2020

I've rebased the PR. I will merge it when I get a clean build.

@dajac dajac merged commit 02a30a5 into apache:trunk Dec 17, 2020
@dajac dajac deleted the KAFKA-10740 branch December 17, 2020 16:40
@hachikuji
Copy link
Copy Markdown
Contributor

@dajac Yeah, I'm definitely in favor of the improvements you have suggested, but I'm not sure they 100% address the concern. The case of Optional is an interesting example. A field might become optional because it is not supported in older version of the protocol. That optionality leaks into the code even when we can guarantee that internal data structures will have the field present. Anyway, I still think we should be thinking about these conversions when they make sense. Probably down the Fetch paths (including the replica fetcher) is the biggest opportunity for performance improvements. I'm just highlighting some downsides.

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Jan 22, 2021

Like Jason said, I think the main challenge is how to deal with protocol evolution. It's typically possible to deal with, but it can require quite a lot of contortion. LeaderAndIsrRequest is one example:

https://github.com/a0x8o/kafka/blob/master/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java

@chia7712
Copy link
Copy Markdown
Member

It look like a bad pattern if we have to "contort" a bunch of data from auto-generated protocol when processing.

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Jan 22, 2021

It's the nature of protocol evolution. You don't know how things will change over time ahead of time.

@chia7712
Copy link
Copy Markdown
Member

You don't know how things will change over time ahead of time.

You are right. However, it seems to me it is hard to apply “single” solution to all auto-generated data. Adding wrap to auto-generated data could cause extra cost (we can address lazy evaluation but it could make complicated code). By contrast, using auto-generated data in server-side directly gets less flexibility.

It is trade off and it seems to me a happy medium is to add help methods to each request. Server-side can call help methods explicitly when it does need different behavior for different data version. The benefit is the extra cost is produced iff we does need different data struct.

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Jan 22, 2021

It definitely requires analysis on a case by case basis. However, having helpers that do conversion lazily also have performance issues if the method gets called repeatedly, unless you cache the converted data (which has its own complexities). Generally, a good thing to aim for is that recent versions don't require conversions and older version conversions into the latest format are done only once.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants