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-9615: Clean up task/producer create and close #8213

Merged
merged 14 commits into from
Mar 5, 2020
Merged

KAFKA-9615: Clean up task/producer create and close #8213

merged 14 commits into from
Mar 5, 2020

Conversation

vvcephei
Copy link
Contributor

@vvcephei vvcephei commented Mar 3, 2020

Consolidates task/producer management.

Now, exactly one component manages the creation and destruction of Producers,
whether they are per-thread or per-task.

Committer Checklist (excluded from commit message)

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

@vvcephei vvcephei requested a review from guozhangwang March 3, 2020 22:16
Copy link
Contributor Author

@vvcephei vvcephei left a comment

Choose a reason for hiding this comment

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

@guozhangwang , do you mind taking a look at this refactor?


import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE;

class ActiveTaskCreator {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Pulled out of StreamThread (could have been done a long time ago, since it was a static class anyway). I didn't embed it in TaskManager, just to keep the file size lower.

I also dropped AbstractTaskCreator, since the creation of Active and Standby tasks are only similar, not exactly the same. We weren't really using the abstraction for much except de-duplicating a few field declarations. On the con side, the abstraction made it hard to see that we were requiring several arguments for Standby task creation that were actually not ever used. The indirection also made it harder to read the task creation logic.

Copy link
Contributor

Choose a reason for hiding this comment

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

Sounds good to me

@@ -25,7 +25,7 @@

import java.util.Map;

public interface RecordCollector extends AutoCloseable {
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 never used it as an AutoCloseable, and having it makes it hard to trace the callers.

@@ -262,7 +262,7 @@ public void close() {
if (eosEnabled) {
streamsProducer.abortTransaction();
}
streamsProducer.close();
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Only the creator may close the producer, but we can go ahead and call flush() here.

@@ -544,11 +306,7 @@ public static StreamThread create(final InternalTopologyBuilder builder,

final ThreadCache cache = new ThreadCache(logContext, cacheSizeBytes, streamsMetrics);

final Map<TaskId, Producer<byte[], byte[]>> taskProducers = new HashMap<>();
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Managed fully inside the task factory now.

You'll also notice a bunch of references to the producers are similarly gone in the following lines.

Comment on lines -1130 to -1132
threadProducer == null ?
Collections.emptySet() :
Collections.singleton(getThreadProducerClientId(this.getName())),
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sooo many switch statements are now gone.

Copy link
Contributor

Choose a reason for hiding this comment

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

Nice!

}
}
return result;
return taskManager.producerMetrics();
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 don't manage the producers anymore, so we can defer to the taskManager (who will defer to the active task creator, but that's none of the thread's business)

Comment on lines +59 to +62
public StreamsProducer(final Producer<byte[], byte[]> producer,
final boolean eosEnabled,
final LogContext logContext,
final String applicationId) {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is a bit on the side, but there was some wacky stuff going on in here, and afaict, the only purpose of all the nullable fields was to allow including the task id in exception messages. Do we really need to do that? If so, I'll just add the logContext to the exception message instead, since it already has the task id in it.

Copy link
Contributor

Choose a reason for hiding this comment

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

Actually the logContext.logPrefix() should have the format of stream-thread [%s] task [%s] already, so all the log4j entries are good. For exception messages, we can just get the prefix and then encode that into the exception message.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yep, that's what I was thinking. I'll go ahead and do it.

"Error encountered sending record to topic %s%s due to:%n%s",
record.topic(),
taskId == null ? "" : " " + logMessage,
uncaughtException.toString());
Copy link
Contributor Author

Choose a reason for hiding this comment

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

no need to add the exception to the message, since we also pass it as the cause below.

@@ -212,17 +190,6 @@ public void flush() {
producer.flush();
}

public void close() {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

No one who has a reference to the StreamsProducer has any business calling close, so the method is gone now.

@@ -227,7 +227,7 @@ public void handleAssignment(final Map<TaskId, Set<TopicPartition>> activeTasks,
}

if (!standbyTasksToCreate.isEmpty()) {
standbyTaskCreator.createTasks(mainConsumer, standbyTasksToCreate).forEach(this::addNewTask);
standbyTaskCreator.createTasks(standbyTasksToCreate).forEach(this::addNewTask);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Standby tasks don't need the mainConsumer (obvious, in retrospect).

Copy link
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.

I made a pass over the PR.

// Instead, we should register and record the metrics properly inside of the record collector.
Map<TaskId, StreamTask> fixmeStreamTasks() {
return tasks.values().stream().filter(t -> t instanceof StreamTask).map(t -> (StreamTask) t).collect(Collectors.toMap(Task::id, t -> t));
Map<MetricName, Metric> producerMetrics() {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: These two functions are not for testing only.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks for the feedback; I didn't understand this particular comment, though.

Copy link
Contributor

Choose a reason for hiding this comment

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

The comment line above these two method declaration says the following functions are for test only, but these two functions are not.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ah, I just found what you were talking about:

    // below are for testing only

I didn't notice that up there.

final StreamThread.AbstractTaskCreator<? extends Task> activeTaskCreator,
final StreamThread.AbstractTaskCreator<? extends Task> standbyTaskCreator,
final Map<TaskId, Producer<byte[], byte[]>> taskProducers,
final ActiveTaskCreator activeTaskCreator,
Copy link
Contributor

Choose a reason for hiding this comment

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

In shutdown(final boolean clean) we should also release task producers as well right?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yep. Good catch.

Comment on lines -1130 to -1132
threadProducer == null ?
Collections.emptySet() :
Collections.singleton(getThreadProducerClientId(this.getName())),
Copy link
Contributor

Choose a reason for hiding this comment

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

Nice!

Comment on lines +59 to +62
public StreamsProducer(final Producer<byte[], byte[]> producer,
final boolean eosEnabled,
final LogContext logContext,
final String applicationId) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Actually the logContext.logPrefix() should have the format of stream-thread [%s] task [%s] already, so all the log4j entries are good. For exception messages, we can just get the prefix and then encode that into the exception message.

@@ -262,7 +262,7 @@ public void close() {
if (eosEnabled) {
streamsProducer.abortTransaction();
}
streamsProducer.close();
streamsProducer.flush();
Copy link
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 are introducing a latency regression without EOS here: in the old code when closing without EOS we actually do nothing, and now we would block on flushing.

On the other hand, flushing maybe needed when we close a task to make sure all the tasks' records are acked already.

If the task is in RUNNING before shutting down, we would always commit before closing, so flush is already called; if the task is in RESTORING / SUSPENDED there's nothing written from this task, so a flush is not needed. So I think it is safe to not call flush after all.

Copy link
Member

Choose a reason for hiding this comment

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

Sounds reasonable to me

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ok, I'll swap this out.

@@ -612,25 +582,4 @@ public void shouldFailOnEosAbortTxFatal() {

assertThat(thrown.getMessage(), equalTo("KABOOM!"));
}

@Test
public void shouldFailOnCloseFatal() {
Copy link
Contributor

Choose a reason for hiding this comment

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

We do not created new test classes of ActiveTaskCreator / StandbyTaskCreator, but we should still have those coverage to make sure the exception thrown from producers are wrapped correctly.

Also it seems in the new code we no long rethrow -- is that intentional. I left a comment above.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good catch. Actually, there's a bunch of coverage missing from TaskManager. I'll add several more tests.

try {
threadProducer.close();
} catch (final RuntimeException e) {
log.error("Failed to close producer due to the following error:", e);
Copy link
Contributor

Choose a reason for hiding this comment

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

The old code we re-throw exceptions whereas here we just swallow the error. Is that intentional?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ah, thanks for noticing this. I meant to ask about it in here.

I thought it was strange that we would only re-throw when closing the task producers, not the thread producer. It seems like we should do the same thing in both cases, but which thing should we do?

I went with an error log in both cases, but it sounds like you wanted to throw the exception instead. Should we also rethrow for the thread producer?

Copy link
Contributor

Choose a reason for hiding this comment

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

Yeah I think we should make them consistent: previously the closing producer is within task#close and if it is called via closeDirty we should make sure it never throws. Now since it is extracted out of the close call we should just rethrow for both cases.

@@ -201,7 +201,7 @@ private KeyValueStoreTestDriver(final StateSerdes<K, V> serdes) {
logContext,
new TaskId(0, 0),
consumer,
new StreamsProducer(logContext, producer),
new StreamsProducer(producer, null != null, logContext, null),
Copy link
Contributor

Choose a reason for hiding this comment

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

null != null?

Copy link
Contributor

Choose a reason for hiding this comment

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

lol

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Oops! Auto-refactoring. I already made two passes to clean these up, looks like I missed one.

Copy link
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.

Nice cleanup. There is quite some overlap with #8215 even if both PR address a different issue.

return createdTasks;
}

public void releaseProducer() {
Copy link
Member

Choose a reason for hiding this comment

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

closeThreadProducer()

The method below is not public -- does this one need to be public?

Copy link
Contributor

Choose a reason for hiding this comment

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

even better maybeCloseThreadProducer

}
}

void releaseProducer(final TaskId id) {
Copy link
Member

Choose a reason for hiding this comment

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

closeProducerForTask(TaskId) ?

}
}

public InternalTopologyBuilder builder() {
Copy link
Member

Choose a reason for hiding this comment

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

Does this need to be public?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hah, doesn't need to be there at all, actually.

return builder;
}

public StateDirectory stateDirectory() {
Copy link
Member

Choose a reason for hiding this comment

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

Does this need to be public?

@@ -262,7 +262,7 @@ public void close() {
if (eosEnabled) {
streamsProducer.abortTransaction();
}
streamsProducer.close();
streamsProducer.flush();
Copy link
Member

Choose a reason for hiding this comment

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

Sounds reasonable to me


if (threadProducer == null) {
// create one producer per task for EOS
// TODO: after KIP-447 this would be removed
Copy link
Contributor

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 need to keep this TODO, as only after a stream 3.0 is there, we shall remove the support for task producer.

log.error("Failed to close producer due to the following error:", e);
}
}
if (!taskProducers.isEmpty()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Should we throw illegal state first, since we are already in an error state?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

If we're going to rename the method to specify that it should do exactly "close thread producer", then this check is no longer appropriate.

return createdTasks;
}

public void releaseProducer() {
Copy link
Contributor

Choose a reason for hiding this comment

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

even better maybeCloseThreadProducer

@vvcephei
Copy link
Contributor Author

vvcephei commented Mar 4, 2020

Hey @guozhangwang , @mjsax , and @abbccdda ,

I've addressed all the feedback. In particular I added about 500 lines of missing tests to TaskManagerTest. It looks like we've done a fair amount of un-unit-tested refactoring in TaskManagerTest, so I figured I'd go ahead and add the missing coverage while I was adding coverage for this refactor itself. Be forewarned, the tests are... extensive.

Copy link
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.

@vvcephei I made a pass over the testing code, and only have a couple minor comments.

After addressing them please feel free to merge.

try {
activeTaskCreator.closeAndRemoveTaskProducerIfNeeded(task.id());
} catch (final RuntimeException e) {
log.debug("Error handling lostAll", e);
Copy link
Contributor

Choose a reason for hiding this comment

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

  1. Let's make it a warn instead of a debug.
  2. The error message can be more specific here: Error closing task producer for task {} while handling lostAll.

Copy link
Contributor

Choose a reason for hiding this comment

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

+1

if (clean) {
firstException.compareAndSet(null, e);
} else {
log.warn("Ignoring an exception while closing task producer.", e);
Copy link
Contributor

Choose a reason for hiding this comment

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

Ditto here about error message

@@ -440,6 +872,41 @@ public void shouldCommitActiveAndStandbyTasks() {
assertThat(taskManager.commitAll(), equalTo(2));
}

@Test
public void shouldNotCommitActiveAndStandbyTasks() {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: ...WhileRebalanceInProgress

assertTrue(eosMockProducer.history().isEmpty());
assertThat(eosMockProducer.uncommittedRecords().size(), equalTo(1));
assertThat(eosMockProducer.uncommittedRecords().get(0), equalTo(record));
assertThat(eosMockProducer.transactionInFlight(), is(true));
Copy link
Contributor

Choose a reason for hiding this comment

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

I was wondering what's the standard for using assertThat vs assertTrue? Do we have a convention to follow?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

assertThat is nicer in general, but it doesn't really matter. In this case, IDEA offered to translate, and I was already changing a lot of assertions, so I just accepted the translation.

@@ -24,14 +24,20 @@
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Metric;
Copy link
Contributor

Choose a reason for hiding this comment

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

Are the tests added in TaskManager only trying for more coverage? @vvcephei

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yep, that's right.

Copy link
Contributor

@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, only have some minor comments about tests. After addressing them, feel free to merge.

@vvcephei
Copy link
Contributor Author

vvcephei commented Mar 5, 2020

The last commit was trivial, and the Streams tests passed locally for me, so I'm going to go ahead and merge.

@vvcephei vvcephei merged commit 78374a1 into apache:trunk Mar 5, 2020
@vvcephei vvcephei deleted the KAFKA-9615-cleanup-task-producers-2 branch March 5, 2020 20:20
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants