Skip to content

Commit 2f086d1

Browse files
authored
KAFKA-18892: Add KIP-877 support for ClientQuotaCallback (#19068)
Allow ClientQuotaCallback to implement Monitorable and register metrics. Reviewers: Mickael Maison <[email protected]>, TaiJuWu <[email protected]>, Jhen-Yung Hsu <[email protected]>
1 parent 9a2b8b6 commit 2f086d1

22 files changed

+183
-69
lines changed

Diff for: clients/clients-integration-tests/src/test/java/org/apache/kafka/server/quota/CustomQuotaCallbackTest.java

+74-17
Original file line numberDiff line numberDiff line change
@@ -19,39 +19,44 @@
1919
import org.apache.kafka.clients.admin.Admin;
2020
import org.apache.kafka.clients.admin.NewTopic;
2121
import org.apache.kafka.common.Cluster;
22+
import org.apache.kafka.common.MetricName;
23+
import org.apache.kafka.common.metrics.Gauge;
24+
import org.apache.kafka.common.metrics.Metrics;
25+
import org.apache.kafka.common.metrics.Monitorable;
26+
import org.apache.kafka.common.metrics.PluginMetrics;
2227
import org.apache.kafka.common.security.auth.KafkaPrincipal;
2328
import org.apache.kafka.common.test.ClusterInstance;
2429
import org.apache.kafka.common.test.TestUtils;
2530
import org.apache.kafka.common.test.api.ClusterConfigProperty;
2631
import org.apache.kafka.common.test.api.ClusterTest;
27-
import org.apache.kafka.common.test.api.ClusterTestDefaults;
2832
import org.apache.kafka.common.test.api.Type;
2933
import org.apache.kafka.server.config.QuotaConfig;
3034

35+
import java.util.LinkedHashMap;
3136
import java.util.List;
3237
import java.util.Map;
3338
import java.util.concurrent.ConcurrentHashMap;
3439
import java.util.concurrent.atomic.AtomicInteger;
3540

36-
@ClusterTestDefaults(controllers = 3,
37-
types = {Type.KRAFT},
38-
serverProperties = {
39-
@ClusterConfigProperty(id = 3000, key = QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, value = "org.apache.kafka.server.quota.CustomQuotaCallbackTest$CustomQuotaCallback"),
40-
@ClusterConfigProperty(id = 3001, key = QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, value = "org.apache.kafka.server.quota.CustomQuotaCallbackTest$CustomQuotaCallback"),
41-
@ClusterConfigProperty(id = 3002, key = QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, value = "org.apache.kafka.server.quota.CustomQuotaCallbackTest$CustomQuotaCallback"),
42-
}
43-
)
44-
public class CustomQuotaCallbackTest {
41+
import static org.junit.jupiter.api.Assertions.assertEquals;
4542

46-
private final ClusterInstance cluster;
43+
public class CustomQuotaCallbackTest {
4744

48-
public CustomQuotaCallbackTest(ClusterInstance clusterInstance) {
49-
this.cluster = clusterInstance;
45+
private static int controllerId(Type type) {
46+
return type == Type.KRAFT ? 3000 : 0;
5047
}
5148

52-
@ClusterTest
53-
public void testCustomQuotaCallbackWithControllerServer() throws InterruptedException {
54-
49+
@ClusterTest(
50+
controllers = 3,
51+
types = {Type.KRAFT},
52+
serverProperties = {
53+
@ClusterConfigProperty(id = 3000, key = QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, value = "org.apache.kafka.server.quota.CustomQuotaCallbackTest$CustomQuotaCallback"),
54+
@ClusterConfigProperty(id = 3001, key = QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, value = "org.apache.kafka.server.quota.CustomQuotaCallbackTest$CustomQuotaCallback"),
55+
@ClusterConfigProperty(id = 3002, key = QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, value = "org.apache.kafka.server.quota.CustomQuotaCallbackTest$CustomQuotaCallback"),
56+
}
57+
)
58+
public void testCustomQuotaCallbackWithControllerServer(ClusterInstance cluster) throws InterruptedException {
59+
5560
try (Admin admin = cluster.admin(Map.of())) {
5661
admin.createTopics(List.of(new NewTopic("topic", 1, (short) 1)));
5762
TestUtils.waitForCondition(
@@ -69,10 +74,49 @@ public void testCustomQuotaCallbackWithControllerServer() throws InterruptedExce
6974
&& CustomQuotaCallback.COUNTERS.values().stream().allMatch(counter -> counter.get() > 0),
7075
"The CustomQuotaCallback not triggered in all controllers. "
7176
);
72-
77+
78+
}
79+
}
80+
81+
@ClusterTest(
82+
types = {Type.CO_KRAFT, Type.KRAFT},
83+
serverProperties = {
84+
@ClusterConfigProperty(key = QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, value = "org.apache.kafka.server.quota.CustomQuotaCallbackTest$MonitorableCustomQuotaCallback"),
7385
}
86+
)
87+
public void testMonitorableCustomQuotaCallbackWithCombinedMode(ClusterInstance cluster) {
88+
assertMetrics(
89+
cluster.brokers().get(0).metrics(),
90+
expectedTags(Map.of("role", "broker"))
91+
);
92+
assertMetrics(
93+
cluster.controllers().get(controllerId(cluster.type())).metrics(),
94+
expectedTags(Map.of("role", "controller"))
95+
);
7496
}
7597

98+
private void assertMetrics(Metrics metrics, Map<String, String> expectedTags) {
99+
int found = 0;
100+
for (MetricName metricName : metrics.metrics().keySet()) {
101+
if (metricName.group().equals("plugins")) {
102+
Map<String, String> tags = metricName.tags();
103+
if (expectedTags.equals(tags)) {
104+
assertEquals(MonitorableCustomQuotaCallback.METRIC_NAME, metricName.name());
105+
assertEquals(MonitorableCustomQuotaCallback.METRIC_DESCRIPTION, metricName.description());
106+
found++;
107+
}
108+
}
109+
}
110+
assertEquals(1, found);
111+
}
112+
113+
private static Map<String, String> expectedTags(Map<String, String> extraTags) {
114+
Map<String, String> tags = new LinkedHashMap<>();
115+
tags.put("config", QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG);
116+
tags.put("class", MonitorableCustomQuotaCallback.class.getSimpleName());
117+
tags.putAll(extraTags);
118+
return tags;
119+
}
76120

77121
public static class CustomQuotaCallback implements ClientQuotaCallback {
78122

@@ -121,4 +165,17 @@ public void configure(Map<String, ?> configs) {
121165
}
122166

123167
}
168+
169+
public static class MonitorableCustomQuotaCallback extends CustomQuotaCallback implements Monitorable {
170+
171+
private static final String METRIC_NAME = "monitorable-custom-quota-callback-name";
172+
private static final String METRIC_DESCRIPTION = "monitorable-custom-quota-callback-description";
173+
174+
@Override
175+
public void withPluginMetrics(PluginMetrics metrics) {
176+
MetricName metricName = metrics.metricName(METRIC_NAME, METRIC_DESCRIPTION, Map.of());
177+
metrics.addMetric(metricName, (Gauge<Integer>) (config, now) -> 1);
178+
}
179+
180+
}
124181
}

Diff for: clients/src/main/java/org/apache/kafka/common/internals/Plugin.java

+6
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,12 @@ public static <T> Plugin<T> wrapInstance(T instance, Metrics metrics, String key
4444
return wrapInstance(instance, metrics, () -> tags(key, instance));
4545
}
4646

47+
public static <T> Plugin<T> wrapInstance(T instance, Metrics metrics, String key, Map<String, String> extraTags) {
48+
Map<String, String> tags = tags(key, instance);
49+
tags.putAll(extraTags);
50+
return wrapInstance(instance, metrics, () -> tags);
51+
}
52+
4753
private static <T> Map<String, String> tags(String key, T instance) {
4854
Map<String, String> tags = new LinkedHashMap<>();
4955
tags.put("config", key);

Diff for: clients/src/main/java/org/apache/kafka/server/quota/ClientQuotaCallback.java

+7
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,13 @@
2424

2525
/**
2626
* Quota callback interface for brokers and controllers that enables customization of client quota computation.
27+
* Implement {@link org.apache.kafka.common.metrics.Monitorable} to enable the callback to register metrics.
28+
* The following tags are automatically added to all metrics registered:
29+
* <ul>
30+
* <li><code>config</code> set to <code>client.quota.callback.class</code></li>
31+
* <li><code>class</code> set to the ClientQuotaCallback class name</li>
32+
* <li><code>role</code> set to broker/controller, which indicates the role of the server</li>
33+
* </ul>
2734
*/
2835
public interface ClientQuotaCallback extends Configurable {
2936

Diff for: core/src/main/java/kafka/server/ClientRequestQuotaManager.java

+8-2
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
import kafka.network.RequestChannel;
2020

2121
import org.apache.kafka.common.MetricName;
22+
import org.apache.kafka.common.internals.Plugin;
2223
import org.apache.kafka.common.metrics.Metrics;
2324
import org.apache.kafka.common.metrics.QuotaViolationException;
2425
import org.apache.kafka.common.metrics.Sensor;
@@ -49,8 +50,13 @@ public class ClientRequestQuotaManager extends ClientQuotaManager {
4950
// Visible for testing
5051
private final Sensor exemptSensor;
5152

52-
public ClientRequestQuotaManager(ClientQuotaManagerConfig config, Metrics metrics, Time time, String threadNamePrefix, Optional<ClientQuotaCallback> quotaCallback) {
53-
super(config, metrics, QuotaType.REQUEST, time, threadNamePrefix, OptionConverters.toScala(quotaCallback));
53+
public ClientRequestQuotaManager(
54+
ClientQuotaManagerConfig config,
55+
Metrics metrics, Time time,
56+
String threadNamePrefix,
57+
Optional<Plugin<ClientQuotaCallback>> quotaCallbackPlugin
58+
) {
59+
super(config, metrics, QuotaType.REQUEST, time, threadNamePrefix, OptionConverters.toScala(quotaCallbackPlugin));
5460
this.maxThrottleTimeMs = TimeUnit.SECONDS.toMillis(config.quotaWindowSizeSeconds);
5561
this.metrics = metrics;
5662
this.exemptMetricName = metrics.metricName("exempt-request-time", QuotaType.REQUEST.toString(), "Tracking exempt-request-time utilization percentage");

Diff for: core/src/main/java/kafka/server/QuotaFactory.java

+40-15
Original file line numberDiff line numberDiff line change
@@ -17,17 +17,21 @@
1717
package kafka.server;
1818

1919
import org.apache.kafka.common.TopicPartition;
20+
import org.apache.kafka.common.internals.Plugin;
2021
import org.apache.kafka.common.metrics.Metrics;
2122
import org.apache.kafka.common.utils.Time;
23+
import org.apache.kafka.common.utils.Utils;
2224
import org.apache.kafka.server.config.ClientQuotaManagerConfig;
2325
import org.apache.kafka.server.config.QuotaConfig;
2426
import org.apache.kafka.server.config.ReplicationQuotaManagerConfig;
2527
import org.apache.kafka.server.quota.ClientQuotaCallback;
2628
import org.apache.kafka.server.quota.QuotaType;
2729

30+
import java.util.Map;
2831
import java.util.Optional;
2932

3033
import scala.Option;
34+
import scala.jdk.javaapi.OptionConverters;
3135

3236
public class QuotaFactory {
3337

@@ -56,20 +60,20 @@ public static class QuotaManagers {
5660
private final ReplicationQuotaManager leader;
5761
private final ReplicationQuotaManager follower;
5862
private final ReplicationQuotaManager alterLogDirs;
59-
private final Optional<ClientQuotaCallback> clientQuotaCallback;
63+
private final Optional<Plugin<ClientQuotaCallback>> clientQuotaCallbackPlugin;
6064

6165
public QuotaManagers(ClientQuotaManager fetch, ClientQuotaManager produce, ClientRequestQuotaManager request,
6266
ControllerMutationQuotaManager controllerMutation, ReplicationQuotaManager leader,
6367
ReplicationQuotaManager follower, ReplicationQuotaManager alterLogDirs,
64-
Optional<ClientQuotaCallback> clientQuotaCallback) {
68+
Optional<Plugin<ClientQuotaCallback>> clientQuotaCallbackPlugin) {
6569
this.fetch = fetch;
6670
this.produce = produce;
6771
this.request = request;
6872
this.controllerMutation = controllerMutation;
6973
this.leader = leader;
7074
this.follower = follower;
7175
this.alterLogDirs = alterLogDirs;
72-
this.clientQuotaCallback = clientQuotaCallback;
76+
this.clientQuotaCallbackPlugin = clientQuotaCallbackPlugin;
7377
}
7478

7579
public ClientQuotaManager fetch() {
@@ -100,35 +104,56 @@ public ReplicationQuotaManager alterLogDirs() {
100104
return alterLogDirs;
101105
}
102106

103-
public Optional<ClientQuotaCallback> clientQuotaCallback() {
104-
return clientQuotaCallback;
107+
public Optional<Plugin<ClientQuotaCallback>> clientQuotaCallbackPlugin() {
108+
return clientQuotaCallbackPlugin;
105109
}
106110

107111
public void shutdown() {
108112
fetch.shutdown();
109113
produce.shutdown();
110114
request.shutdown();
111115
controllerMutation.shutdown();
112-
clientQuotaCallback.ifPresent(ClientQuotaCallback::close);
116+
clientQuotaCallbackPlugin.ifPresent(plugin -> Utils.closeQuietly(plugin, "client quota callback plugin"));
113117
}
114118
}
115119

116-
public static QuotaManagers instantiate(KafkaConfig cfg, Metrics metrics, Time time, String threadNamePrefix) {
117-
ClientQuotaCallback clientQuotaCallback = cfg.getConfiguredInstance(
118-
QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, ClientQuotaCallback.class);
120+
public static QuotaManagers instantiate(
121+
KafkaConfig cfg,
122+
Metrics metrics,
123+
Time time,
124+
String threadNamePrefix,
125+
String role
126+
) {
127+
Optional<Plugin<ClientQuotaCallback>> clientQuotaCallbackPlugin = createClientQuotaCallback(cfg, metrics, role);
128+
Option<Plugin<ClientQuotaCallback>> clientQuotaCallbackPluginOption = OptionConverters.toScala(clientQuotaCallbackPlugin);
119129

120130
return new QuotaManagers(
121-
new ClientQuotaManager(clientConfig(cfg), metrics, QuotaType.FETCH, time, threadNamePrefix, Option.apply(clientQuotaCallback)),
122-
new ClientQuotaManager(clientConfig(cfg), metrics, QuotaType.PRODUCE, time, threadNamePrefix, Option.apply(clientQuotaCallback)),
123-
new ClientRequestQuotaManager(clientConfig(cfg), metrics, time, threadNamePrefix, Optional.ofNullable(clientQuotaCallback)),
124-
new ControllerMutationQuotaManager(clientControllerMutationConfig(cfg), metrics, time, threadNamePrefix, Option.apply(clientQuotaCallback)),
131+
new ClientQuotaManager(clientConfig(cfg), metrics, QuotaType.FETCH, time, threadNamePrefix, clientQuotaCallbackPluginOption),
132+
new ClientQuotaManager(clientConfig(cfg), metrics, QuotaType.PRODUCE, time, threadNamePrefix, clientQuotaCallbackPluginOption),
133+
new ClientRequestQuotaManager(clientConfig(cfg), metrics, time, threadNamePrefix, clientQuotaCallbackPlugin),
134+
new ControllerMutationQuotaManager(clientControllerMutationConfig(cfg), metrics, time, threadNamePrefix, clientQuotaCallbackPluginOption),
125135
new ReplicationQuotaManager(replicationConfig(cfg), metrics, QuotaType.LEADER_REPLICATION, time),
126136
new ReplicationQuotaManager(replicationConfig(cfg), metrics, QuotaType.FOLLOWER_REPLICATION, time),
127137
new ReplicationQuotaManager(alterLogDirsReplicationConfig(cfg), metrics, QuotaType.ALTER_LOG_DIRS_REPLICATION, time),
128-
Optional.ofNullable(clientQuotaCallback)
138+
clientQuotaCallbackPlugin
129139
);
130140
}
131141

142+
private static Optional<Plugin<ClientQuotaCallback>> createClientQuotaCallback(
143+
KafkaConfig cfg,
144+
Metrics metrics,
145+
String role
146+
) {
147+
ClientQuotaCallback clientQuotaCallback = cfg.getConfiguredInstance(
148+
QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, ClientQuotaCallback.class);
149+
return clientQuotaCallback == null ? Optional.empty() : Optional.of(Plugin.wrapInstance(
150+
clientQuotaCallback,
151+
metrics,
152+
QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG,
153+
Map.of("role", role)
154+
));
155+
}
156+
132157
private static ClientQuotaManagerConfig clientConfig(KafkaConfig cfg) {
133158
return new ClientQuotaManagerConfig(
134159
cfg.quotaConfig().numQuotaSamples(),
@@ -156,4 +181,4 @@ private static ReplicationQuotaManagerConfig alterLogDirsReplicationConfig(Kafka
156181
cfg.quotaConfig().alterLogDirsReplicationQuotaWindowSizeSeconds()
157182
);
158183
}
159-
}
184+
}

Diff for: core/src/main/scala/kafka/server/BrokerServer.scala

+2-2
Original file line numberDiff line numberDiff line change
@@ -54,7 +54,7 @@ import org.apache.kafka.server.share.persister.{DefaultStatePersister, NoOpState
5454
import org.apache.kafka.server.share.session.ShareSessionCache
5555
import org.apache.kafka.server.util.timer.{SystemTimer, SystemTimerReaper}
5656
import org.apache.kafka.server.util.{Deadline, FutureUtils, KafkaScheduler}
57-
import org.apache.kafka.server.{AssignmentsManager, BrokerFeatures, ClientMetricsManager, DelayedActionQueue}
57+
import org.apache.kafka.server.{AssignmentsManager, BrokerFeatures, ClientMetricsManager, DelayedActionQueue, ProcessRole}
5858
import org.apache.kafka.storage.internals.log.LogDirFailureChannel
5959
import org.apache.kafka.storage.log.metrics.BrokerTopicStats
6060

@@ -196,7 +196,7 @@ class BrokerServer(
196196
val clientMetricsReceiverPlugin = new ClientMetricsReceiverPlugin()
197197

198198
config.dynamicConfig.initialize(Some(clientMetricsReceiverPlugin))
199-
quotaManagers = QuotaFactory.instantiate(config, metrics, time, s"broker-${config.nodeId}-")
199+
quotaManagers = QuotaFactory.instantiate(config, metrics, time, s"broker-${config.nodeId}-", ProcessRole.BrokerRole.toString)
200200
DynamicBrokerConfig.readDynamicBrokerConfigsFromSnapshot(raftManager, config, quotaManagers)
201201

202202
/* start scheduler */

Diff for: core/src/main/scala/kafka/server/ClientQuotaManager.scala

+9-4
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ import java.util.function.Consumer
2323
import kafka.network.RequestChannel
2424
import kafka.server.ClientQuotaManager._
2525
import kafka.utils.Logging
26+
import org.apache.kafka.common.internals.Plugin
2627
import org.apache.kafka.common.{Cluster, MetricName}
2728
import org.apache.kafka.common.metrics._
2829
import org.apache.kafka.common.metrics.Metrics
@@ -137,22 +138,26 @@ object ClientQuotaManager {
137138
* @param quotaType Quota type of this quota manager
138139
* @param time @Time object to use
139140
* @param threadNamePrefix The thread prefix to use
140-
* @param clientQuotaCallback An optional @ClientQuotaCallback
141+
* @param clientQuotaCallbackPlugin An optional @ClientQuotaCallback and
142+
* wrap it in a {@link org.apache.kafka.common.internals.Plugin}
141143
*/
142144
class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
143145
private val metrics: Metrics,
144146
private val quotaType: QuotaType,
145147
private val time: Time,
146148
private val threadNamePrefix: String,
147-
private val clientQuotaCallback: Option[ClientQuotaCallback] = None) extends Logging {
149+
private val clientQuotaCallbackPlugin: Option[Plugin[ClientQuotaCallback]] = None) extends Logging {
148150

149151
private val lock = new ReentrantReadWriteLock()
150152
private val sensorAccessor = new SensorAccess(lock, metrics)
151-
private val quotaCallback = clientQuotaCallback.getOrElse(new DefaultQuotaCallback)
153+
private val quotaCallback = clientQuotaCallbackPlugin match {
154+
case Some(plugin) => plugin.get()
155+
case None => new DefaultQuotaCallback
156+
}
152157
private val clientQuotaType = QuotaType.toClientQuotaType(quotaType)
153158

154159
@volatile
155-
private var quotaTypesEnabled = clientQuotaCallback match {
160+
private var quotaTypesEnabled = clientQuotaCallbackPlugin match {
156161
case Some(_) => QuotaTypes.CustomQuotas
157162
case None => QuotaTypes.NoQuotas
158163
}

Diff for: core/src/main/scala/kafka/server/ControllerMutationQuotaManager.scala

+2-1
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ package kafka.server
1919
import kafka.network.RequestChannel
2020
import org.apache.kafka.common.MetricName
2121
import org.apache.kafka.common.errors.ThrottlingQuotaExceededException
22+
import org.apache.kafka.common.internals.Plugin
2223
import org.apache.kafka.common.metrics.Metrics
2324
import org.apache.kafka.common.metrics.QuotaViolationException
2425
import org.apache.kafka.common.metrics.Sensor
@@ -165,7 +166,7 @@ class ControllerMutationQuotaManager(private val config: ClientQuotaManagerConfi
165166
private val metrics: Metrics,
166167
private val time: Time,
167168
private val threadNamePrefix: String,
168-
private val quotaCallback: Option[ClientQuotaCallback])
169+
private val quotaCallback: Option[Plugin[ClientQuotaCallback]])
169170
extends ClientQuotaManager(config, metrics, QuotaType.CONTROLLER_MUTATION, time, threadNamePrefix, quotaCallback) {
170171

171172
override protected def clientQuotaMetricName(quotaMetricTags: Map[String, String]): MetricName = {

Diff for: core/src/main/scala/kafka/server/ControllerServer.scala

+2-1
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,7 @@ import org.apache.kafka.metadata.bootstrap.BootstrapMetadata
4040
import org.apache.kafka.metadata.publisher.{AclPublisher, FeaturesPublisher}
4141
import org.apache.kafka.raft.QuorumConfig
4242
import org.apache.kafka.security.CredentialProvider
43+
import org.apache.kafka.server.ProcessRole
4344
import org.apache.kafka.server.authorizer.Authorizer
4445
import org.apache.kafka.server.config.ServerLogConfigs.{ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG}
4546
import org.apache.kafka.server.common.{ApiMessageAndVersion, KRaftVersion, NodeToControllerChannelManager}
@@ -266,7 +267,7 @@ class ControllerServer(
266267
quotaManagers = QuotaFactory.instantiate(config,
267268
metrics,
268269
time,
269-
s"controller-${config.nodeId}-")
270+
s"controller-${config.nodeId}-", ProcessRole.ControllerRole.toString)
270271
clientQuotaMetadataManager = new ClientQuotaMetadataManager(quotaManagers, socketServer.connectionQuotas)
271272
controllerApis = new ControllerApis(socketServer.dataPlaneRequestChannel,
272273
authorizer,

0 commit comments

Comments
 (0)