satishd commented on a change in pull request #11390:
URL: https://github.com/apache/kafka/pull/11390#discussion_r767428241



##########
File path: core/src/main/scala/kafka/log/remote/RemoteLogManager.scala
##########
@@ -0,0 +1,287 @@
+/**
+ * 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 kafka.log.remote
+
+import kafka.cluster.Partition
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.KafkaConfig
+import kafka.utils.Logging
+import org.apache.kafka.common._
+import org.apache.kafka.common.internals.Topic
+import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
+import org.apache.kafka.common.record.{RecordBatch, RemoteLogInputStream}
+import org.apache.kafka.common.utils.{ChildFirstClassLoader, Utils}
+import 
org.apache.kafka.server.log.remote.metadata.storage.{ClassLoaderAwareRemoteLogMetadataManager,
 TopicBasedRemoteLogMetadataManagerConfig}
+import org.apache.kafka.server.log.remote.storage.{RemoteLogManagerConfig, 
RemoteLogMetadataManager, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{Closeable, InputStream}
+import java.security.{AccessController, PrivilegedAction}
+import java.util
+import java.util.Optional
+import scala.collection.{Set, mutable}
+import scala.jdk.CollectionConverters._
+
+/**
+ * This class is responsible for
+ *  - initializing `RemoteStorageManager` and `RemoteLogMetadataManager` 
instances.
+ *  - receives any leader and follower replica events and partition stop 
events and act on them
+ *  - also provides APIs to fetch indexes, metadata about remote log segments.
+ *
+ * @param rlmConfig
+ * @param brokerId
+ * @param logDir
+ */
+class RemoteLogManager(rlmConfig: RemoteLogManagerConfig,
+                       brokerId: Int,
+                       logDir: String) extends Logging with Closeable with 
KafkaMetricsGroup {
+
+  // topic ids received on leadership changes
+  private val topicIds: mutable.Map[String, Uuid] = mutable.Map.empty
+
+  private val remoteLogStorageManager: RemoteStorageManager = 
createRemoteStorageManager()
+  private val remoteLogMetadataManager: RemoteLogMetadataManager = 
createRemoteLogMetadataManager()
+
+  private val indexCache = new RemoteIndexCache(remoteStorageManager = 
remoteLogStorageManager, logDir = logDir)
+
+  private var closed = false
+
+  private[remote] def createRemoteStorageManager(): RemoteStorageManager = {
+    AccessController.doPrivileged(new 
PrivilegedAction[ClassLoaderAwareRemoteStorageManager] {
+      private val classPath = rlmConfig.remoteStorageManagerClassPath()
+
+      override def run(): ClassLoaderAwareRemoteStorageManager = {
+        val classLoader =
+          if (classPath != null && classPath.trim.nonEmpty) {
+            new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+          } else {
+            this.getClass.getClassLoader
+          }
+        val delegate = 
classLoader.loadClass(rlmConfig.remoteStorageManagerClassName())
+          
.getDeclaredConstructor().newInstance().asInstanceOf[RemoteStorageManager]
+        new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)
+      }
+    })
+  }
+
+  private def configureRSM(): Unit = {
+    val rsmProps = new util.HashMap[String, Any]()
+    rlmConfig.remoteStorageManagerProps().asScala.foreach { case (k, v) => 
rsmProps.put(k, v) }
+    rsmProps.put(KafkaConfig.BrokerIdProp, brokerId)
+    remoteLogStorageManager.configure(rsmProps)
+  }
+
+  private[remote] def createRemoteLogMetadataManager(): 
RemoteLogMetadataManager = {
+    AccessController.doPrivileged(new 
PrivilegedAction[RemoteLogMetadataManager] {
+      private val classPath = rlmConfig.remoteLogMetadataManagerClassPath
+
+      override def run(): RemoteLogMetadataManager = {
+        var classLoader = this.getClass.getClassLoader
+        if (classPath != null && classPath.trim.nonEmpty) {
+          classLoader = new ChildFirstClassLoader(classPath, classLoader)
+          val delegate = 
classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+          new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)
+        } else {
+          classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+        }
+      }
+    })
+  }
+
+  private def configureRLMM(): Unit = {
+    val rlmmProps = new util.HashMap[String, Any]()
+    rlmConfig.remoteLogMetadataManagerProps().asScala.foreach { case (k, v) => 
rlmmProps.put(k, v) }
+    rlmmProps.put(KafkaConfig.BrokerIdProp, brokerId)
+    rlmmProps.put(KafkaConfig.LogDirProp, logDir)
+    remoteLogMetadataManager.configure(rlmmProps)
+  }
+
+  def startup(): Unit = {
+    // Initialize and configure RSM and RLMM. This will start RSM, RLMM 
resources which may need to start resources
+    // in connecting to the brokers or remote storages.
+    configureRSM()
+    configureRLMM()
+  }
+
+  def storageManager(): RemoteStorageManager = {
+    remoteLogStorageManager
+  }
+
+  /**
+   * Callback to receive any leadership changes for the topic partitions 
assigned to this broker. If there are no
+   * existing tasks for a given topic partition then it will assign new leader 
or follower task else it will convert the
+   * task to respective target state(leader or follower).
+   *
+   * @param partitionsBecomeLeader   partitions that have become leaders on 
this broker.
+   * @param partitionsBecomeFollower partitions that have become followers on 
this broker.
+   */
+  def onLeadershipChange(partitionsBecomeLeader: Set[Partition],
+                         partitionsBecomeFollower: Set[Partition],
+                         topicIds: util.Map[String, Uuid]): Unit = {
+    debug(s"Received leadership changes for leaders: $partitionsBecomeLeader 
and followers: $partitionsBecomeFollower")
+    topicIds.forEach((topic, uuid) => this.topicIds.put(topic, uuid))
+
+    // Partitions logs are available when this callback is invoked.
+    // Compact topics and internal topics are filtered here as they are not 
supported with tiered storage.
+    def filterPartitions(partitions: Set[Partition]): Set[Partition] = {
+      partitions.filterNot(partition => Topic.isInternal(partition.topic) ||
+        partition.log.exists(log => log.config.compact || 
!log.config.remoteLogConfig.remoteStorageEnable) ||
+        
partition.topicPartition.topic().equals(TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME)
+      )
+    }
+
+    val followerTopicPartitions = 
filterPartitions(partitionsBecomeFollower).map(partition =>
+      new TopicIdPartition(topicIds.get(partition.topic), 
partition.topicPartition))
+
+    val filteredLeaderPartitions = filterPartitions(partitionsBecomeLeader)
+    val leaderTopicPartitions = filteredLeaderPartitions.map(partition =>
+      new TopicIdPartition(topicIds.get(partition.topic), 
partition.topicPartition))
+
+    debug(s"Effective topic partitions after filtering compact and internal 
topics, leaders: $leaderTopicPartitions " +
+      s"and followers: $followerTopicPartitions")
+
+    if (leaderTopicPartitions.nonEmpty || followerTopicPartitions.nonEmpty) {
+      
remoteLogMetadataManager.onPartitionLeadershipChanges(leaderTopicPartitions.asJava,
 followerTopicPartitions.asJava)
+    }
+
+  }
+
+
+  /**
+   * Stops partitions for copying segments, building indexes and deletes the 
partition in remote storage if delete flag
+   * is set as true.
+   *
+   * @param topicPartition topic partition to be stopped.
+   * @param delete         flag to indicate whether the given topic partitions 
to be deleted or not.
+   */
+  def stopPartitions(topicPartition: TopicPartition, delete: Boolean): Unit = {
+    // Unassign topic partitions from RLM leader/follower
+    val topicIdPartition =
+      topicIds.remove(topicPartition.topic()) match {

Review comment:
       Good point. Changed it to use TopicPartition instead of topic as we need 
to maintain a different data structure for all the partitions and need to 
synchronize between them. Having ConcurrentMap of TopicPartion with topicId is 
simpler to manage here.
   




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