27
27
import org .apache .kafka .common .test .api .ClusterTest ;
28
28
import org .apache .kafka .common .test .api .Type ;
29
29
import org .apache .kafka .metadata .authorizer .StandardAuthorizer ;
30
+ import org .apache .kafka .server .log .remote .storage .NoOpRemoteLogMetadataManager ;
31
+ import org .apache .kafka .server .log .remote .storage .NoOpRemoteStorageManager ;
30
32
31
33
import java .util .LinkedHashMap ;
32
34
import java .util .Map ;
33
35
34
36
import static org .apache .kafka .server .config .ReplicationConfigs .REPLICA_SELECTOR_CLASS_CONFIG ;
35
37
import static org .apache .kafka .server .config .ServerConfigs .AUTHORIZER_CLASS_NAME_CONFIG ;
38
+ import static org .apache .kafka .server .log .remote .storage .RemoteLogManagerConfig .REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP ;
39
+ import static org .apache .kafka .server .log .remote .storage .RemoteLogManagerConfig .REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP ;
40
+ import static org .apache .kafka .server .log .remote .storage .RemoteLogManagerConfig .REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP ;
36
41
import static org .junit .jupiter .api .Assertions .assertEquals ;
37
42
38
43
public class MonitorablePluginsIntegrationTest {
39
-
40
44
private static int controllerId (Type type ) {
41
45
return type == Type .KRAFT ? 3000 : 0 ;
42
46
}
43
47
44
- private static Map <String , String > expectedTags (String config , String clazz ) {
45
- return expectedTags (config , clazz , Map .of ());
46
- }
47
-
48
- private static Map <String , String > expectedTags (String config , String clazz , Map <String , String > extraTags ) {
49
- Map <String , String > tags = new LinkedHashMap <>();
50
- tags .put ("config" , config );
51
- tags .put ("class" , clazz );
52
- tags .putAll (extraTags );
53
- return tags ;
54
- }
55
-
56
48
@ ClusterTest (
57
49
types = {Type .KRAFT , Type .CO_KRAFT },
58
50
serverProperties = {
59
51
@ ClusterConfigProperty (key = StandardAuthorizer .SUPER_USERS_CONFIG , value = "User:ANONYMOUS" ),
60
52
@ ClusterConfigProperty (key = AUTHORIZER_CLASS_NAME_CONFIG , value = "org.apache.kafka.metadata.authorizer.StandardAuthorizer" ),
61
- @ ClusterConfigProperty (key = REPLICA_SELECTOR_CLASS_CONFIG , value = "org.apache.kafka.server.MonitorablePluginsIntegrationTest$MonitorableReplicaSelector" )
53
+ @ ClusterConfigProperty (key = REPLICA_SELECTOR_CLASS_CONFIG , value = "org.apache.kafka.server.MonitorablePluginsIntegrationTest$MonitorableReplicaSelector" ),
54
+ @ ClusterConfigProperty (key = REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP , value = "true" ),
55
+ @ ClusterConfigProperty (key = REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP ,
56
+ value = "org.apache.kafka.server.MonitorablePluginsIntegrationTest$MonitorableNoOpRemoteLogMetadataManager" ),
57
+ @ ClusterConfigProperty (key = REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP ,
58
+ value = "org.apache.kafka.server.MonitorablePluginsIntegrationTest$MonitorableNoOpRemoteStorageManager" )
62
59
}
63
60
)
64
61
public void testMonitorableServerPlugins (ClusterInstance clusterInstance ) {
65
62
assertAuthorizerMetrics (clusterInstance );
66
63
assertReplicaSelectorMetrics (clusterInstance );
64
+ assertRemoteLogManagerMetrics (clusterInstance );
67
65
}
68
66
69
67
private void assertAuthorizerMetrics (ClusterInstance clusterInstance ) {
@@ -78,6 +76,17 @@ private void assertAuthorizerMetrics(ClusterInstance clusterInstance) {
78
76
expectedTags (AUTHORIZER_CLASS_NAME_CONFIG , "StandardAuthorizer" , Map .of ("role" , "controller" )));
79
77
}
80
78
79
+ private void assertRemoteLogManagerMetrics (ClusterInstance clusterInstance ) {
80
+ assertMetrics (
81
+ clusterInstance .brokers ().get (0 ).metrics (),
82
+ MonitorableNoOpRemoteLogMetadataManager .METRICS_COUNT ,
83
+ expectedTags (REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP , MonitorableNoOpRemoteLogMetadataManager .class .getSimpleName ()));
84
+ assertMetrics (
85
+ clusterInstance .brokers ().get (0 ).metrics (),
86
+ MonitorableNoOpRemoteStorageManager .METRICS_COUNT ,
87
+ expectedTags (REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP , MonitorableNoOpRemoteStorageManager .class .getSimpleName ()));
88
+ }
89
+
81
90
private void assertReplicaSelectorMetrics (ClusterInstance clusterInstance ) {
82
91
assertMetrics (
83
92
clusterInstance .brokers ().get (0 ).metrics (),
@@ -98,6 +107,17 @@ private void assertMetrics(Metrics metrics, int expected, Map<String, String> ex
98
107
assertEquals (expected , found );
99
108
}
100
109
110
+ public static class MonitorableNoOpRemoteLogMetadataManager extends NoOpRemoteLogMetadataManager implements Monitorable {
111
+
112
+ private static final int METRICS_COUNT = 1 ;
113
+
114
+ @ Override
115
+ public void withPluginMetrics (PluginMetrics metrics ) {
116
+ MetricName name = metrics .metricName ("name" , "description" , Map .of ());
117
+ metrics .addMetric (name , (Measurable ) (config , now ) -> 123 );
118
+ }
119
+ }
120
+
101
121
public static class MonitorableReplicaSelector extends RackAwareReplicaSelector implements Monitorable {
102
122
103
123
private static final int METRICS_COUNT = 1 ;
@@ -108,4 +128,27 @@ public void withPluginMetrics(PluginMetrics metrics) {
108
128
metrics .addMetric (name , (Measurable ) (config , now ) -> 123 );
109
129
}
110
130
}
131
+
132
+ public static class MonitorableNoOpRemoteStorageManager extends NoOpRemoteStorageManager implements Monitorable {
133
+
134
+ private static final int METRICS_COUNT = 1 ;
135
+
136
+ @ Override
137
+ public void withPluginMetrics (PluginMetrics metrics ) {
138
+ MetricName name = metrics .metricName ("name" , "description" , Map .of ());
139
+ metrics .addMetric (name , (Measurable ) (config , now ) -> 123 );
140
+ }
141
+ }
142
+
143
+ private static Map <String , String > expectedTags (String config , String clazz ) {
144
+ return expectedTags (config , clazz , Map .of ());
145
+ }
146
+
147
+ private static Map <String , String > expectedTags (String config , String clazz , Map <String , String > extraTags ) {
148
+ Map <String , String > tags = new LinkedHashMap <>();
149
+ tags .put ("config" , config );
150
+ tags .put ("class" , clazz );
151
+ tags .putAll (extraTags );
152
+ return tags ;
153
+ }
111
154
}
0 commit comments