Skip to content
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-14466: Move ClassloaderAwareRemoteStorageManager to storage module #13013

Merged
merged 3 commits into from
Dec 19, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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._

Expand Down
Original file line number Diff line number Diff line change
@@ -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;
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -69,8 +70,7 @@ public Optional<Long> highestOffsetForEpoch(TopicIdPartition topicIdPartition,

@Override
public CompletableFuture<Void> putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata) throws RemoteStorageException {
return withClassLoader(() -> delegate.putRemotePartitionDeleteMetadata(remotePartitionDeleteMetadata)
);
return withClassLoader(() -> delegate.putRemotePartitionDeleteMetadata(remotePartitionDeleteMetadata));
}

@Override
Expand All @@ -87,23 +87,23 @@ public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition
@Override
public void onPartitionLeadershipChanges(Set<TopicIdPartition> leaderPartitions,
Set<TopicIdPartition> followerPartitions) {
withTryCatchClassLoader(() -> {
withClassLoader(() -> {
delegate.onPartitionLeadershipChanges(leaderPartitions, followerPartitions);
return null;
});
}

@Override
public void onStopPartitions(Set<TopicIdPartition> partitions) {
withTryCatchClassLoader(() -> {
withClassLoader(() -> {
delegate.onStopPartitions(partitions);
return null;
});
}

@Override
public void configure(Map<String, ?> configs) {
withTryCatchClassLoader(() -> {
withClassLoader(() -> {
delegate.configure(configs);
return null;
});
Expand All @@ -120,28 +120,14 @@ public void close() throws IOException {
}
}

@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;
}
}
Original file line number Diff line number Diff line change
@@ -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;
});
}

}