-
Notifications
You must be signed in to change notification settings - Fork 14.1k
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-18058: Share group state record pruning impl. #18014
Changes from 2 commits
546eda7
dd575d4
3dc1798
9d19c0b
87f5a1d
d566e22
6ae8337
c49b7db
f616f85
bea9566
b0d4cca
9a342c2
e77c16d
6d889bc
b30df09
f56477b
055ef21
7e62122
18a44e1
74dcd45
f254399
a9a986e
b68b014
1cc890a
e1a38ab
ae6333a
8244bdc
c798254
75eb4d2
be0310f
a5aa93f
78d8c49
9714ce6
7e420cb
e3db1dc
1d60249
502b5e7
d55abdd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,6 +24,7 @@ | |
import java.util.Optional; | ||
|
||
import static org.apache.kafka.common.config.ConfigDef.Importance.HIGH; | ||
import static org.apache.kafka.common.config.ConfigDef.Importance.LOW; | ||
import static org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM; | ||
import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; | ||
import static org.apache.kafka.common.config.ConfigDef.Type.INT; | ||
|
@@ -71,6 +72,10 @@ public class ShareCoordinatorConfig { | |
public static final int APPEND_LINGER_MS_DEFAULT = 10; | ||
public static final String APPEND_LINGER_MS_DOC = "The duration in milliseconds that the share coordinator will wait for writes to accumulate before flushing them to disk."; | ||
|
||
public static final String STATE_TOPIC_PRUNE_INTERVAL = "share.coordinator.state.topic.prune.interval.ms"; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This should be |
||
public static final int STATE_TOPIC_PRUNE_INTERVAL_DEFAULT = 5 * 60 * 1000; // 5 minutes | ||
public static final String STATE_TOPIC_PRUNE_INTERVAL_DOC = "The duration in milliseconds that the share coordinator will wait between pruning eligible records in share-group state topic."; | ||
|
||
public static final ConfigDef CONFIG_DEF = new ConfigDef() | ||
.define(STATE_TOPIC_NUM_PARTITIONS_CONFIG, INT, STATE_TOPIC_NUM_PARTITIONS_DEFAULT, atLeast(1), HIGH, STATE_TOPIC_NUM_PARTITIONS_DOC) | ||
.define(STATE_TOPIC_REPLICATION_FACTOR_CONFIG, SHORT, STATE_TOPIC_REPLICATION_FACTOR_DEFAULT, atLeast(1), HIGH, STATE_TOPIC_REPLICATION_FACTOR_DOC) | ||
|
@@ -81,7 +86,8 @@ public class ShareCoordinatorConfig { | |
.define(LOAD_BUFFER_SIZE_CONFIG, INT, LOAD_BUFFER_SIZE_DEFAULT, atLeast(1), HIGH, LOAD_BUFFER_SIZE_DOC) | ||
.define(STATE_TOPIC_COMPRESSION_CODEC_CONFIG, INT, (int) STATE_TOPIC_COMPRESSION_CODEC_DEFAULT.id, HIGH, STATE_TOPIC_COMPRESSION_CODEC_DOC) | ||
.define(APPEND_LINGER_MS_CONFIG, INT, APPEND_LINGER_MS_DEFAULT, atLeast(0), MEDIUM, APPEND_LINGER_MS_DOC) | ||
.define(WRITE_TIMEOUT_MS_CONFIG, INT, WRITE_TIMEOUT_MS_DEFAULT, atLeast(1), HIGH, WRITE_TIMEOUT_MS_DOC); | ||
.define(WRITE_TIMEOUT_MS_CONFIG, INT, WRITE_TIMEOUT_MS_DEFAULT, atLeast(1), HIGH, WRITE_TIMEOUT_MS_DOC) | ||
.defineInternal(STATE_TOPIC_PRUNE_INTERVAL, INT, STATE_TOPIC_PRUNE_INTERVAL_DEFAULT, atLeast(1), LOW, STATE_TOPIC_PRUNE_INTERVAL_DOC); | ||
|
||
private final int stateTopicNumPartitions; | ||
private final short stateTopicReplicationFactor; | ||
|
@@ -93,6 +99,7 @@ public class ShareCoordinatorConfig { | |
private final int loadBufferSize; | ||
private final CompressionType compressionType; | ||
private final int appendLingerMs; | ||
private final int pruneIntervalMs; | ||
|
||
|
||
public ShareCoordinatorConfig(AbstractConfig config) { | ||
|
@@ -108,6 +115,7 @@ public ShareCoordinatorConfig(AbstractConfig config) { | |
.map(CompressionType::forId) | ||
.orElse(null); | ||
appendLingerMs = config.getInt(APPEND_LINGER_MS_CONFIG); | ||
pruneIntervalMs = config.getInt(STATE_TOPIC_PRUNE_INTERVAL); | ||
validate(); | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,135 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.kafka.coordinator.share; | ||
|
||
import org.apache.kafka.server.share.SharePartitionKey; | ||
|
||
import java.util.Comparator; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.Optional; | ||
import java.util.PriorityQueue; | ||
import java.util.Queue; | ||
import java.util.concurrent.atomic.AtomicLong; | ||
|
||
public class ShareCoordinatorOffsetsManager { | ||
private enum State { | ||
ACTIVE, | ||
INACTIVE | ||
} | ||
|
||
static class Entry { | ||
private final SharePartitionKey key; | ||
private final long offset; | ||
private State state; | ||
|
||
private Entry(SharePartitionKey key, long offset) { | ||
this.key = key; | ||
this.offset = offset; | ||
this.state = State.ACTIVE; | ||
} | ||
|
||
public SharePartitionKey key() { | ||
return key; | ||
} | ||
|
||
public long offset() { | ||
return offset; | ||
} | ||
|
||
public State state() { | ||
return state; | ||
} | ||
|
||
public void setState(State state) { | ||
this.state = state; | ||
} | ||
|
||
public static Entry instance(SharePartitionKey key, long offset) { | ||
return new Entry(key, offset); | ||
} | ||
} | ||
|
||
private final Map<SharePartitionKey, Entry> entries = new HashMap<>(); | ||
private final Queue<Entry> curState; | ||
private long minOffset = Long.MAX_VALUE; | ||
private final AtomicLong lastRedundantOffset = new AtomicLong(0); | ||
|
||
public ShareCoordinatorOffsetsManager() { | ||
curState = new PriorityQueue<>(new Comparator<Entry>() { | ||
@Override | ||
public int compare(Entry o1, Entry o2) { | ||
return Long.compare(o1.offset, o2.offset); | ||
} | ||
}); | ||
} | ||
|
||
public Optional<Long> updateState(SharePartitionKey key, long offset) { | ||
minOffset = Math.min(minOffset, offset); | ||
if (entries.containsKey(key)) { | ||
entries.get(key).setState(State.INACTIVE); | ||
} | ||
Entry newEntry = Entry.instance(key, offset); | ||
curState.add(newEntry); | ||
entries.put(key, newEntry); | ||
|
||
purge(); | ||
|
||
Optional<Long> deleteTillOpt = findLastRedundantOffset(); | ||
deleteTillOpt.ifPresent(off -> minOffset = off); | ||
return deleteTillOpt; | ||
} | ||
|
||
private Optional<Long> findLastRedundantOffset() { | ||
if (curState.isEmpty()) { | ||
return Optional.empty(); | ||
} | ||
|
||
Entry candidate = curState.peek(); | ||
if (candidate == null) { | ||
return Optional.empty(); | ||
} | ||
|
||
if (candidate.offset() <= 0 || candidate.offset() == minOffset) { | ||
return Optional.empty(); | ||
} | ||
lastRedundantOffset.set(candidate.offset()); | ||
return Optional.of(candidate.offset()); | ||
} | ||
|
||
public Optional<Long> lastRedundantOffset() { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This method is ok, but is very customized for the usage in the current only caller. If there is another caller, it can unexpectedly break the existing caller. A better api is probably to always expose the lastRedundantOffset and let the caller handle the case when the same value is returned. |
||
return Optional.of(lastRedundantOffset.get()); | ||
} | ||
|
||
// test visibility | ||
void purge() { | ||
while (!curState.isEmpty() && curState.peek().state() == State.INACTIVE) { | ||
curState.poll(); | ||
} | ||
} | ||
|
||
//test visibility | ||
Queue<Entry> curState() { | ||
return curState; | ||
} | ||
|
||
//test visibility | ||
Map<SharePartitionKey, Entry> entries() { | ||
return entries; | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In my opinion, this is the wrong approach because it will allow users of the cluster to delete too. When I suggested it, I thought that we would add a boolean to the method, e.g. allowInternalTopics, and use it from the component doing the deletion.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Understood, will rectify