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-6150: Purge repartition topics #4270

Closed

Conversation

guozhangwang
Copy link
Contributor

@guozhangwang guozhangwang commented Nov 28, 2017

  1. Add the repartition topics information into ProcessorTopology: personally I do not like leaking this information into the topology but it seems not other simple way around.
  2. StreamTask: added one more function to expose the consumed offsets from repartition topics only.
  3. TaskManager: use the AdminClient to send the gathered offsets to delete only if a) previous call has completed and client intentionally ignore-and-log any errors, or b) no requests have ever called before.

NOTE that this code depends on the assumption that purge is only called right after the commit has succeeded, hence we presume all consumed offsets are committed.

  1. MINOR: Added a few more constructor for ProcessorTopology for cleaner unit tests.
  2. MINOR: Extracted MockStateStore out of the deprecated class.
  3. MINOR: Made a pass over some unit test classes for clean ups.

Committer Checklist (excluded from commit message)

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

@guozhangwang guozhangwang changed the title [WIP] KAFKA-6150: Purge repartition topics KAFKA-6150: Purge repartition topics Nov 29, 2017
@@ -890,17 +888,17 @@ public synchronized ProcessorTopology buildGlobalStateTopology() {
}

private ProcessorTopology build(final Set<String> nodeGroup) {
final List<ProcessorNode> processorNodes = new ArrayList<>(nodeFactories.size());
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Removed the arraylist and use linked hashmap for processorMap to achieve the same thing: following insertion ordering.

@guozhangwang
Copy link
Contributor Author

@dguy @mjsax @bbejeck this is ready to review.

@dguy
Copy link
Contributor

dguy commented Dec 1, 2017

@guozhangwang should this PR have all of those commits in it?

@guozhangwang
Copy link
Contributor Author

@dguy yeah a lot of those are rebasing from trunk and other branches, and many others are minor commits.

@guozhangwang
Copy link
Contributor Author

A few more notes on the PR itself:

  1. Today no repartition topics are used as the changelog topics of state store, so re-processing / restoration is not affected by this change. But we need to keep in mind to exclude repartition topics used for changelogs if optimizations make them so.

  2. I am still working on the integration test of it, but I'd rather have another PR after this to not make it too large.

  3. There is another change I need to make: set the default segment size of repartition topic partitions to a small value (I'm thinking 50MB) so that record deletion can kicks in. But I may do that after KAFKA-4857: Replace StreamsKafkaClient with AdminClient in Kafka Streams #4242

Copy link
Contributor

@dguy dguy left a comment

Choose a reason for hiding this comment

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

Thanks @guozhangwang, left some comments.

@@ -33,7 +33,7 @@

private final Map<TopicPartition, KafkaFuture<DeletedRecords>> futures;

DeleteRecordsResult(Map<TopicPartition, KafkaFuture<DeletedRecords>> futures) {
public DeleteRecordsResult(Map<TopicPartition, KafkaFuture<DeletedRecords>> futures) {
Copy link
Contributor

Choose a reason for hiding this comment

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

making this public means it becomes part of the public API - 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.

Good point. I will just keep the KafkaFuture from the result in TaskManager then.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@dguy Tried to remove this constructor from TaskManagerTest but it turns out ugly. So I plan to use the MockAdminClient after #4242 instead of the mock and create the object.

Collections.<String>emptySet());
}

public static ProcessorTopology withSources(final List<ProcessorNode> processorNodes,
Copy link
Contributor

Choose a reason for hiding this comment

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

package private?

Collections.<String>emptySet());
}

public static ProcessorTopology withLocalStores(final List<StateStore> stateStores,
Copy link
Contributor

Choose a reason for hiding this comment

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

package private ?

Collections.<String>emptySet());
}

public static ProcessorTopology withGlobalStores(final List<StateStore> stateStores,
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto

Collections.<String>emptySet());
}

public static ProcessorTopology withRepartitionTopics(final List<ProcessorNode> processorNodes,
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto

}

public Set<SinkNode> sinks() {
return new HashSet<>(sinkByTopics.values());
return new HashSet<>(sinksByTopic.values());
Copy link
Contributor

Choose a reason for hiding this comment

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

as above

public Map<String, String> storeToChangelogTopic() {
return storeToChangelogTopic;
}

public List<StateStore> globalStateStores() {
return globalStateStores;
public boolean isTopicInternalTransient(String topic) {
Copy link
Contributor

Choose a reason for hiding this comment

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

package private ?

Copy link
Contributor

Choose a reason for hiding this comment

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

maybe name it isInternalTransientTopic?

Copy link
Member

Choose a reason for hiding this comment

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

isInternalTopicTransient ?

final Map<TopicPartition, Long> purgableConsumedOffsets = new HashMap<>();
for (Map.Entry<TopicPartition, Long> entry : consumedOffsets.entrySet()) {
if (topology.isTopicInternalTransient(entry.getKey().topic()))
purgableConsumedOffsets.put(entry.getKey(), entry.getValue() + 1);
Copy link
Contributor

Choose a reason for hiding this comment

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

is that right to purge the consumed offset + 1?

Copy link
Member

Choose a reason for hiding this comment

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

nit: extract .getKey() and call only once.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

is that right to purge the consumed offset + 1?

The delete records call is to delete records before the given offset, similar to commit offset, so this is correct.

if (deleteRecordsResult == null || deleteRecordsResult.all().isDone()) {

if (deleteRecordsResult != null && deleteRecordsResult.all().isCompletedExceptionally()) {
log.info("Previous delete-records request has failed: {}. Try sending the new request now", deleteRecordsResult.lowWatermarks());
Copy link
Contributor

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 info level? seems more like debug to me, i.e., is a user actually going to care about this?

Copy link
Member

Choose a reason for hiding this comment

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

Do we want to print the whole map as-is? This might be quite large. Maybe filter for failed request?

@@ -542,6 +552,39 @@ public void shouldPropagateExceptionFromStandbyCommit() {
verify(standby);
}

@Test
public void shouldSendPurgeData() {
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 add a test that also checks that a purge request is sent after a previous one has completed?

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.

Thanks for the patch! Couple of comments.

if (!factory.isWindowStore()) {
return new InternalTopicConfig(name,
Collections.singleton(InternalTopicConfig.CleanupPolicy.compact),
factory.logConfig());
} else {
final InternalTopicConfig config = new InternalTopicConfig(name,
Copy link
Member

Choose a reason for hiding this comment

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

nit: move name to next line

if (!factory.isWindowStore()) {
return new InternalTopicConfig(name,
Collections.singleton(InternalTopicConfig.CleanupPolicy.compact),
factory.logConfig());
} else {
Copy link
Member

Choose a reason for hiding this comment

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

Nit: We don't need and else here as the if part end with return -- if you prefer if-else pattern, maybe more return after if-else block (seems to be cleaner to me -- but it's quite subjective)

public Map<String, String> storeToChangelogTopic() {
return storeToChangelogTopic;
}

public List<StateStore> globalStateStores() {
return globalStateStores;
public boolean isTopicInternalTransient(String topic) {
Copy link
Member

Choose a reason for hiding this comment

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

isInternalTopicTransient ?

@@ -364,6 +365,16 @@ private void commitOffsets(final boolean startNewTransaction) {
}
}

Map<TopicPartition, Long> purgableOffsets() {
final Map<TopicPartition, Long> purgableConsumedOffsets = new HashMap<>();
for (Map.Entry<TopicPartition, Long> entry : consumedOffsets.entrySet()) {
Copy link
Member

Choose a reason for hiding this comment

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

nit: add final

final Map<TopicPartition, Long> purgableConsumedOffsets = new HashMap<>();
for (Map.Entry<TopicPartition, Long> entry : consumedOffsets.entrySet()) {
if (topology.isTopicInternalTransient(entry.getKey().topic()))
purgableConsumedOffsets.put(entry.getKey(), entry.getValue() + 1);
Copy link
Member

Choose a reason for hiding this comment

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

nit: extract .getKey() and call only once.

assertThat(map, equalTo(Collections.singletonMap(repartition, 11L)));
}

private StreamTask createStatefulTask(boolean eosEnabled, boolean logged) {
Copy link
Member

Choose a reason for hiding this comment

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

nit: add final (same below)

@@ -328,7 +328,7 @@ public void shouldCommitAfterTheCommitInterval() {

@SuppressWarnings({"ThrowableNotThrown", "unchecked"})
private TaskManager mockTaskManagerCommit(final Consumer<byte[], byte[]> consumer, final int numberOfCommits, final int commits) {
final TaskManager taskManager = EasyMock.createMock(TaskManager.class);
final TaskManager taskManager = EasyMock.createNiceMock(TaskManager.class);
Copy link
Member

Choose a reason for hiding this comment

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

Why?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It has one more functions that is not focus of the test now.


EasyMock.expect(active.recordsToDelete()).andReturn(recordsToDelete).once();
EasyMock.expect(adminClient.deleteRecords(recordsToDelete)).andReturn(deleteRecordsResult).once();
replay();
Copy link
Member

Choose a reason for hiding this comment

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

Do we not need to pass in active and adminClient here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

replay() will replay all the mock clients.

public void shouldIgnorePurgeDataErrors() {
final KafkaFutureImpl<DeletedRecords> futureDeletedRecords = new KafkaFutureImpl<>();
final Map<TopicPartition, RecordsToDelete> recordsToDelete = Collections.singletonMap(t1p1, RecordsToDelete.beforeOffset(5L));
final DeleteRecordsResult deleteRecordsResult = new DeleteRecordsResult(Collections.singletonMap(t1p1, (KafkaFuture<DeletedRecords>) futureDeletedRecords));
Copy link
Member

Choose a reason for hiding this comment

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

Cast unnecessary?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Unfortunately it is not, since we are using KafkaFutureImpl for this test (but not the other) in order to call completeExceptionally.

Copy link
Member

Choose a reason for hiding this comment

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

Can't we specify the type via Collections.<KafkaFuture>singletonMap(..) to make it work? A Map of type KafkaFuture should accept KafkaFutureImpl -- or do I miss something? Generics can be confusion sometimes :)


EasyMock.expect(active.recordsToDelete()).andReturn(recordsToDelete).times(2);
EasyMock.expect(adminClient.deleteRecords(recordsToDelete)).andReturn(deleteRecordsResult).times(2);
replay();
Copy link
Member

Choose a reason for hiding this comment

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

as above. pass in mocks?

@guozhangwang
Copy link
Contributor Author

Addressed comments.

@dguy
Copy link
Contributor

dguy commented Dec 4, 2017

retest this please

Copy link
Contributor

@dguy dguy left a comment

Choose a reason for hiding this comment

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

Thanks @guozhangwang, LGTM

@guozhangwang
Copy link
Contributor Author

Thanks for the reviews! Merged to trunk.

@asfgit asfgit closed this in 4b8a29f Dec 4, 2017
@guozhangwang guozhangwang deleted the K6150-purge-repartition-topics branch February 14, 2018 19:51
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants