Skip to content

KAFKA-9146: KIP-571 Add option to force delete active members in StreamsResetter#8589

Merged
mjsax merged 19 commits intoapache:trunkfrom
feyman2016:KAFKA-9146-formal
May 28, 2020
Merged

KAFKA-9146: KIP-571 Add option to force delete active members in StreamsResetter#8589
mjsax merged 19 commits intoapache:trunkfrom
feyman2016:KAFKA-9146-formal

Conversation

@feyman2016
Copy link
Copy Markdown
Contributor

This PR is mainly to enhance https://issues.apache.org/jira/browse/KAFKA-9146.

  • KafkaAdminClient#removeMembersFromConsumerGroup now support "removeAll" members in a given group
  • New cmdline option: --force for StreamsResetter is introduced, if --force specified when using the StreamsResetter, then all the active static/dynamic members will be removed.

Related KIP:
KIP-571: https://cwiki.apache.org/confluence/display/KAFKA/KIP-571%3A+Add+option+to+force+remove+members+in+StreamsResetter

Committer Checklist (excluded from commit message)

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

@feyman2016 feyman2016 changed the title KAFKA-9146: Add option to force delete active members in StreamsResetter KAFKA-9146: KIP-571 Add option to force delete active members in StreamsResetter Apr 30, 2020
@mjsax
Copy link
Copy Markdown
Member

mjsax commented Apr 30, 2020

Retest this please.

Call for review @abbccdda

@feyman2016
Copy link
Copy Markdown
Contributor Author

oh, sorry I didn't run the Checkstyle and spotbugs quality checks locally, I will update shortly with these fixed

@feyman2016
Copy link
Copy Markdown
Contributor Author

Retest this please.

@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 1, 2020

@feyman2016 Only committers can trigger Jenkins retesting...

Retest this please.

@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 2, 2020

Retest this please

@feyman2016
Copy link
Copy Markdown
Contributor Author

feyman2016 commented May 2, 2020

@mjsax I see, thanks!
And call for review :)

Copy link
Copy Markdown

@abbccdda abbccdda 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, left some initial comments.

if (options.removeAll()) {
List<MemberIdentity> members = getMembersFromGroup(groupId);
findCoordinatorCall = getFindCoordinatorCall(context,
() -> getRemoveMembersFromGroupCall(context, members));
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

could we pass the members into the context?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

My initial thought was to put the members in the context, but hesitated to do so because the ConsumerGroupOperationContext seems to be for generic usage. So I just refer to KafkaAdminClient#getAlterConsumerGroupOffsetsCall and make the members as a separate input param. Anyway, I'm glad to make the change if we think it's preferred to put the members in context.

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Yes, I feel this is more consistent for internal calls not to do a second round of interpretation for which members set to use.

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

And to be clear, I'm not suggesting we have to put stuff into the context, just always passing in the intended removal list and do not depend on context.removeAll again inside internal function.

try {
members = describeConsumerGroups(Collections.singleton(groupId)).describedGroups().get(groupId).get().members();
} catch (Throwable ex) {
System.out.println("Encounter exception when trying to get members from group: " + groupId);
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Remove print statements

Copy link
Copy Markdown
Contributor Author

@feyman2016 feyman2016 May 6, 2020

Choose a reason for hiding this comment

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

Fixed~

members = describeConsumerGroups(Collections.singleton(groupId)).describedGroups().get(groupId).get().members();
} catch (Throwable ex) {
System.out.println("Encounter exception when trying to get members from group: " + groupId);
ex.printStackTrace();
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Curious why we are still continuing in this case, as the member lookup already fails.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Thanks, will fix this .

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed

@@ -32,12 +32,23 @@
public class RemoveMembersFromConsumerGroupOptions extends AbstractOptions<RemoveMembersFromConsumerGroupOptions> {

private Set<MemberToRemove> members;
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Could we just make members to be Optional<Set<MemberToRemove>> so that we don't need a separate removeAll parameter?

Copy link
Copy Markdown
Contributor Author

@feyman2016 feyman2016 May 6, 2020

Choose a reason for hiding this comment

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

Sure. Taking a step further, can we just keep the the type Set<MemberToRemove> for members unchanged and treat it as removeAll if the members is empty set?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Updated~

}

List<MemberIdentity> memberToRemove = new ArrayList<>();
for (MemberDescription member: members) {
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

style error here.

I would recommend doing a self style check like:
./gradlew checkstyleMain checkstyleTest spotbugsMain spotbugsTest spotbugsScoverage compileTestJava otherwise we still need to fix those failures after we do jenkins build.

Copy link
Copy Markdown
Contributor Author

@feyman2016 feyman2016 May 6, 2020

Choose a reason for hiding this comment

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

Thanks for the advice, will fix it in the next commit.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I reran the self style check, but didn't capture any error. I assume the error would be the missed final in for loop, updated.

@feyman2016
Copy link
Copy Markdown
Contributor Author

@abbccdda Thanks a lot for the review, will update soon.

@feyman2016
Copy link
Copy Markdown
Contributor Author

@abbccdda Hey, updated based on comments, and also left some comments there, thanks.

@feyman2016
Copy link
Copy Markdown
Contributor Author

Call for retest and review, thanks!

@abbccdda
Copy link
Copy Markdown

@feyman2016 Sure thing!

Copy link
Copy Markdown

@abbccdda abbccdda 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 update. I could see we are still having a couple of places with style violation, did you properly configure and run checkstyle locally?

}

List<MemberIdentity> memberToRemove = new ArrayList<>();
for (final MemberDescription member: members) {
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

nit: space before :

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed~

if (options.removeAll()) {
List<MemberIdentity> members = getMembersFromGroup(groupId);
findCoordinatorCall = getFindCoordinatorCall(context,
() -> getRemoveMembersFromGroupCall(context, members));
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Yes, I feel this is more consistent for internal calls not to do a second round of interpretation for which members set to use.

public boolean removeAll() {
return members.isEmpty();
}

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

nit: remove extra line

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed

() -> getRemoveMembersFromGroupCall(context, members));
} else {
findCoordinatorCall = getFindCoordinatorCall(context,
() -> getRemoveMembersFromGroupCall(context, new ArrayList<>()));
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Should be Collection.emptyList()

// If coordinator changed since we fetched it, retry
if (ConsumerGroupOperationContext.hasCoordinatorMoved(response)) {
Call call = getRemoveMembersFromGroupCall(context);
Call call = getRemoveMembersFromGroupCall(context, allMembers);
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Why do we blindly put allMembers? I believe we base on context to interpret, but like discussed earlier, this is easy to make mistake, we should rely on one source for members.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed, now we explicitly pass in the members to be deleted to the private getRemoveMembersFromGroupCall

Utils.closeQuietly(producer, "producer")
}

val EMPTY_GROUP_INSTANCE_ID = ""
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

I don't think we really need this struct, could we just put null in groupInstanceSet?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I feel like this is more informative, so didn't update it, but yeah, I can update if we really not prefer this~

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Fair enough

try {
while (true) {
consumer.poll(JDuration.ofSeconds(5))
if ( !consumer.assignment.isEmpty && latch.getCount > 0L)
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

nit: space

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed

consumer.commitSync()
}
} catch {
case _: InterruptException => // Suppress the output to stderr
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Why do we suppress here?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Didn't change the exception handling logic here, just extract the Thread creation logic to reuse~

assertFalse(testGroupDescription.isSimpleConsumerGroup)
assertEquals(consumerSet.size -1, testGroupDescription.members().size())

// Delete all active members remained (a static member + a dynamic member)
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

remained -> remaining

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed

.withRequiredArg()
.ofType(String.class)
.describedAs("file name");
forceOption = optionParser.accepts("force", "Force remove members when long session time out has been configured, " +
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Do we also want to edit the usage info on top to mention the force delete option?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I think so, updated

@feyman2016
Copy link
Copy Markdown
Contributor Author

@abbccdda Sorry for causing too much trouble about style....will make sure style check correctly executed before requesting for review next time.

@abbccdda
Copy link
Copy Markdown

@feyman2016 no worry! It's just for first time :)

@feyman2016
Copy link
Copy Markdown
Contributor Author

feyman2016 commented May 19, 2020

@abbccdda I'm fixing the style error, and I found that it showed success if I run the below cmd locally :
./gradlew checkstyleMain checkstyleTest spotbugsMain spotbugsTest spotbugsScoverage compileTestJava
I'm didn't find the reason after some investigating, but the style check cmdline can capture style error if I cherry-pick the first commit of this PR to another branch, guessing that it might be the second commit: merge trunk somehow made the style check doesn't work...
So would it break our convention if I revert to the first commit:Add option to force delete active members in StreamsResetter and fix style error from there ? This may change the commit history of this PR.
I'm asking because I noticed :"Please address feedback via additional commits instead of amending existing commits." on https://cwiki.apache.org/confluence/display/KAFKA/Contributing+Code+Changes#ContributingCodeChanges-PullRequest
Thanks!

@abbccdda
Copy link
Copy Markdown

@feyman2016 Thanks for the context. I don't worry too much for comment vanish if you change the commit history, as they would not be gone but just show as outdated on github. Just do whatever you feel makes sense.

Also just a reminder that we are one week away from the feature freeze, so let's try to ramp up and get this into 2.6.

@feyman2016
Copy link
Copy Markdown
Contributor Author

@abbccdda Thanks, will try my best to get this into 2.6

@feyman2016
Copy link
Copy Markdown
Contributor Author

@abbccdda Updated, call for review, thanks!

Copy link
Copy Markdown

@abbccdda abbccdda 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 update @feyman2016 ! Got some more comments, but I think we are getting there :)

Collection<MemberDescription> members;
try {
members = describeConsumerGroups(Collections.singleton(groupId)).describedGroups().get(groupId).get().members();
} catch (Throwable ex) {
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Make sense, fixed~

for (final MemberDescription member : members) {
if (member.groupInstanceId().isPresent()) {
memberToRemove.add(new MemberIdentity().setGroupInstanceId(member.groupInstanceId().get())
);
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

This indentation is a bit weird, let's just merge L3625-3626

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed


ConsumerGroupOperationContext<Map<MemberIdentity, Errors>, RemoveMembersFromConsumerGroupOptions> context =
new ConsumerGroupOperationContext<>(groupId, options, deadline, future);
new ConsumerGroupOperationContext<>(groupId, options, deadline, future);
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Let's get back the original indentation.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed

return new LeaveGroupRequest.Builder(context.groupId(),
context.options().members().stream().map(
MemberToRemove::toMemberIdentity).collect(Collectors.toList()));
return new LeaveGroupRequest.Builder(context.groupId(),
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

nit: we could merge L3666-3667

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Updated


private Call getRemoveMembersFromGroupCall(ConsumerGroupOperationContext<Map<MemberIdentity, Errors>, RemoveMembersFromConsumerGroupOptions> context) {
private Call getRemoveMembersFromGroupCall(ConsumerGroupOperationContext<Map<MemberIdentity, Errors>,
RemoveMembersFromConsumerGroupOptions> context, List<MemberIdentity> allMembers) {
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

nit: we could name it members now

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Yeah, fixed

consumerThread.join()
}
}
}finally {
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

nit: format
I'm pretty surprised this wasn't caught in my previous template. Let me check how to cover this in style test as well.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Thanks, but I wonder what does the template refer to here?

public void testResetWhenLongSessionTimeoutConfiguredWithForceOption() throws Exception {
appID = testId + "-with-force-option";
streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, appID);
streamsConfig.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "" + STREAMS_CONSUMER_TIMEOUT * 100);
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

What does "" + mean?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Without the "" + to convert the value to String, we will get exception like: it is because STREAMS_CONSUMER_TIMEOUT = 2000L, ""+ is widely used in this test, just follow it here without any change to not enlarge the scope of this PR, I can help to create a Jira to enhance it if we think this workaround is not quite intuitive~

org.apache.kafka.common.config.ConfigException: Invalid value 200000 for configuration session.timeout.ms: Expected value to be a 32-bit integer, but it was a java.lang.Long
        at org.apache.kafka.common.config.ConfigDef.parseType(ConfigDef.java:672)
        at org.apache.kafka.common.config.ConfigDef.parseValue(ConfigDef.java:474)
        at org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:467)
        at org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:108)
        at org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:129)
        at org.apache.kafka.clients.consumer.ConsumerConfig.<init>(ConsumerConfig.java:606)
        at org.apache.kafka.clients.consumer.KafkaConsumer.<init>(KafkaConsumer.java:630)
        at org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier.getRestoreConsumer(DefaultKafkaClientSupplier.java:56)
        at org.apache.kafka.streams.processor.internals.StreamThread.create(StreamThread.java:313)
        at org.apache.kafka.streams.KafkaStreams.<init>(KafkaStreams.java:766)
        at org.apache.kafka.streams.KafkaStreams.<init>(KafkaStreams.java:652)
        at org.apache.kafka.streams.KafkaStreams.<init>(KafkaStreams.java:562)
        at org.apache.kafka.streams.integration.AbstractResetIntegrationTest.testResetWhenLongSessionTimeoutConfiguredWithForceOption(AbstractResetIntegrationTest.java:270)
        at org.apache.kafka.streams.integration.ResetIntegrationTest.testResetWhenLongSessionTimeoutConfiguredWithForceOption(ResetIntegrationTest.java:77)

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

I see, this is indeed weird, please file a JIRA so that we could clean in a follow-up PR if others feel the same way.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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


private void cleanGlobal(final boolean withIntermediateTopics,
private int tryCleanGlobal(final boolean withIntermediateTopics,
final String resetScenario,
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

nit: parameters are not aligned.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed

cleanUpConfig.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "" + CLEANUP_CONSUMER_TIMEOUT);

final int exitCode = new StreamsResetter().run(parameters, cleanUpConfig);
return exitCode;
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Like said earlier, I think we could just return
return new StreamsResetter().run(parameters, cleanUpConfig) == 0

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Updated

}

private void cleanGlobal(final boolean withIntermediateTopics,
private int tryCleanGlobal(final boolean withIntermediateTopics,
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

We could add meta comment for the return value here, and instead of returning an exit code, I feel a boolean is suffice to indicate whether the clean operation was successful or not.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Indeed, updated as suggested

*/
public KafkaFuture<Void> memberResult(MemberToRemove member) {
if (removeAll()) {
throw new IllegalArgumentException("The method: memberResult is not applicable in 'removeAll' mode");
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.

Why that? I understand that we expect that users don't know the memberId if the so a "remove all"; however, I don't see why we need to disallow this call? Can you elaborate?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Since in the removeAll scenario, we don't save the members to be deleted in RemoveMembersFromConsumerGroupResult, so I think calling memberResult doesn't seem applicative.

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.

I see. Makes sense.

MetadataResponse.AUTHORIZED_OPERATIONS_OMITTED);
}

private static DescribeGroupsResponseData prepareDescribeGroupsResponseData(String groupId, List<String> groupInstances,
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.

Nit: formatting

private static DescribeGroupsResponseData prepareDescribeGroupsResponseData(String groupId,
                                                                            List<String> groupInstances,
                                                                            List<TopicPartition> topicPartitions) {

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed

new NewTopic(testTopicName, testNumPartitions, 1.toShort))).all().get()
waitForTopics(client, List(testTopicName), List())

client.createTopics(util.Arrays.asList(new NewTopic(testTopicName, testNumPartitions, 1.toShort),
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.

nit: formatting: move new NewTopic(...) to next line

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Fixed

Copy link
Copy Markdown
Member

@mjsax mjsax left a comment

Choose a reason for hiding this comment

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

Overall, LGTM. Couple of comments/questions.

feyman2016 and others added 6 commits May 27, 2020 15:26
Co-authored-by: Matthias J. Sax <mjsax@apache.org>
Co-authored-by: Matthias J. Sax <mjsax@apache.org>
Co-authored-by: Matthias J. Sax <mjsax@apache.org>
Co-authored-by: Matthias J. Sax <mjsax@apache.org>
@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 27, 2020

Retest this please.

1 similar comment
@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 27, 2020

Retest this please.

@feyman2016
Copy link
Copy Markdown
Contributor Author

@mjsax Thanks a lot for the review and tests triggering!

@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 27, 2020

Java 8:

org.apache.kafka.streams.integration.RegexSourceIntegrationTest.testRegexRecordsAreProcessedAfterReassignment
org.apache.kafka.streams.integration.RegexSourceIntegrationTest.testRegexMatchesTopicsAWhenCreated

Java 11:

org.apache.kafka.connect.integration.InternalTopicsIntegrationTest.testCreateInternalTopicsWithDefaultSettings
org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true]

Java 14:

org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true]

@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 27, 2020

Retest this please.

3 similar comments
@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 27, 2020

Retest this please.

@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 27, 2020

Retest this please.

@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 27, 2020

Retest this please.

@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 27, 2020

Jenkins does not cooperate... will try again later

@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 27, 2020

Retest this please.

@feyman2016
Copy link
Copy Markdown
Contributor Author

Finally, jenkins cooperated :)

@mjsax mjsax merged commit 90045f6 into apache:trunk May 28, 2020
@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 28, 2020

Thanks for the KIP and PR @feyman2016!

@abbccdda
Copy link
Copy Markdown

Great work, @feyman2016 @mjsax !

@feyman2016
Copy link
Copy Markdown
Contributor Author

@abbccdda @mjsax Thanks a lot for your kindly review and help! Are you guys using some formatting tool, I asked this because this PR had too many formatting issues, would be good if next time I could effectively avoid them, thanks!

@feyman2016
Copy link
Copy Markdown
Contributor Author

FYI, since we took a slightly different implementation(leveraging the empty members rather than introducing a new field to imply the removeAll scenario), I updated the KIP-571 accordingly to keep them consistent.

@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 28, 2020

Are you guys using some formatting tool

I use IntelliJ that does have some auto-formatting. Use it basically with default settings. In doubt, disable auto-formatting to avoid unnecessary reformatting. -- Some things are also a little bit of "personal taste"...

I updated the KIP-571 accordingly to keep them consistent.

Thanks! Can you send a follow up email to the voting thread explaining the change? To make sure nobody has concerns about it.

@feyman2016
Copy link
Copy Markdown
Contributor Author

@mjsax Sure, updated in the voting thread of KIP-571

Kvicii pushed a commit to Kvicii/kafka that referenced this pull request May 30, 2020
* 'trunk' of github.com:apache/kafka: (36 commits)
  Remove redundant `containsKey` call in KafkaProducer (apache#8761)
  KAFKA-9494; Include additional metadata information in DescribeConfig response (KIP-569) (apache#8723)
  KAFKA-10061; Fix flaky `ReassignPartitionsIntegrationTest.testCancellation` (apache#8749)
  KAFKA-9130; KIP-518 Allow listing consumer groups per state (apache#8238)
  KAFKA-9501: convert between active and standby without closing stores (apache#8248)
  KAFKA-10056; Ensure consumer metadata contains new topics on subscription change (apache#8739)
  MINOR: Log the reason for coordinator discovery failure (apache#8747)
  KAFKA-10029; Don't update completedReceives when channels are closed to avoid ConcurrentModificationException (apache#8705)
  MINOR: remove unnecessary timeout for admin request (apache#8738)
  MINOR: Relax Percentiles test (apache#8748)
  MINOR: regression test for task assignor config (apache#8743)
  MINOR: Update documentation.html to refer to 2.6 (apache#8745)
  MINOR: Update documentation.html to refer to 2.5 (apache#8744)
  KAFKA-9673: Filter and Conditional SMTs (apache#8699)
  KAFKA-9971: Error Reporting in Sink Connectors (KIP-610) (apache#8720)
  KAFKA-10052: Harden assertion of topic settings in Connect integration tests (apache#8735)
  MINOR: Slight MetadataCache tweaks to avoid unnecessary work (apache#8728)
  KAFKA-9802; Increase transaction timeout in system tests to reduce flakiness (apache#8736)
  KAFKA-10050: kafka_log4j_appender.py fixed for JDK11 (apache#8731)
  KAFKA-9146: Add option to force delete active members in StreamsResetter (apache#8589)
  ...

# Conflicts:
#	core/src/main/scala/kafka/log/Log.scala
@mjsax mjsax added the kip Requires or implements a KIP label Jun 12, 2020
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

kip Requires or implements a KIP streams tools

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants