You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ij...@apache.org on 2022/12/19 14:37:43 UTC
[kafka] branch trunk updated: KAFKA-14466: Move ClassloaderAwareRemoteStorageManager to storage module (#13013)
This is an automated email from the ASF dual-hosted git repository.
ijuma pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new e3cb2deff6e KAFKA-14466: Move ClassloaderAwareRemoteStorageManager to storage module (#13013)
e3cb2deff6e is described below
commit e3cb2deff6e983aeb1744d4403af4b3c9aa85e57
Author: Satish Duggana <sa...@apache.org>
AuthorDate: Mon Dec 19 20:07:33 2022 +0530
KAFKA-14466: Move ClassloaderAwareRemoteStorageManager to storage module (#13013)
Reviewers: Ismael Juma <is...@juma.me.uk>
---
.../ClassLoaderAwareRemoteStorageManager.scala | 76 ---------------
.../scala/kafka/log/remote/RemoteLogManager.scala | 2 +-
.../ClassLoaderAwareRemoteStorageManagerTest.scala | 2 +-
.../kafka/log/remote/RemoteLogManagerTest.scala | 10 +-
.../kafka/server/log/internals/StorageAction.java | 28 ++++++
.../ClassLoaderAwareRemoteLogMetadataManager.java | 28 ++----
.../ClassLoaderAwareRemoteStorageManager.java | 103 +++++++++++++++++++++
7 files changed, 145 insertions(+), 104 deletions(-)
diff --git a/core/src/main/scala/kafka/log/remote/ClassLoaderAwareRemoteStorageManager.scala b/core/src/main/scala/kafka/log/remote/ClassLoaderAwareRemoteStorageManager.scala
deleted file mode 100644
index d35c70ed85e..00000000000
--- a/core/src/main/scala/kafka/log/remote/ClassLoaderAwareRemoteStorageManager.scala
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * 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 org.apache.kafka.server.log.remote.storage.{LogSegmentData, RemoteLogSegmentMetadata, RemoteStorageManager}
-
-import java.io.InputStream
-import java.util
-
-/**
- * A wrapper class of RemoteStorageManager that sets the context class loader when calling RSM methods.
- */
-class ClassLoaderAwareRemoteStorageManager(val rsm: RemoteStorageManager,
- val rsmClassLoader: ClassLoader) extends RemoteStorageManager {
-
- def withClassLoader[T](fun: => T): T = {
- val originalClassLoader = Thread.currentThread.getContextClassLoader
- Thread.currentThread.setContextClassLoader(rsmClassLoader)
- try {
- fun
- } finally {
- Thread.currentThread.setContextClassLoader(originalClassLoader)
- }
- }
-
- def delegate(): RemoteStorageManager = {
- rsm
- }
-
- override def close(): Unit = withClassLoader {
- rsm.close()
- }
-
- override def configure(configs: util.Map[String, _]): Unit = withClassLoader {
- rsm.configure(configs)
- }
-
- override def copyLogSegmentData(remoteLogSegmentMetadata: RemoteLogSegmentMetadata,
- logSegmentData: LogSegmentData): Unit = withClassLoader {
- rsm.copyLogSegmentData(remoteLogSegmentMetadata, logSegmentData)
- }
-
- override def fetchLogSegment(remoteLogSegmentMetadata: RemoteLogSegmentMetadata,
- startPosition: Int): InputStream = withClassLoader {
- rsm.fetchLogSegment(remoteLogSegmentMetadata, startPosition)
- }
-
- override def fetchLogSegment(remoteLogSegmentMetadata: RemoteLogSegmentMetadata,
- startPosition: Int,
- endPosition: Int): InputStream = withClassLoader {
- rsm.fetchLogSegment(remoteLogSegmentMetadata, startPosition, endPosition)
- }
-
- override def fetchIndex(remoteLogSegmentMetadata: RemoteLogSegmentMetadata,
- indexType: RemoteStorageManager.IndexType): InputStream = withClassLoader {
- rsm.fetchIndex(remoteLogSegmentMetadata, indexType)
- }
-
- override def deleteLogSegmentData(remoteLogSegmentMetadata: RemoteLogSegmentMetadata): Unit = withClassLoader {
- rsm.deleteLogSegmentData(remoteLogSegmentMetadata)
- }
-}
\ No newline at end of file
diff --git a/core/src/main/scala/kafka/log/remote/RemoteLogManager.scala b/core/src/main/scala/kafka/log/remote/RemoteLogManager.scala
index 6558094842a..8324028c5ed 100644
--- a/core/src/main/scala/kafka/log/remote/RemoteLogManager.scala
+++ b/core/src/main/scala/kafka/log/remote/RemoteLogManager.scala
@@ -26,7 +26,7 @@ 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
-import org.apache.kafka.server.log.remote.storage.{RemoteLogManagerConfig, RemoteLogMetadataManager, RemoteLogSegmentMetadata, RemoteStorageManager}
+import org.apache.kafka.server.log.remote.storage.{ClassLoaderAwareRemoteStorageManager, RemoteLogManagerConfig, RemoteLogMetadataManager, RemoteLogSegmentMetadata, RemoteStorageManager}
import java.io.{Closeable, InputStream}
import java.security.{AccessController, PrivilegedAction}
diff --git a/core/src/test/scala/kafka/log/remote/ClassLoaderAwareRemoteStorageManagerTest.scala b/core/src/test/scala/kafka/log/remote/ClassLoaderAwareRemoteStorageManagerTest.scala
index 3cb1516a38b..54d0aee9447 100644
--- a/core/src/test/scala/kafka/log/remote/ClassLoaderAwareRemoteStorageManagerTest.scala
+++ b/core/src/test/scala/kafka/log/remote/ClassLoaderAwareRemoteStorageManagerTest.scala
@@ -16,7 +16,7 @@
*/
package kafka.log.remote
-import org.apache.kafka.server.log.remote.storage.RemoteStorageManager
+import org.apache.kafka.server.log.remote.storage.{ClassLoaderAwareRemoteStorageManager, RemoteStorageManager}
import org.junit.jupiter.api.Test
import org.mockito.Mockito.mock
import org.mockito.Mockito.when
diff --git a/core/src/test/scala/unit/kafka/log/remote/RemoteLogManagerTest.scala b/core/src/test/scala/unit/kafka/log/remote/RemoteLogManagerTest.scala
index 92128c1e7d9..6fb62c14843 100644
--- a/core/src/test/scala/unit/kafka/log/remote/RemoteLogManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/remote/RemoteLogManagerTest.scala
@@ -22,23 +22,23 @@ import kafka.server.KafkaConfig
import kafka.server.checkpoints.LeaderEpochCheckpoint
import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache}
import kafka.utils.MockTime
-import org.apache.kafka.common.{KafkaException, TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.common.config.AbstractConfig
import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
+import org.apache.kafka.common.{KafkaException, TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType
-import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteLogMetadataManager, RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteStorageManager}
+import org.apache.kafka.server.log.remote.storage._
import org.apache.kafka.test.TestUtils
+import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{BeforeEach, Test}
+import org.mockito.ArgumentMatchers.{any, anyInt, anyLong}
import org.mockito.Mockito._
-import org.junit.jupiter.api.Assertions._
import org.mockito.{ArgumentCaptor, ArgumentMatchers}
-import org.mockito.ArgumentMatchers.{any, anyInt, anyLong}
import java.io.{ByteArrayInputStream, File, FileInputStream}
import java.nio.file.Files
-import java.util.{Optional, Properties}
import java.util
+import java.util.{Optional, Properties}
import scala.collection.Seq
import scala.jdk.CollectionConverters._
diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/StorageAction.java b/storage/src/main/java/org/apache/kafka/server/log/internals/StorageAction.java
new file mode 100644
index 00000000000..434acd5eb88
--- /dev/null
+++ b/storage/src/main/java/org/apache/kafka/server/log/internals/StorageAction.java
@@ -0,0 +1,28 @@
+/*
+ * 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.internals;
+
+/**
+ * This interface is used to execute any storage related operations.
+ *
+ * @param <T> return type for execute operation
+ * @param <E> Exception type to be thrown
+ */
+@FunctionalInterface
+public interface StorageAction<T, E extends Exception> {
+ T execute() throws E;
+}
diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ClassLoaderAwareRemoteLogMetadataManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ClassLoaderAwareRemoteLogMetadataManager.java
index a40e34031d3..47c0942e5a2 100644
--- a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ClassLoaderAwareRemoteLogMetadataManager.java
+++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ClassLoaderAwareRemoteLogMetadataManager.java
@@ -17,6 +17,7 @@
package org.apache.kafka.server.log.remote.metadata.storage;
import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.server.log.internals.StorageAction;
import org.apache.kafka.server.log.remote.storage.RemoteLogMetadataManager;
import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate;
@@ -69,8 +70,7 @@ public class ClassLoaderAwareRemoteLogMetadataManager implements RemoteLogMetada
@Override
public CompletableFuture<Void> putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata) throws RemoteStorageException {
- return withClassLoader(() -> delegate.putRemotePartitionDeleteMetadata(remotePartitionDeleteMetadata)
- );
+ return withClassLoader(() -> delegate.putRemotePartitionDeleteMetadata(remotePartitionDeleteMetadata));
}
@Override
@@ -87,7 +87,7 @@ public class ClassLoaderAwareRemoteLogMetadataManager implements RemoteLogMetada
@Override
public void onPartitionLeadershipChanges(Set<TopicIdPartition> leaderPartitions,
Set<TopicIdPartition> followerPartitions) {
- withTryCatchClassLoader(() -> {
+ withClassLoader(() -> {
delegate.onPartitionLeadershipChanges(leaderPartitions, followerPartitions);
return null;
});
@@ -95,7 +95,7 @@ public class ClassLoaderAwareRemoteLogMetadataManager implements RemoteLogMetada
@Override
public void onStopPartitions(Set<TopicIdPartition> partitions) {
- withTryCatchClassLoader(() -> {
+ withClassLoader(() -> {
delegate.onStopPartitions(partitions);
return null;
});
@@ -103,7 +103,7 @@ public class ClassLoaderAwareRemoteLogMetadataManager implements RemoteLogMetada
@Override
public void configure(Map<String, ?> configs) {
- withTryCatchClassLoader(() -> {
+ withClassLoader(() -> {
delegate.configure(configs);
return null;
});
@@ -120,28 +120,14 @@ public class ClassLoaderAwareRemoteLogMetadataManager implements RemoteLogMetada
}
}
- @SuppressWarnings("UnusedReturnValue")
- private <T> T withTryCatchClassLoader(Worker<T> worker) {
- try {
- return withClassLoader(worker);
- } catch (final RemoteStorageException ex) {
- // ignore, this exception is not thrown by the method.
- }
- return null;
- }
-
- private <T> T withClassLoader(Worker<T> worker) throws RemoteStorageException {
+ private <T, E extends Exception> T withClassLoader(StorageAction<T, E> action) throws E {
ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
Thread.currentThread().setContextClassLoader(loader);
try {
- return worker.doWork();
+ return action.execute();
} finally {
Thread.currentThread().setContextClassLoader(originalClassLoader);
}
}
- @FunctionalInterface
- public interface Worker<T> {
- T doWork() throws RemoteStorageException;
- }
}
diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/ClassLoaderAwareRemoteStorageManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/ClassLoaderAwareRemoteStorageManager.java
new file mode 100644
index 00000000000..976a2a0d83f
--- /dev/null
+++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/ClassLoaderAwareRemoteStorageManager.java
@@ -0,0 +1,103 @@
+/*
+ * 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.storage;
+
+import org.apache.kafka.server.log.internals.StorageAction;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+
+/**
+ * A wrapper class of {@link RemoteStorageManager} that sets the context class loader when calling the respective
+ * methods.
+ */
+public class ClassLoaderAwareRemoteStorageManager implements RemoteStorageManager {
+
+ private final RemoteStorageManager delegate;
+ private final ClassLoader rsmClassLoader;
+
+ public ClassLoaderAwareRemoteStorageManager(RemoteStorageManager rsm, ClassLoader rsmClassLoader) {
+ this.delegate = rsm;
+ this.rsmClassLoader = rsmClassLoader;
+ }
+
+ public RemoteStorageManager delegate() {
+ return delegate;
+ }
+
+ @Override
+ public void configure(Map<String, ?> configs) {
+ ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
+ Thread.currentThread().setContextClassLoader(rsmClassLoader);
+ try {
+ delegate.configure(configs);
+ } finally {
+ Thread.currentThread().setContextClassLoader(originalClassLoader);
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ withClassLoader(() -> {
+ delegate.close();
+ return null;
+ });
+ }
+
+ private <T, E extends Exception> T withClassLoader(StorageAction<T, E> action) throws E {
+ ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
+ Thread.currentThread().setContextClassLoader(rsmClassLoader);
+ try {
+ return action.execute();
+ } finally {
+ Thread.currentThread().setContextClassLoader(originalClassLoader);
+ }
+ }
+
+ public void copyLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+ LogSegmentData logSegmentData) throws RemoteStorageException {
+ withClassLoader(() -> {
+ delegate.copyLogSegmentData(remoteLogSegmentMetadata, logSegmentData);
+ return null;
+ });
+ }
+
+ @Override
+ public InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata, int startPosition) throws RemoteStorageException {
+ return withClassLoader(() -> delegate.fetchLogSegment(remoteLogSegmentMetadata, startPosition));
+ }
+
+ @Override
+ public InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata, int startPosition, int endPosition) throws RemoteStorageException {
+ return withClassLoader(() -> delegate.fetchLogSegment(remoteLogSegmentMetadata, startPosition, endPosition));
+ }
+
+ @Override
+ public InputStream fetchIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata, IndexType indexType) throws RemoteStorageException {
+ return withClassLoader(() -> delegate.fetchIndex(remoteLogSegmentMetadata, indexType));
+ }
+
+ @Override
+ public void deleteLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws RemoteStorageException {
+ withClassLoader(() -> {
+ delegate.deleteLogSegmentData(remoteLogSegmentMetadata);
+ return null;
+ });
+ }
+
+}
\ No newline at end of file