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-15034: Improve performance of the ReplaceField SMT; add JMH benchmark #13776

Merged
merged 3 commits into from
Jun 1, 2023

Conversation

yashmayya
Copy link
Contributor

@yashmayya yashmayya commented May 29, 2023

  • https://issues.apache.org/jira/browse/KAFKA-15034
  • The ReplaceField SMT can be configured with a list of fields that are to be included or excluded during every record transformation.
  • Currently, it uses an ArrayList for these fields which causes the filter operations to be of O(N) complexity resulting in poor performance when configured with a large number of include / exclude fields.
  • This patch refactors it to use a HashSet instead (O(1) expected time complexity) and adds a JMH benchmark to demonstrate the performance improvements.

JMH Benchmark Result Before (ArrayList based implementation):

Benchmark                                                  (includeExcludeFieldCount)  (valueFieldCount)  Mode  Cnt          Score         Error  Units
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                           1                100  avgt    5        928.115 ±      20.251  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                           1               1000  avgt    5      10380.401 ±     286.643  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                           1              10000  avgt    5      97058.104 ±    3834.409  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                         100                100  avgt    5      15052.629 ±     112.337  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                         100               1000  avgt    5     301212.390 ±    6400.402  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                         100              10000  avgt    5    2218226.090 ±   38198.098  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                       10000                100  avgt    5     582789.404 ±   11436.565  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                       10000               1000  avgt    5    6263588.619 ±  530370.435  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                       10000              10000  avgt    5  133424024.627 ± 9497024.791  ns/op

JMH Benchmark Result After (HashSet based implementation):

Benchmark                                                  (includeExcludeFieldCount)  (valueFieldCount)  Mode  Cnt       Score      Error  Units
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                           1                100  avgt    5    1205.928 ±   32.611  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                           1               1000  avgt    5   10124.067 ±  212.876  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                           1              10000  avgt    5  105143.540 ± 1813.534  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                         100                100  avgt    5    1602.392 ±   15.756  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                         100               1000  avgt    5   11543.659 ±  193.129  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                         100              10000  avgt    5  171689.002 ± 5691.014  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                       10000                100  avgt    5    1686.155 ±   21.922  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                       10000               1000  avgt    5   20584.457 ±  429.614  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                       10000              10000  avgt    5  221015.401 ± 8108.798  ns/op

Committer Checklist (excluded from commit message)

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

@yashmayya yashmayya changed the title KAFKA-15034: Use HashSet for include / exclude fields in ReplaceField SMT; add JMH benchmark KAFKA-15034: Improve performance of ReplaceField SMT when configured with lots of include / exclude fields; add JMH benchmark May 29, 2023
…with a large number of include / exclude fields; add JMH benchmark
@yashmayya yashmayya changed the title KAFKA-15034: Improve performance of ReplaceField SMT when configured with lots of include / exclude fields; add JMH benchmark KAFKA-15034: Improve performance of the ReplaceField SMT; add JMH benchmark May 30, 2023
Copy link
Contributor

@C0urante C0urante left a comment

Choose a reason for hiding this comment

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

Thanks Yash. I like the small-change, large-gain performance improvement that's made possible here, but I'm curious about potential fallout under certain scenarios. LMKWYT

Comment on lines 63 to 66
replaceFieldConfigs.put("exclude",
IntStream.range(0, fieldCount).filter(x -> (x & 1) == 0).mapToObj(x -> "Field-" + x).collect(Collectors.joining(",")));
replaceFieldConfigs.put("include",
IntStream.range(0, fieldCount).filter(x -> (x & 1) == 1).mapToObj(x -> "Field-" + x).collect(Collectors.joining(",")));
Copy link
Contributor

Choose a reason for hiding this comment

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

We may want to add a separate parameter for the number of included/excluded fields (can be a single parameter to control both, or a separate parameter for each) in order to cover the case of a value with a large number of fields and a small number of included/excluded fields.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Given the above observations, do you feel like this is still required?

Copy link
Contributor

Choose a reason for hiding this comment

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

I feel they are going to perform the same. The time complexity of String#hashCode is O(n) and same is the case for equals method. No harm in trying though :)

Copy link
Contributor

Choose a reason for hiding this comment

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

Object lookup in a hash set usually involves both computing its hash and performing an equality check, since multiple objects may occupy the same bucket. Lookup in a single-element list may theoretically be faster if it only involves a single equality check.

I think it's worth including in the benchmark for a few reasons:

  • Saves people the trouble of having to look up this PR discussion
  • Covers a more-common case (it's much more likely that someone configures this SMT with 1 field than 10,000)
  • Guards against performance regressions if we change things in the future

But if it's too much work then we can merge as-is. @yashmayya let me know what your decision is.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Makes sense, I've added a separate parameter controlling the number of include and exclude fields since it was a pretty minor change. These are the new results:

JMH Benchmark Result Before (ArrayList based implementation):

Benchmark                                                  (includeExcludeFieldCount)  (valueFieldCount)  Mode  Cnt          Score         Error  Units
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                           1                100  avgt    5        928.115 ±      20.251  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                           1               1000  avgt    5      10380.401 ±     286.643  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                           1              10000  avgt    5      97058.104 ±    3834.409  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                         100                100  avgt    5      15052.629 ±     112.337  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                         100               1000  avgt    5     301212.390 ±    6400.402  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                         100              10000  avgt    5    2218226.090 ±   38198.098  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                       10000                100  avgt    5     582789.404 ±   11436.565  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                       10000               1000  avgt    5    6263588.619 ±  530370.435  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                       10000              10000  avgt    5  133424024.627 ± 9497024.791  ns/op

JMH Benchmark Result After (HashSet based implementation):

Benchmark                                                  (includeExcludeFieldCount)  (valueFieldCount)  Mode  Cnt       Score      Error  Units
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                           1                100  avgt    5    1205.928 ±   32.611  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                           1               1000  avgt    5   10124.067 ±  212.876  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                           1              10000  avgt    5  105143.540 ± 1813.534  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                         100                100  avgt    5    1602.392 ±   15.756  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                         100               1000  avgt    5   11543.659 ±  193.129  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                         100              10000  avgt    5  171689.002 ± 5691.014  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                       10000                100  avgt    5    1686.155 ±   21.922  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                       10000               1000  avgt    5   20584.457 ±  429.614  ns/op
ReplaceFieldBenchmark.includeExcludeReplaceFieldBenchmark                       10000              10000  avgt    5  221015.401 ± 8108.798  ns/op

Copy link
Contributor

Choose a reason for hiding this comment

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

Awesome, thanks!

Copy link
Contributor Author

@yashmayya yashmayya 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 taking a look Chris!

Comment on lines 63 to 66
replaceFieldConfigs.put("exclude",
IntStream.range(0, fieldCount).filter(x -> (x & 1) == 0).mapToObj(x -> "Field-" + x).collect(Collectors.joining(",")));
replaceFieldConfigs.put("include",
IntStream.range(0, fieldCount).filter(x -> (x & 1) == 1).mapToObj(x -> "Field-" + x).collect(Collectors.joining(",")));
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Given the above observations, do you feel like this is still required?

Copy link
Contributor

@C0urante C0urante left a comment

Choose a reason for hiding this comment

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

LGTM!

@C0urante C0urante merged commit 9bb2f78 into apache:trunk Jun 1, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
3 participants