clolov commented on code in PR #14127:
URL: https://github.com/apache/kafka/pull/14127#discussion_r1288608136


##########
storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.server.log.remote.metadata.storage;
+
+
+import kafka.utils.EmptyTestInfo;
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for 
usages of JavaConverters
+public class TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest {
+    private static final Logger log = 
LoggerFactory.getLogger(TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.class);
+
+    private static final int SEG_SIZE = 1024 * 1024;
+
+    private final Time time = new MockTime(1);
+    private final TopicBasedRemoteLogMetadataManagerHarness 
remoteLogMetadataManagerHarness = new 
TopicBasedRemoteLogMetadataManagerHarness();
+
+    private TopicBasedRemoteLogMetadataManager rlmm() {
+        return remoteLogMetadataManagerHarness.remoteLogMetadataManager();
+    }
+
+    @BeforeEach
+    public void setup() {
+        // Start the cluster only.
+        remoteLogMetadataManagerHarness.setUp(new EmptyTestInfo());
+    }
+
+    @AfterEach
+    public void teardown() throws IOException {
+        remoteLogMetadataManagerHarness.close();
+    }
+
+    @Test
+    public void testMultiplePartitionSubscriptions() throws Exception {
+        // Create topics.
+        String leaderTopic = "leader";
+        HashMap<Object, Seq<Object>> assignedLeaderTopicReplicas = new 
HashMap<>();
+        List<Object> leaderTopicReplicas = new ArrayList<>();
+        // Set broker id 0 as the first entry which is taken as the leader.
+        leaderTopicReplicas.add(0);
+        leaderTopicReplicas.add(1);
+        leaderTopicReplicas.add(2);
+        assignedLeaderTopicReplicas.put(0, 
JavaConverters.asScalaBuffer(leaderTopicReplicas));
+        remoteLogMetadataManagerHarness.createTopicWithAssignment(leaderTopic,
+            JavaConverters.mapAsScalaMap(assignedLeaderTopicReplicas),
+            remoteLogMetadataManagerHarness.listenerName());
+
+        String followerTopic = "follower";
+        HashMap<Object, Seq<Object>> assignedFollowerTopicReplicas = new 
HashMap<>();
+        List<Object> followerTopicReplicas = new ArrayList<>();
+        // Set broker id 1 as the first entry which is taken as the leader.
+        followerTopicReplicas.add(1);
+        followerTopicReplicas.add(2);
+        followerTopicReplicas.add(0);
+        assignedFollowerTopicReplicas.put(0, 
JavaConverters.asScalaBuffer(followerTopicReplicas));
+        remoteLogMetadataManagerHarness.createTopicWithAssignment(
+            followerTopic, 
JavaConverters.mapAsScalaMap(assignedFollowerTopicReplicas),
+            remoteLogMetadataManagerHarness.listenerName());
+
+        String topicWithNoMessages = "no-messages-topic";
+        HashMap<Object, Seq<Object>> assignedTopicReplicas = new HashMap<>();
+        List<Object> noMessagesTopicReplicas = new ArrayList<>();
+        // Set broker id 1 as the first entry which is taken as the leader.
+        noMessagesTopicReplicas.add(1);
+        noMessagesTopicReplicas.add(2);
+        noMessagesTopicReplicas.add(0);
+        assignedTopicReplicas.put(0, 
JavaConverters.asScalaBuffer(noMessagesTopicReplicas));
+        remoteLogMetadataManagerHarness.createTopicWithAssignment(
+            topicWithNoMessages, 
JavaConverters.mapAsScalaMap(assignedTopicReplicas),
+            remoteLogMetadataManagerHarness.listenerName());
+
+        final TopicIdPartition leaderTopicIdPartition = new 
TopicIdPartition(Uuid.randomUuid(), new TopicPartition(leaderTopic, 0));
+        final TopicIdPartition followerTopicIdPartition = new 
TopicIdPartition(Uuid.randomUuid(), new TopicPartition(followerTopic, 0));
+        final TopicIdPartition emptyTopicIdPartition = new 
TopicIdPartition(Uuid.randomUuid(), new TopicPartition(topicWithNoMessages, 0));
+
+        RemoteLogMetadataTopicPartitioner partitioner = new 
RemoteLogMetadataTopicPartitioner(10) {
+            @Override
+            public int metadataPartition(TopicIdPartition topicIdPartition) {
+                // Always return partition 0 except for 
noMessagesTopicIdPartition. So that, any new user
+                // partition(other than noMessagesTopicIdPartition) added to 
RLMM will use the same metadata partition.
+                // That will make the secondary consumer assignment.
+                if (emptyTopicIdPartition.equals(topicIdPartition)) {
+                    return 1;
+                } else {
+                    return 0;
+                }
+            }
+        };
+
+        
remoteLogMetadataManagerHarness.initializeRemoteLogMetadataManager(Collections.emptySet(),
 true, partitioner);
+
+        // Add segments for these partitions but an exception is received as 
they have not yet been subscribed.
+        // These messages would have been published to the respective metadata 
topic partitions but the ConsumerManager
+        // has not yet been subscribing as they are not yet registered.
+        RemoteLogSegmentMetadata leaderSegmentMetadata = new 
RemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, 
Uuid.randomUuid()),
+            0, 100, -1L, 0,
+            time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L));
+        Assertions.assertThrows(ExecutionException.class, () -> 
rlmm().addRemoteLogSegmentMetadata(leaderSegmentMetadata).get(),
+            "org.apache.kafka.common.KafkaException: This consumer is not 
assigned to the target partition 0. Partitions currently assigned: []");

Review Comment:
   I thought that the last argument to an assertThrows is the message you would 
like to return if the assertion fails rather than the message you would like to 
check is contained in the exception? Or am I misunderstanding the purpose of 
this message?



##########
storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTaskTest.java:
##########
@@ -0,0 +1,414 @@
+/*
+ * 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.server.log.remote.metadata.storage;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.MockConsumer;
+import org.apache.kafka.clients.consumer.OffsetResetStrategy;
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.AuthorizationException;
+import org.apache.kafka.common.errors.LeaderNotAvailableException;
+import org.apache.kafka.common.errors.TimeoutException;
+import 
org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import 
org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate;
+import 
org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteMetadata;
+import org.apache.kafka.test.TestCondition;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.kafka.server.log.remote.metadata.storage.ConsumerTask.UserTopicIdPartition;
+import static 
org.apache.kafka.server.log.remote.metadata.storage.ConsumerTask.toRemoteLogPartition;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class ConsumerTaskTest {
+
+    private final int numMetadataTopicPartitions = 5;
+    private final RemoteLogMetadataTopicPartitioner partitioner = new 
RemoteLogMetadataTopicPartitioner(numMetadataTopicPartitions);
+    private final DummyEventHandler handler = new DummyEventHandler();
+    private final Set<TopicPartition> remoteLogPartitions = IntStream.range(0, 
numMetadataTopicPartitions).boxed()
+        .map(ConsumerTask::toRemoteLogPartition).collect(Collectors.toSet());
+    private final Uuid topicId = Uuid.randomUuid();
+    private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+    private ConsumerTask consumerTask;
+    private MockConsumer<byte[], byte[]> consumer;
+    private Thread thread;
+
+    @BeforeEach
+    public void beforeEach() {
+        final Map<TopicPartition, Long> offsets = remoteLogPartitions.stream()
+            .collect(Collectors.toMap(Function.identity(), e -> 0L));
+        consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
+        consumer.updateBeginningOffsets(offsets);
+        ConsumerTask.pollIntervalMs = 10L;
+        consumerTask = new ConsumerTask(handler, partitioner, ignored -> 
consumer);
+        thread = new Thread(consumerTask);
+    }
+
+    @AfterEach
+    public void afterEach() throws InterruptedException {
+        if (thread != null) {
+            consumerTask.close();
+            thread.join(10_000);
+        }
+    }
+
+    /**
+     * Tests that the consumer task shuts down gracefully when there were no 
assignments.
+     */
+    @Test
+    public void testCloseOnNoAssignment() throws InterruptedException {
+        thread.start();
+        Thread.sleep(10);
+    }
+
+    @Test
+    public void testIdempotentClose() {
+        thread.start();
+        consumerTask.close();
+        consumerTask.close();
+    }
+
+    @Test
+    public void testUserTopicIdPartitionEquals() {
+        final TopicIdPartition tpId = new TopicIdPartition(topicId, new 
TopicPartition("sample", 0));
+        final UserTopicIdPartition utp1 = new UserTopicIdPartition(tpId, 
partitioner.metadataPartition(tpId));
+        final UserTopicIdPartition utp2 = new UserTopicIdPartition(tpId, 
partitioner.metadataPartition(tpId));
+        utp1.isInitialized = true;
+        utp1.isAssigned = true;
+
+        assertFalse(utp2.isInitialized);
+        assertFalse(utp2.isAssigned);
+        assertEquals(utp1, utp2);
+    }
+
+    @Test
+    public void testAddAssignmentsForPartitions() throws InterruptedException {
+        final List<TopicIdPartition> idPartitions = getIdPartitions("sample", 
3);
+        final Map<TopicPartition, Long> endOffsets = idPartitions.stream()
+            .map(idp -> 
toRemoteLogPartition(partitioner.metadataPartition(idp)))
+            .collect(Collectors.toMap(Function.identity(), e -> 0L, (a, b) -> 
b));

Review Comment:
   Is the merge function needed? If not, can we get rid of it, because its 
presence leads me to think that we can have multiple values associated with the 
same key and unless I am wrong that isn't the case here?



##########
storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.server.log.remote.metadata.storage;
+
+
+import kafka.utils.EmptyTestInfo;
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for 
usages of JavaConverters
+public class TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest {
+    private static final Logger log = 
LoggerFactory.getLogger(TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.class);
+
+    private static final int SEG_SIZE = 1024 * 1024;
+
+    private final Time time = new MockTime(1);
+    private final TopicBasedRemoteLogMetadataManagerHarness 
remoteLogMetadataManagerHarness = new 
TopicBasedRemoteLogMetadataManagerHarness();
+
+    private TopicBasedRemoteLogMetadataManager rlmm() {
+        return remoteLogMetadataManagerHarness.remoteLogMetadataManager();
+    }
+
+    @BeforeEach
+    public void setup() {
+        // Start the cluster only.
+        remoteLogMetadataManagerHarness.setUp(new EmptyTestInfo());
+    }
+
+    @AfterEach
+    public void teardown() throws IOException {
+        remoteLogMetadataManagerHarness.close();
+    }
+
+    @Test
+    public void testMultiplePartitionSubscriptions() throws Exception {
+        // Create topics.
+        String leaderTopic = "leader";
+        HashMap<Object, Seq<Object>> assignedLeaderTopicReplicas = new 
HashMap<>();
+        List<Object> leaderTopicReplicas = new ArrayList<>();
+        // Set broker id 0 as the first entry which is taken as the leader.
+        leaderTopicReplicas.add(0);
+        leaderTopicReplicas.add(1);
+        leaderTopicReplicas.add(2);
+        assignedLeaderTopicReplicas.put(0, 
JavaConverters.asScalaBuffer(leaderTopicReplicas));
+        remoteLogMetadataManagerHarness.createTopicWithAssignment(leaderTopic,
+            JavaConverters.mapAsScalaMap(assignedLeaderTopicReplicas),
+            remoteLogMetadataManagerHarness.listenerName());
+
+        String followerTopic = "follower";
+        HashMap<Object, Seq<Object>> assignedFollowerTopicReplicas = new 
HashMap<>();
+        List<Object> followerTopicReplicas = new ArrayList<>();
+        // Set broker id 1 as the first entry which is taken as the leader.
+        followerTopicReplicas.add(1);
+        followerTopicReplicas.add(2);
+        followerTopicReplicas.add(0);
+        assignedFollowerTopicReplicas.put(0, 
JavaConverters.asScalaBuffer(followerTopicReplicas));
+        remoteLogMetadataManagerHarness.createTopicWithAssignment(
+            followerTopic, 
JavaConverters.mapAsScalaMap(assignedFollowerTopicReplicas),
+            remoteLogMetadataManagerHarness.listenerName());
+
+        String topicWithNoMessages = "no-messages-topic";
+        HashMap<Object, Seq<Object>> assignedTopicReplicas = new HashMap<>();
+        List<Object> noMessagesTopicReplicas = new ArrayList<>();
+        // Set broker id 1 as the first entry which is taken as the leader.
+        noMessagesTopicReplicas.add(1);
+        noMessagesTopicReplicas.add(2);
+        noMessagesTopicReplicas.add(0);
+        assignedTopicReplicas.put(0, 
JavaConverters.asScalaBuffer(noMessagesTopicReplicas));
+        remoteLogMetadataManagerHarness.createTopicWithAssignment(
+            topicWithNoMessages, 
JavaConverters.mapAsScalaMap(assignedTopicReplicas),
+            remoteLogMetadataManagerHarness.listenerName());
+
+        final TopicIdPartition leaderTopicIdPartition = new 
TopicIdPartition(Uuid.randomUuid(), new TopicPartition(leaderTopic, 0));
+        final TopicIdPartition followerTopicIdPartition = new 
TopicIdPartition(Uuid.randomUuid(), new TopicPartition(followerTopic, 0));
+        final TopicIdPartition emptyTopicIdPartition = new 
TopicIdPartition(Uuid.randomUuid(), new TopicPartition(topicWithNoMessages, 0));
+
+        RemoteLogMetadataTopicPartitioner partitioner = new 
RemoteLogMetadataTopicPartitioner(10) {
+            @Override
+            public int metadataPartition(TopicIdPartition topicIdPartition) {
+                // Always return partition 0 except for 
noMessagesTopicIdPartition. So that, any new user
+                // partition(other than noMessagesTopicIdPartition) added to 
RLMM will use the same metadata partition.
+                // That will make the secondary consumer assignment.
+                if (emptyTopicIdPartition.equals(topicIdPartition)) {
+                    return 1;
+                } else {
+                    return 0;
+                }
+            }
+        };
+
+        
remoteLogMetadataManagerHarness.initializeRemoteLogMetadataManager(Collections.emptySet(),
 true, partitioner);
+
+        // Add segments for these partitions but an exception is received as 
they have not yet been subscribed.
+        // These messages would have been published to the respective metadata 
topic partitions but the ConsumerManager
+        // has not yet been subscribing as they are not yet registered.
+        RemoteLogSegmentMetadata leaderSegmentMetadata = new 
RemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, 
Uuid.randomUuid()),
+            0, 100, -1L, 0,
+            time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L));
+        Assertions.assertThrows(ExecutionException.class, () -> 
rlmm().addRemoteLogSegmentMetadata(leaderSegmentMetadata).get(),
+            "org.apache.kafka.common.KafkaException: This consumer is not 
assigned to the target partition 0. Partitions currently assigned: []");

Review Comment:
   Can you give a bit more explanation here, because I do not understand this? 
Are we expecting this to throw, in which case, why would it accept the metadata 
entry or are we not expecting it to throw?



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