KAFKA-17468 Move kafka/log/remote/quota classes to storage module (#17074)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Mickael Maison 2024-09-03 20:18:47 +02:00 committed by GitHub
parent 88b9ff30ad
commit 839431e591
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 18 additions and 15 deletions

View File

@ -80,6 +80,7 @@
<subpackage name="log.remote">
<allow pkg="org.apache.kafka.server.common" />
<allow pkg="org.apache.kafka.server.log.remote" />
<allow pkg="org.apache.kafka.server.log.remote.quota" />
<allow pkg="org.apache.kafka.server.metrics" />
<allow pkg="org.apache.kafka.storage.internals" />
<allow pkg="org.apache.kafka.storage.log.metrics" />

View File

@ -63,6 +63,9 @@
<allow pkg="org.apache.kafka.storage"/>
<subpackage name="remote">
<allow pkg="scala.collection" />
<subpackage name="quota">
<allow pkg="org.apache.kafka.server.quota" />
</subpackage>
<subpackage name="storage">
<allow pkg="com.yammer.metrics.core" />
<allow pkg="org.apache.kafka.server.metrics" />

View File

@ -19,9 +19,6 @@ package kafka.log.remote;
import kafka.cluster.EndPoint;
import kafka.cluster.Partition;
import kafka.log.UnifiedLog;
import kafka.log.remote.quota.RLMQuotaManager;
import kafka.log.remote.quota.RLMQuotaManagerConfig;
import kafka.log.remote.quota.RLMQuotaMetrics;
import kafka.server.StopPartition;
import org.apache.kafka.common.KafkaException;
@ -51,6 +48,9 @@ import org.apache.kafka.server.common.CheckpointFile;
import org.apache.kafka.server.common.OffsetAndEpoch;
import org.apache.kafka.server.config.ServerConfigs;
import org.apache.kafka.server.log.remote.metadata.storage.ClassLoaderAwareRemoteLogMetadataManager;
import org.apache.kafka.server.log.remote.quota.RLMQuotaManager;
import org.apache.kafka.server.log.remote.quota.RLMQuotaManagerConfig;
import org.apache.kafka.server.log.remote.quota.RLMQuotaMetrics;
import org.apache.kafka.server.log.remote.storage.ClassLoaderAwareRemoteStorageManager;
import org.apache.kafka.server.log.remote.storage.LogSegmentData;
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig;
@ -138,9 +138,9 @@ import java.util.stream.Stream;
import scala.Option;
import scala.collection.JavaConverters;
import static kafka.log.remote.quota.RLMQuotaManagerConfig.INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS;
import static org.apache.kafka.server.config.ServerLogConfigs.LOG_DIR_CONFIG;
import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX;
import static org.apache.kafka.server.log.remote.quota.RLMQuotaManagerConfig.INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS;
import static org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics.REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC;
import static org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics.REMOTE_LOG_READER_FETCH_RATE_AND_TIME_METRIC;

View File

@ -16,9 +16,8 @@
*/
package kafka.log.remote;
import kafka.log.remote.quota.RLMQuotaManager;
import org.apache.kafka.common.errors.OffsetOutOfRangeException;
import org.apache.kafka.server.log.remote.quota.RLMQuotaManager;
import org.apache.kafka.storage.internals.log.FetchDataInfo;
import org.apache.kafka.storage.internals.log.RemoteLogReadResult;
import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;

View File

@ -19,8 +19,6 @@ package kafka.log.remote;
import kafka.cluster.EndPoint;
import kafka.cluster.Partition;
import kafka.log.UnifiedLog;
import kafka.log.remote.quota.RLMQuotaManager;
import kafka.log.remote.quota.RLMQuotaManagerConfig;
import kafka.server.KafkaConfig;
import kafka.server.StopPartition;
@ -44,6 +42,8 @@ import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.server.common.OffsetAndEpoch;
import org.apache.kafka.server.config.ServerConfigs;
import org.apache.kafka.server.log.remote.quota.RLMQuotaManager;
import org.apache.kafka.server.log.remote.quota.RLMQuotaManagerConfig;
import org.apache.kafka.server.log.remote.storage.ClassLoaderAwareRemoteStorageManager;
import org.apache.kafka.server.log.remote.storage.LogSegmentData;
import org.apache.kafka.server.log.remote.storage.NoOpRemoteLogMetadataManager;

View File

@ -16,11 +16,11 @@
*/
package kafka.log.remote;
import kafka.log.remote.quota.RLMQuotaManager;
import kafka.utils.TestUtils;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.record.Records;
import org.apache.kafka.server.log.remote.quota.RLMQuotaManager;
import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
import org.apache.kafka.storage.internals.log.FetchDataInfo;
import org.apache.kafka.storage.internals.log.LogOffsetMetadata;

View File

@ -22,8 +22,8 @@ import kafka.cluster.PartitionTest.MockPartitionListener
import kafka.cluster.{BrokerEndPoint, Partition}
import kafka.log._
import kafka.log.remote.RemoteLogManager
import kafka.log.remote.quota.RLMQuotaManagerConfig.INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS
import kafka.log.remote.quota.RLMQuotaMetrics
import org.apache.kafka.server.log.remote.quota.RLMQuotaManagerConfig.INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS
import org.apache.kafka.server.log.remote.quota.RLMQuotaMetrics
import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
import kafka.server.epoch.util.MockBlockingSender
import kafka.utils.TestUtils.waitUntilTrue

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.log.remote.quota;
package org.apache.kafka.server.log.remote.quota;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.metrics.KafkaMetric;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.log.remote.quota;
package org.apache.kafka.server.log.remote.quota;
public class RLMQuotaManagerConfig {
public static final int INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS = 3600;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.log.remote.quota;
package org.apache.kafka.server.log.remote.quota;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.Sensor;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.log.remote.quota;
package org.apache.kafka.server.log.remote.quota;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.metrics.KafkaMetric;