junrao commented on code in PR #18014:
URL: https://github.com/apache/kafka/pull/18014#discussion_r1879044017


##########
share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java:
##########
@@ -240,9 +251,96 @@ public void startup(
 
         log.info("Starting up.");
         numPartitions = shareGroupTopicPartitionCount.getAsInt();
+        Map<TopicPartition, Long> offsets = new ConcurrentHashMap<>();

Review Comment:
   1. offsets => lastPrunedOffsets? 
   2. Would it be better to make that an instance val so that we don't have to 
pass it around?
   3. Should we remove entries when `onResignation` is called?



##########
share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorOffsetsManager.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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 org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.apache.kafka.timeline.TimelineLong;
+
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * Util class to track the offsets written into the internal topic
+ * per share partition key.
+ * It calculates the minimum offset globally up to which the records
+ * in the internal partition are redundant i.e. they have been overridden
+ * by newer records.
+ */
+public class ShareCoordinatorOffsetsManager {
+
+    // Map to store share partition key => current partition offset
+    // being written.
+    private final TimelineHashMap<SharePartitionKey, Long> offsets;
+
+    // Minimum offset representing the smallest necessary offset 
(non-redundant)
+    // across the internal partition.
+    // We are using timeline object here because the offsets which are passed 
into
+    // updateState might not be committed yet. In case of retry, these offsets 
would
+    // be invalidated via the snapshot registry. Hence, using timeline object
+    // the values would automatically revert in accordance with the last 
committed offset.
+    private final TimelineLong minOffset;
+
+    public ShareCoordinatorOffsetsManager(SnapshotRegistry snapshotRegistry) {
+        Objects.requireNonNull(snapshotRegistry);
+        offsets = new TimelineHashMap<>(snapshotRegistry, 0);
+        minOffset = new TimelineLong(snapshotRegistry);

Review Comment:
   minOffset => lastRedundantOffset ?



##########
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##########
@@ -6660,6 +6661,61 @@ class ReplicaManagerTest {
     }
   }
 
+  @Test
+  def testDeleteRecordsInternalTopicDeleteDisallowed(): Unit = {

Review Comment:
   There are lots of existing usage of `rm.becomeLeaderOrFollower`. It would be 
useful to clean them up in a followup jira.



##########
share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorOffsetsManager.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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 org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.apache.kafka.timeline.TimelineLong;
+
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * Util class to track the offsets written into the internal topic
+ * per share partition key.
+ * It calculates the minimum offset globally up to which the records
+ * in the internal partition are redundant i.e. they have been overridden
+ * by newer records.
+ */
+public class ShareCoordinatorOffsetsManager {
+
+    // Map to store share partition key => current partition offset
+    // being written.
+    private final TimelineHashMap<SharePartitionKey, Long> offsets;
+
+    // Minimum offset representing the smallest necessary offset 
(non-redundant)
+    // across the internal partition.
+    // We are using timeline object here because the offsets which are passed 
into
+    // updateState might not be committed yet. In case of retry, these offsets 
would
+    // be invalidated via the snapshot registry. Hence, using timeline object
+    // the values would automatically revert in accordance with the last 
committed offset.
+    private final TimelineLong minOffset;
+
+    public ShareCoordinatorOffsetsManager(SnapshotRegistry snapshotRegistry) {
+        Objects.requireNonNull(snapshotRegistry);
+        offsets = new TimelineHashMap<>(snapshotRegistry, 0);
+        minOffset = new TimelineLong(snapshotRegistry);
+        minOffset.set(Long.MAX_VALUE);  // For easy min update.
+    }
+
+    /**
+     * Method updates internal state with the supplied offset for the provided
+     * share partition key. It then calculates the minimum offset, if possible,
+     * below which all offsets are redundant.
+     *
+     * @param key    - represents {@link SharePartitionKey} whose offset needs 
updating
+     * @param offset - represents the latest partition offset for provided key
+     */
+    public void updateState(SharePartitionKey key, long offset) {
+        minOffset.set(Math.min(minOffset.get(), offset));
+        offsets.put(key, offset);
+
+        Optional<Long> deleteTillOffset = findRedundantOffset();

Review Comment:
   deleteTillOffset => redundantOffset ?



##########
share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java:
##########
@@ -240,9 +251,96 @@ public void startup(
 
         log.info("Starting up.");
         numPartitions = shareGroupTopicPartitionCount.getAsInt();
+        Map<TopicPartition, Long> offsets = new ConcurrentHashMap<>();
+        setupRecordPruning(offsets);
         log.info("Startup complete.");
     }
 
+    private void setupRecordPruning(Map<TopicPartition, Long> offsets) {
+        log.info("Scheduling share state topic prune job.");
+        timer.add(new TimerTask(config.shareCoordinatorTopicPruneIntervalMs()) 
{
+            @Override
+            public void run() {
+                List<CompletableFuture<Void>> futures = new ArrayList<>();
+                runtime.activeTopicPartitions().forEach(tp -> 
futures.add(performRecordPruning(tp, offsets)));
+
+                CompletableFuture.allOf(futures.toArray(new 
CompletableFuture[]{}))
+                    .whenComplete((res, exp) -> {
+                        if (exp != null) {
+                            log.error("Received error in share state topic 
prune.", exp);
+                        }
+                        // Perpetual recursion, failure or not.
+                        setupRecordPruning(offsets);
+                    });
+            }
+        });
+    }
+
+    private CompletableFuture<Void> performRecordPruning(TopicPartition tp, 
Map<TopicPartition, Long> offsets) {
+        // This future will always be completed normally, exception or not.
+        CompletableFuture<Void> fut = new CompletableFuture<>();
+        runtime.scheduleWriteOperation(
+            "write-state-record-prune",
+            tp,
+            Duration.ofMillis(config.shareCoordinatorWriteTimeoutMs()),
+            ShareCoordinatorShard::lastRedundantOffset
+        ).whenComplete((result, exception) -> {
+            if (exception != null) {
+                log.debug("Last redundant offset for tp {} lookup threw an 
error.", tp, exception);
+                Errors error = Errors.forException(exception);
+                // These errors might result from partition metadata not loaded
+                // or shard re-election. Will cause unnecessary noise, hence 
not logging
+                if (!(error.equals(Errors.COORDINATOR_LOAD_IN_PROGRESS) || 
error.equals(Errors.NOT_COORDINATOR))) {
+                    log.error("Last redundant offset lookup for tp {} threw an 
error.", tp, exception);
+                    // Should not reschedule -> unknown exception.
+                    fut.completeExceptionally(exception);
+                    return;
+                }
+                // Should reschedule -> could be transient.
+                fut.complete(null);
+                return;
+            }
+            if (result.isPresent()) {
+                Long off = result.get();
+                // Guard and optimization.
+                if (off == Long.MAX_VALUE || off <= 0) {

Review Comment:
   This test seems redundant since 
`ShareCoordinatorOffsetsManager.lastRedundantOffset` does that already.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to