Skip to content

KAFKA-9441: remove prepareClose() to simplify task management#8833

Merged
mjsax merged 8 commits intoapache:trunkfrom
mjsax:kafka-9441-kip-447-follow-up-tm-refactoring
Jun 11, 2020
Merged

KAFKA-9441: remove prepareClose() to simplify task management#8833
mjsax merged 8 commits intoapache:trunkfrom
mjsax:kafka-9441-kip-447-follow-up-tm-refactoring

Conversation

@mjsax
Copy link
Copy Markdown
Member

@mjsax mjsax commented Jun 8, 2020

  • Removes prepareCloseClean() and prepareCloseDirty().
  • Removes state transition RUNNING -> CLOSED (tasks must be suspended before closing now)
  • replaces suspend() with suspendDirty and suspendAndPrepareCommit()
  • Decouples suspending/committing/closing (ie, no redundant code any longer, but enforces "linear" order of calls)

Call for review @guozhangwang @abbccdda @vvcephei @ableegoldman @cadonna

@mjsax mjsax added the streams label Jun 8, 2020
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.

When we suspend, we always want to commit.

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.

This logic is now followed in suspendCleanAndPrepareCommit() that must be called before a task can be closed.

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.

This is now done via suspendAndPrepareCommit()

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.

The "old" suspend() was called after committing, the "new" suspend() is now called before committing!

The old suspend logic is now handled via postCommit and close

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.

Instead of "blindly" writing a checkpoint in postCommit(), we only do it if a checkpoint get's scheduled.

Copy link
Copy Markdown
Member Author

@mjsax mjsax Jun 8, 2020

Choose a reason for hiding this comment

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

Mainly rewrite to use switch now -- however, we return a proper non-empty checkpoint on SUSPEND now.

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 comments.

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 update the exception text like while scheduling checkpoint

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

similar here

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

if commit is needed

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.

Should be if no commit is needed (ie, if we need to commit, we need to commit first! -- writing a checkpoint as long as a commit is needed implies we write the checkpoint too early).

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Seems applied to both RUNNING and SUSPENDED?

Copy link
Copy Markdown
Member Author

@mjsax mjsax Jun 9, 2020

Choose a reason for hiding this comment

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

For SUSPENDING, we should always write the checkpoint. Fixing.

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 comment could be removed IMHO.

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 want to throw here if the current state is CLOSED?

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.

My proposal is, to keep the methods idempotent.

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.

nit: we can throw illegal-state if the state() == RESTORING since it should never happen.

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 not just make suspend a no-op if the task is RESTORING? That seems more in line with how we handle things elsewhere

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.

For StandbyTasks, we never restore. When we do the state transition, we away make two transitions directly after each other from CREATE -> RESTORING -> RUNNING -- thus, state RESTORING is an invalid state for standby tasks.

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.

Right, by "check for RESTORING" I meant "throw an exception if state is restoring". It seems odd to check for RESTORING during suspend but not in any other StandbyTask method. Either it can never be in RESTORING and we are completely sure of that, and shouldn't check for RESTORING, or we should always check whether it's RESTORING and not just during suspend (eg also in postCommit)

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.

Just to clarify, I would support doing the former, ie don't check whether it's RESTORING here at all. But we should at least be consistent

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.

Ah, I see. I guess we check it in almost all method though. (we just missed initializeIfNeeded and resume() -- will add it there).

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 do this check implicitly for some case already, ie:

if (RUNNING) {
} else {
  throw
}

ie, only RUNNING is a valid state, and all others are invalid. Thus, it seems to be consistent if we add those checks elsewhere (or, what would be odd, exclude RESTORING from those implicit checks).

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.

Cool 👍

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Similar here, maybe we could leverage transitionTo to help throw the exception.

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

The partitionGroup.clear and partitionGroup.close are interchangeable right now, should we just consolidate both?

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 call.

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Prefer to throw different illegal state exception here than making comments

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.

What does it improve?

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Maybe not necessary after a second thought. However, one more question: why not making closeAndRecycleState idempotent as well?

Copy link
Copy Markdown
Member Author

@mjsax mjsax Jun 10, 2020

Choose a reason for hiding this comment

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

I see your point, but for this case, I would prefer to introduce a new state -- atm, closeAndRecycleState transits to CLOSED state what is the same as when we actually close a task -- however, the stateMgr would be closed for a proper CLOSED state, while for recycling the stateMgr is not closed -- so in general, the CLOSED state is not a "safe" state to provide idempotence. Thoughts?

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

That makes sense

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 could see we are trying to maintain the same behavior, but still why a restoring task won't need to close topology?

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.

Because the topology is only initialized when restoring is finished.

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Sg, should we move the log on L811 inside the if statement?

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.

nit: since this is a private function only called by suspend, we can modify the caller such that we only call this in RUNNING not in RESTORING, and then inside this function we do not need this check anymore.

Copy link
Copy Markdown
Contributor

@guozhangwang guozhangwang left a comment

Choose a reason for hiding this comment

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

@mjsax thanks for the cleanup! I left some meta comment about the Task interface thinking they can further be simplified, lmk wdyt.

Will make a thorough pass on the other code once we have the meta approach nailed down.

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.

I'm wondering if we could merge committableOffsetsAndMetadata with prepareCommit as well, letting the latter to return the map? See my other comment aside.

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.

That is certainly possible. Good catch!

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.

It seems to me that the reason we want to have two suspends and also merging the suspendClean with prepareCommit is that for StreamTask, if state SUSPENDED we want to skip prepareCommit. I feel it is a tad cleaner to separate them further into one suspend which does not try to call prepareCommit, and rely on whether prepareCommit should do anything or not based on both state (i.e. only running/restoring/suspended need to commit) and commitNeeded flag.

With that we can convert the callers as follows:

  1. suspendDirty(): just call suspend(), do not call prepareCommit().
  2. suspendCleanAndPrepareCommit():
    2.a) from task.closeAndRecycleState: call suspend(), and then call prepareCommit(); the second would check commitNeeded and if it was false, we would not try to flush / commit. Hence if the task just transited from other states to suspended, then commitNeeded should still be true.
    2.b) from taskManager directly: same as above, but for this call we always follow with a committableOffsetsAndMetadata getting the map of offsets, so I'm thinking we can merge prepareCommit with committableOffsetsAndMetadata as well: if the state is right and commitNeeded is set, execute the prepare committing procedure, and accumulate the offsets, otherwise returning null indicating no offsets needed to be committed.

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 also want to swallow exceptions in closeTopology() but we can work around this.

@mjsax mjsax force-pushed the kafka-9441-kip-447-follow-up-tm-refactoring branch from 7a98613 to 120fb8d Compare June 9, 2020 18:12
@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jun 9, 2020

Updates this PR.

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.

Just some minor comments

case CREATED:
// the task is created and not initialized, just re-write the checkpoint file
scheduleCheckpoint(emptyMap());
case RESTORING:
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 merge RESTORING and SUSPENDED?

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.

+1, IDEA also suggests it :)

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Sg, should we move the log on L811 inside the if statement?

* @throws StreamsException fatal error, should close the thread
*/
void prepareCommit();
Map<TopicPartition, OffsetAndMetadata> prepareCommit();
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

add a @return comment for the struct

task.prepareCloseDirty();
try {
task.suspend();
} catch (final RuntimeException swallow) {
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 could get a follow-up newbie ticket, but it seems that we have a couple of catch and swallow cases in the task manager with clean flag, does it make sense to extract the executeAndMaybeSwallow to TaskManager class and share between cases?

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.

SGTM. Can you create a ticket?

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Done


@Test
public void shouldRecycleTask() {
EasyMock.expectLastCall();
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 this lastCall suggest?

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.

Ups.

Copy link
Copy Markdown
Contributor

@guozhangwang guozhangwang left a comment

Choose a reason for hiding this comment

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

Made a thorough pass over the code.

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.

nit: we can throw illegal-state if the state() == RESTORING since it should never happen.

prepareCommit();

if (state() == State.CREATED || state() == State.RUNNING) {
if (state() == State.CREATED || state() == State.SUSPENDED) {
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.

The comment below is not accurate anymore: we do not write checkpoint during recycle actually.

EDIT: actually, the updated offsetSnapshotSinceLastCommit seems not used since after this function we would create a new StreamTask and in between we do not check if commitNeeded at all. Could we remove line 175 then?

switch (state()) {
case CREATED:
case RUNNING:
case SUSPENDED:
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.

Ditto for line 195: we do not need to update the snapshot since we are closing the task already.

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.

nit: since this is a private function only called by suspend, we can modify the caller such that we only call this in RUNNING not in RESTORING, and then inside this function we do not need this check anymore.

log.info("Suspended restoring");
case RUNNING:
try {
closeTopology();
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 skip calling this if we are in RESTORING; I have another comment below.

Also could we add javadoc on top explaining what exception can be thrown?

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Is this addressed?

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. RESTORING above is it's own "case" branch now (before RUNNING and RESTORING was shared the code).

case RESTORING:
commitNeeded = false;
case RUNNING:
case SUSPENDED:
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.

What about 1) move the line 387/388 out of the switch, also line 400 after the switch block, 2) and then make these three states separate branches, so that we can avoid a mix of switch / if-else.


stateMgr.checkpoint(checkpointableOffsets());
if (state() == State.SUSPENDED) {
partitionGroup.clear();
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... this reads a bit weird to me. Can we call this in suspend instead? Also in that case we do not need to call this in close and closeAndRecycle.

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 cannot call it in suspend, because we would loose the partition-time information that we need in prepareCommit() (that is called after suspend()).

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.

Got it, makes sense.

Could you copy-paste the above as comment to remind other readers?

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.

Sure -- but if they change it (I also did the change originally) a unit test fails anyway :)

prepareClose(true);

suspend();
prepareCommit();
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.

I think we actually do not need to commit (including write-checkpoint) when closeAndRecycle actually, and only need to suspend the task before recycle it. But this is out of the scope and we can discuss about this in another PR (cc @ableegoldman ).

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.

Assuming rebalancing does not happen often (in a stable deployment) it might be re-mature optimization?

* Currently only changelog topic offsets need to be checkpointed.
*/
private Map<TopicPartition, Long> checkpointableOffsets() {
if (state() == State.RESTORING) {
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.

Should we return emptyMap if we are SUSPENDED as well?

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.

EDIT: actually, I think we need to accumulate the consumed offsets when we just transited to suspend and then called prepareCommit, but if we are already in suspended then it is actually okay to return an emptyMap. However since we do not know if we have just transited to suspended and the below code should not be a big overhead, we can just keep it as-is.

So please ignore my previous comment :)

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Is this logic necessary? I don't think we would populate data in record collector or consumed offsets until we start processing?

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 point!

task.prepareCloseClean();
final Map<TopicPartition, OffsetAndMetadata> committableOffsets = task
.committableOffsetsAndMetadata();
task.suspend();
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.

question: I cannot remember why we need to commit those still owned tasks during handle-assignment, is that necessary? Or is that just an optimization: since we are going to commit anyways, let's just commit everyone.

If that's the case, we can refresh the last-commit timestamp as well.

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.

Even in that case, in line 205 we could check task.commitNeeded() && task.isActive right?

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.

For eoa-beta, if we commit, we alway need to commit all tasks. And to not distinguish between non-eos/eos-alpha vs eos-beta, be decided to just commit all tasks for all cases.

And we don't own StreamThread#lastCommitMs so we cannot update it.

For L205: the outter if checks already if task.isActive

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.

@mjsax @guozhangwang why do we need to commit at all during handleAssignment? Shouldn't we have already committed all tasks that need to be committed during handleRevocation?

That's not exactly a bug, I'm just wondering if it's necessary?

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.

We may not call handleRevocation before calling handleAssignment so the task to close may not be in SUSPENDED state yet, and hence do close them we need to commit their states. For other tasks, they are not necessarily committing but I think the point was, that since we are going to send one commit request anyways so just commit for everyone --- note that flushing can indeed be skipped, which is what KAFKA-9450 covers

@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jun 10, 2020

Updated.

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.

Just some minor comments

log.info("Suspended restoring");
case RUNNING:
try {
closeTopology();
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Is this addressed?

case SUSPENDED:
final Map<TopicPartition, Long> partitionTimes = extractPartitionTimes();

committableOffsets = new HashMap<>(consumedOffsets.size());
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

+1, this seems not really necessary atm.

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Maybe not necessary after a second thought. However, one more question: why not making closeAndRecycleState idempotent as well?

* Currently only changelog topic offsets need to be checkpointed.
*/
private Map<TopicPartition, Long> checkpointableOffsets() {
if (state() == State.RESTORING) {
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Is this logic necessary? I don't think we would populate data in record collector or consumed offsets until we start processing?

task.prepareCloseDirty();
try {
task.suspend();
} catch (final RuntimeException swallow) {
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Done

@guozhangwang
Copy link
Copy Markdown
Contributor

LGTM. Please feel free to merge after addressed @abbccdda 's comments above.

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.

LGTM, thanks for the patch!

public void closeAndRecycleState() {
prepareClose(true);
suspend();
prepareCommit();
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 call prepareCommit (or suspend for that matter)?

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.

Both now do what prepareClose() did before.

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 just mean, why not inline that? I'm just imagining coming back to this code in a few months and wondering why we need to suspend a task before recycling, or why we call prepareCommit but don't then actually commit, etc

Copy link
Copy Markdown
Member

@ableegoldman ableegoldman Jun 11, 2020

Choose a reason for hiding this comment

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

Nevermind, I see that's the pattern we follow everywhere else

@mjsax mjsax merged commit 8e083e1 into apache:trunk Jun 11, 2020
mjsax added a commit that referenced this pull request Jun 11, 2020
Reviewers: Boyang Chen <boyang@confluent.io>, Guozhang Wang <guozhang@confluent.io>, A. Sophie Blee-Goldman <sophie@confluent.io>
@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jun 11, 2020

Merged to trunk and cherry-picked to 2.6 branch.

@mjsax mjsax deleted the kafka-9441-kip-447-follow-up-tm-refactoring branch June 11, 2020 01:12
@mjsax mjsax added the kip Requires or implements a KIP label Jun 12, 2020
Kvicii pushed a commit to Kvicii/kafka that referenced this pull request Jun 13, 2020
* 'trunk' of github.com:apache/kafka: (42 commits)
  HOTFIX: Fix compile error in TopicAdminTest (apache#8866)
  KAFKA-10144: clean up corrupted standby tasks before attempting a commit (apache#8849)
  KAFKA-10157: Fix broken tests due to InterruptedException from FinalizedFeatureChangeListener (apache#8857)
  KAFKA-9432: automated protocol for DescribeConfigs (apache#8312)
  KAFKA-10049: Fixed FKJ bug where wrapped serdes are set incorrectly when using default StreamsConfig serdes (apache#8764)
  KAFKA-10027: Implement read path for feature versioning system (KIP-584) (apache#8680)
  KAFKA-10085: correctly compute lag for optimized source changelogs (apache#8787)
  KAFKA-10086: Integration test for ensuring warmups are effective (apache#8818)
  KAFKA-9374: Make connector interactions asynchronous (apache#8069)
  MINOR: reduce sizeInBytes for percentiles metrics (apache#8835)
  KAFKA-10115: Incorporate errors.tolerance with the Errant Record Reporter (apache#8829)
  KAFKA-9216: Enforce that Connect’s internal topics use `compact` cleanup policy (apache#8828)
  KAFKA-9845: Warn users about using config providers with plugin.path property (apache#8455)
  KAFKA-7833: Add missing test (apache#8847)
  KAFKA-9066: Retain metrics for failed tasks (apache#8502)
  KAFKA-9841: Revoke duplicate connectors and tasks when zombie workers return with an outdated assignment (apache#8453)
  KAFKA-9985: Sink connector may exhaust broker when writing in DLQ (apache#8663)
  KAFKA-9441: remove prepareClose() to simplify task management (apache#8833)
  KAFKA-7833: Add Global/StateStore name conflict check (apache#8825)
  KAFKA-9969: Exclude ConnectorClientConfigRequest from class loading isolation (apache#8630)
  ...
ijuma added a commit to ijuma/kafka that referenced this pull request Nov 17, 2020
…t-for-generated-requests

* apache-github/trunk: (248 commits)
  KAFKA-10049: Fixed FKJ bug where wrapped serdes are set incorrectly when using default StreamsConfig serdes (apache#8764)
  KAFKA-10027: Implement read path for feature versioning system (KIP-584) (apache#8680)
  KAFKA-10085: correctly compute lag for optimized source changelogs (apache#8787)
  KAFKA-10086: Integration test for ensuring warmups are effective (apache#8818)
  KAFKA-9374: Make connector interactions asynchronous (apache#8069)
  MINOR: reduce sizeInBytes for percentiles metrics (apache#8835)
  KAFKA-10115: Incorporate errors.tolerance with the Errant Record Reporter (apache#8829)
  KAFKA-9216: Enforce that Connect’s internal topics use `compact` cleanup policy (apache#8828)
  KAFKA-9845: Warn users about using config providers with plugin.path property (apache#8455)
  KAFKA-7833: Add missing test (apache#8847)
  KAFKA-9066: Retain metrics for failed tasks (apache#8502)
  KAFKA-9841: Revoke duplicate connectors and tasks when zombie workers return with an outdated assignment (apache#8453)
  KAFKA-9985: Sink connector may exhaust broker when writing in DLQ (apache#8663)
  KAFKA-9441: remove prepareClose() to simplify task management (apache#8833)
  KAFKA-7833: Add Global/StateStore name conflict check (apache#8825)
  KAFKA-9969: Exclude ConnectorClientConfigRequest from class loading isolation (apache#8630)
  KAFKA-9991: Fix flaky unit tests (apache#8843)
  KAFKA-10014; Always try to close all channels in Selector#close (apache#8685)
  KAFKA-10079: improve thread-level stickiness (apache#8775)
  MINOR: Print all removed dynamic members during join complete (apache#8816)
  ...
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

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants