|
|
@ -18,7 +18,7 @@ package kafka.admin |
|
|
|
|
|
|
|
|
|
|
|
import kafka.api.IntegrationTestHarness |
|
|
|
import kafka.api.IntegrationTestHarness |
|
|
|
import kafka.server.KafkaConfig |
|
|
|
import kafka.server.KafkaConfig |
|
|
|
import kafka.utils.{Logging, TestInfoUtils, TestUtils} |
|
|
|
import kafka.utils.{TestInfoUtils, TestUtils} |
|
|
|
import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry} |
|
|
|
import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry} |
|
|
|
import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} |
|
|
|
import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} |
|
|
|
import org.apache.kafka.common.config.{ConfigResource, TopicConfig} |
|
|
|
import org.apache.kafka.common.config.{ConfigResource, TopicConfig} |
|
|
@ -283,6 +283,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { |
|
|
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) |
|
|
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) |
|
|
|
@ValueSource(strings = Array("zk", "kraft")) |
|
|
|
@ValueSource(strings = Array("zk", "kraft")) |
|
|
|
def testTopicDeletion(quorum: String): Unit = { |
|
|
|
def testTopicDeletion(quorum: String): Unit = { |
|
|
|
|
|
|
|
MyRemoteStorageManager.deleteSegmentEventCounter.set(0) |
|
|
|
val numPartitions = 2 |
|
|
|
val numPartitions = 2 |
|
|
|
val topicConfig = new Properties() |
|
|
|
val topicConfig = new Properties() |
|
|
|
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") |
|
|
|
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") |
|
|
@ -293,12 +294,9 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { |
|
|
|
TestUtils.deleteTopicWithAdmin(createAdminClient(), testTopicName, brokers) |
|
|
|
TestUtils.deleteTopicWithAdmin(createAdminClient(), testTopicName, brokers) |
|
|
|
assertThrowsException(classOf[UnknownTopicOrPartitionException], |
|
|
|
assertThrowsException(classOf[UnknownTopicOrPartitionException], |
|
|
|
() => TestUtils.describeTopic(createAdminClient(), testTopicName), "Topic should be deleted") |
|
|
|
() => TestUtils.describeTopic(createAdminClient(), testTopicName), "Topic should be deleted") |
|
|
|
|
|
|
|
TestUtils.waitUntilTrue(() => |
|
|
|
// FIXME: It seems the storage manager is being instantiated in different class loader so couldn't verify the value |
|
|
|
numPartitions * MyRemoteLogMetadataManager.segmentCountPerPartition == MyRemoteStorageManager.deleteSegmentEventCounter.get(), |
|
|
|
// but ensured it by adding a log statement in the storage manager (manually). |
|
|
|
"Remote log segments should be deleted only once by the leader") |
|
|
|
// assertEquals(numPartitions * MyRemoteLogMetadataManager.segmentCount, |
|
|
|
|
|
|
|
// MyRemoteStorageManager.deleteSegmentEventCounter.get(), |
|
|
|
|
|
|
|
// "Remote log segments should be deleted only once by the leader") |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
private def assertThrowsException(exceptionType: Class[_ <: Throwable], |
|
|
|
private def assertThrowsException(exceptionType: Class[_ <: Throwable], |
|
|
@ -365,12 +363,11 @@ object MyRemoteStorageManager { |
|
|
|
val deleteSegmentEventCounter = new AtomicInteger(0) |
|
|
|
val deleteSegmentEventCounter = new AtomicInteger(0) |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
class MyRemoteStorageManager extends NoOpRemoteStorageManager with Logging { |
|
|
|
class MyRemoteStorageManager extends NoOpRemoteStorageManager { |
|
|
|
import MyRemoteStorageManager._ |
|
|
|
import MyRemoteStorageManager._ |
|
|
|
|
|
|
|
|
|
|
|
override def deleteLogSegmentData(remoteLogSegmentMetadata: RemoteLogSegmentMetadata): Unit = { |
|
|
|
override def deleteLogSegmentData(remoteLogSegmentMetadata: RemoteLogSegmentMetadata): Unit = { |
|
|
|
deleteSegmentEventCounter.incrementAndGet() |
|
|
|
deleteSegmentEventCounter.incrementAndGet() |
|
|
|
info(s"Deleted the remote log segment: $remoteLogSegmentMetadata, counter: ${deleteSegmentEventCounter.get()}") |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
@ -381,7 +378,7 @@ class MyRemoteLogMetadataManager extends NoOpRemoteLogMetadataManager { |
|
|
|
|
|
|
|
|
|
|
|
override def listRemoteLogSegments(topicIdPartition: TopicIdPartition): util.Iterator[RemoteLogSegmentMetadata] = { |
|
|
|
override def listRemoteLogSegments(topicIdPartition: TopicIdPartition): util.Iterator[RemoteLogSegmentMetadata] = { |
|
|
|
val segmentMetadataList = new util.ArrayList[RemoteLogSegmentMetadata]() |
|
|
|
val segmentMetadataList = new util.ArrayList[RemoteLogSegmentMetadata]() |
|
|
|
for (idx <- 0 until segmentCount) { |
|
|
|
for (idx <- 0 until segmentCountPerPartition) { |
|
|
|
val timestamp = time.milliseconds() |
|
|
|
val timestamp = time.milliseconds() |
|
|
|
val startOffset = idx * recordsPerSegment |
|
|
|
val startOffset = idx * recordsPerSegment |
|
|
|
val endOffset = startOffset + recordsPerSegment - 1 |
|
|
|
val endOffset = startOffset + recordsPerSegment - 1 |
|
|
@ -395,7 +392,7 @@ class MyRemoteLogMetadataManager extends NoOpRemoteLogMetadataManager { |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
object MyRemoteLogMetadataManager { |
|
|
|
object MyRemoteLogMetadataManager { |
|
|
|
val segmentCount = 10 |
|
|
|
val segmentCountPerPartition = 10 |
|
|
|
val recordsPerSegment = 100 |
|
|
|
val recordsPerSegment = 100 |
|
|
|
val segmentSize = 1024 |
|
|
|
val segmentSize = 1024 |
|
|
|
} |
|
|
|
} |
|
|
|