From c12bf3ccb69eca8526a213c6847d6d4c0cd7a32a Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 7 May 2024 13:55:37 +0800 Subject: [PATCH] feat(risedev): deprecate zookeeper & use kraft mode for kafka service (#16541) --- Makefile.toml | 18 +--- docs/developer-guide.md | 8 +- risedev.yml | 42 ++------- src/risedevtool/src/bin/risedev-compose.rs | 3 - src/risedevtool/src/bin/risedev-dev.rs | 14 +-- src/risedevtool/src/config.rs | 1 - src/risedevtool/src/config_gen.rs | 2 - src/risedevtool/src/config_gen/kafka_gen.rs | 44 ++++------ .../src/config_gen/zookeeper_gen.rs | 59 ------------- src/risedevtool/src/service_config.rs | 25 +----- src/risedevtool/src/task.rs | 2 - src/risedevtool/src/task/kafka_service.rs | 25 +++++- src/risedevtool/src/task/zookeeper_service.rs | 87 ------------------- 13 files changed, 52 insertions(+), 278 deletions(-) delete mode 100644 src/risedevtool/src/config_gen/zookeeper_gen.rs delete mode 100644 src/risedevtool/src/task/zookeeper_service.rs diff --git a/Makefile.toml b/Makefile.toml index 910642dc8beae..219bad8da6308 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -697,24 +697,11 @@ wait_kafka_exit() { done } -wait_zookeeper_exit() { - # Follow zookeeper-server-stop.sh - while [[ -n "$(ps ax | grep java | grep -i QuorumPeerMain | grep -v grep | awk '{print $1}')" ]]; do - echo "Waiting for zookeeper to exit" - sleep 1 - done -} - kill_kafka() { ${PREFIX_BIN}/kafka/bin/kafka-server-stop.sh wait_kafka_exit } -kill_zookeeper() { - ${PREFIX_BIN}/kafka/bin/zookeeper-server-stop.sh - wait_zookeeper_exit -} - if ! ${TMUX} ls &>/dev/null ; then echo "No risedev cluster to kill. Exiting..." exit 0 @@ -722,7 +709,7 @@ fi # Kill other components with Ctrl+C/Ctrl+D ${TMUX} list-windows -F "#{window_name} #{pane_id}" \ -| grep --invert-match --extended-regexp '(kafka|zookeeper)' \ +| grep --invert-match --extended-regexp '(kafka)' \ | awk '{ print $2 }' \ | xargs -I {} ${TMUX} send-keys -t {} C-c C-d @@ -738,9 +725,6 @@ if [[ -n $(${TMUX} list-windows | grep kafka) ]]; then echo "kill kafka" kill_kafka || true - - echo "kill zookeeper" - kill_zookeeper || true fi ${TMUX} kill-server diff --git a/docs/developer-guide.md b/docs/developer-guide.md index 8cf5dcd4ebbaa..b226ed26e7b8d 100644 --- a/docs/developer-guide.md +++ b/docs/developer-guide.md @@ -185,16 +185,10 @@ For example, you can modify the default section to: - use: frontend - use: prometheus - use: grafana - - use: zookeeper - persist-data: true - use: kafka persist-data: true ``` -> [!NOTE] -> -> The Kafka service depends on the ZooKeeper service. If you want to enable the Kafka component, enable the ZooKeeper component first. - Now you can run `./risedev d` to start a new dev cluster. The new dev cluster will contain components as configured in the yaml file. RiseDev will automatically configure the components to use the available storage service and to monitor the target. You may also add multiple compute nodes in the cluster. The `ci-3cn-1fe` config is an example. @@ -566,4 +560,4 @@ Instructions about submitting PRs are included in the [contribution guidelines]( These correspond to its `depends` field in `pull-request.yml` and `main-cron.yml` . 2. Add `ci/run-e2e-test` to run the step as well. 3. Add `ci/run-main-cron` to run `main-cron` workflow in your pull request, - 4. Add `ci/main-cron/skip-ci` to skip all other steps which were not selected with `ci/run-xxx`. \ No newline at end of file + 4. Add `ci/main-cron/skip-ci` to skip all other steps which were not selected with `ci/run-xxx`. diff --git a/risedev.yml b/risedev.yml index ec057c5bf2c4c..c9b56c19c394b 100644 --- a/risedev.yml +++ b/risedev.yml @@ -43,9 +43,6 @@ profile: # - use: compactor # If you want to create source from Kafka, uncomment the following lines - # Note that kafka depends on zookeeper, so zookeeper must be started beforehand. - # - use: zookeeper - # persist-data: true # - use: kafka # persist-data: true @@ -113,8 +110,6 @@ profile: - use: compactor - use: prometheus - use: grafana - - use: zookeeper - persist-data: true - use: kafka persist-data: true @@ -132,8 +127,6 @@ profile: user-managed: true - use: prometheus - use: grafana - - use: zookeeper - persist-data: true - use: kafka persist-data: true @@ -268,8 +261,6 @@ profile: remote-write-region: "ap-southeast-1" remote-write-url: "https://aps-workspaces.ap-southeast-1.amazonaws.com/workspaces/ws-f3841dad-6a5c-420f-8f62-8f66487f512a/api/v1/remote_write" - use: grafana - - use: zookeeper - persist-data: true - use: kafka persist-data: true @@ -407,10 +398,8 @@ profile: - use: compactor - use: prometheus - use: grafana - # Do not use kafka and zookeeper here, we will spawn it separately, + # Do not use kafka here, we will spawn it separately, # so we don't have to re-generate data each time. - # - use: zookeeper - # persist-data: true # RW will still be ale to talk to it. # - use: kafka # port: 9092 @@ -916,8 +905,6 @@ profile: enable-tiered-cache: true - use: frontend - use: compactor - - use: zookeeper - persist-data: true - use: kafka persist-data: true @@ -1375,17 +1362,17 @@ template: # Listen port of Kafka port: 29092 + # Listen port of KRaft controller + controller-port: 29093 + # Listen address listen-address: ${address} - # ZooKeeper used by this Kafka instance - provide-zookeeper: "zookeeper*" - # If set to true, data will be persisted at data/{id}. persist-data: true - # Kafka broker id. If there are multiple instances of Kafka, we will need to set. - broker-id: 0 + # Kafka node id. If there are multiple instances of Kafka, we will need to set. + node-id: 0 user-managed: false @@ -1399,23 +1386,6 @@ template: persist-data: true - # Apache ZooKeeper service - zookeeper: - # Id to be picked-up by services - id: zookeeper-${port} - - # Advertise address of ZooKeeper - address: "127.0.0.1" - - # Listen address - listen-address: ${address} - - # Listen port of ZooKeeper - port: 2181 - - # If set to true, data will be persisted at data/{id}. - persist-data: true - # Only supported in RiseDev compose redpanda: # Id to be picked-up by services diff --git a/src/risedevtool/src/bin/risedev-compose.rs b/src/risedevtool/src/bin/risedev-compose.rs index c3a7d079e4c4e..10b29e836c66d 100644 --- a/src/risedevtool/src/bin/risedev-compose.rs +++ b/src/risedevtool/src/bin/risedev-compose.rs @@ -201,9 +201,6 @@ fn main() -> Result<()> { ServiceConfig::Pubsub(_) => { return Err(anyhow!("not supported, please use redpanda instead")) } - ServiceConfig::ZooKeeper(_) => { - return Err(anyhow!("not supported, please use redpanda instead")) - } ServiceConfig::Opendal(_) => continue, ServiceConfig::AwsS3(_) => continue, ServiceConfig::RedPanda(c) => { diff --git a/src/risedevtool/src/bin/risedev-dev.rs b/src/risedevtool/src/bin/risedev-dev.rs index 3c5ad8aa082e0..f11bc1b3b5148 100644 --- a/src/risedevtool/src/bin/risedev-dev.rs +++ b/src/risedevtool/src/bin/risedev-dev.rs @@ -27,8 +27,7 @@ use risedev::{ generate_risedev_env, preflight_check, CompactorService, ComputeNodeService, ConfigExpander, ConfigureTmuxTask, DummyService, EnsureStopService, ExecuteContext, FrontendService, GrafanaService, KafkaService, MetaNodeService, MinioService, MySqlService, PrometheusService, - PubsubService, RedisService, ServiceConfig, SqliteConfig, Task, TempoService, ZooKeeperService, - RISEDEV_NAME, + PubsubService, RedisService, ServiceConfig, SqliteConfig, Task, TempoService, RISEDEV_NAME, }; use tempfile::tempdir; use thiserror_ext::AsReport; @@ -272,17 +271,6 @@ fn task_main( ctx.pb .set_message(format!("using Opendal, namenode = {}", c.namenode)); } - ServiceConfig::ZooKeeper(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = ZooKeeperService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = - risedev::ConfigureTcpNodeTask::new(c.address.clone(), c.port, false)?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("zookeeper {}:{}", c.address, c.port)); - } ServiceConfig::Kafka(c) => { let mut ctx = ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); diff --git a/src/risedevtool/src/config.rs b/src/risedevtool/src/config.rs index 45d90daa0c872..541f269ee18c4 100644 --- a/src/risedevtool/src/config.rs +++ b/src/risedevtool/src/config.rs @@ -172,7 +172,6 @@ impl ConfigExpander { "kafka" => ServiceConfig::Kafka(serde_yaml::from_str(&out_str)?), "pubsub" => ServiceConfig::Pubsub(serde_yaml::from_str(&out_str)?), "redis" => ServiceConfig::Redis(serde_yaml::from_str(&out_str)?), - "zookeeper" => ServiceConfig::ZooKeeper(serde_yaml::from_str(&out_str)?), "redpanda" => ServiceConfig::RedPanda(serde_yaml::from_str(&out_str)?), "mysql" => ServiceConfig::MySql(serde_yaml::from_str(&out_str)?), other => return Err(anyhow!("unsupported use type: {}", other)), diff --git a/src/risedevtool/src/config_gen.rs b/src/risedevtool/src/config_gen.rs index 32678c8a94549..d8aaa07b07aed 100644 --- a/src/risedevtool/src/config_gen.rs +++ b/src/risedevtool/src/config_gen.rs @@ -16,8 +16,6 @@ mod prometheus_gen; pub use prometheus_gen::*; mod grafana_gen; pub use grafana_gen::*; -mod zookeeper_gen; -pub use zookeeper_gen::*; mod kafka_gen; pub use kafka_gen::*; mod tempo_gen; diff --git a/src/risedevtool/src/config_gen/kafka_gen.rs b/src/risedevtool/src/config_gen/kafka_gen.rs index 5f54efd5a4e44..b1b1443e907e8 100644 --- a/src/risedevtool/src/config_gen/kafka_gen.rs +++ b/src/risedevtool/src/config_gen/kafka_gen.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; - use crate::KafkaConfig; pub struct KafkaGen; @@ -23,15 +21,10 @@ impl KafkaGen { let kafka_listen_host = &config.listen_address; let kafka_advertise_host = &config.address; let kafka_port = &config.port; - let zookeeper_hosts = config - .provide_zookeeper - .as_ref() - .unwrap() - .iter() - .map(|node| format!("{}:{}", node.address, node.port)) - .join(","); - let kafka_broker_id = config.broker_id; + let controller_port = &config.controller_port; + let kafka_node_id = config.node_id; + // https://github.com/apache/kafka/blob/trunk/config/kraft/server.properties format!( r#"# --- THIS FILE IS AUTO GENERATED BY RISEDEV --- @@ -54,8 +47,14 @@ impl KafkaGen { ############################# Server Basics ############################# -# The id of the broker. This must be set to a unique integer for each broker. -broker.id={kafka_broker_id} +# The role of this server. Setting this puts us in KRaft mode +process.roles=controller,broker + +# The node id associated with this instance's roles +node.id={kafka_node_id} + +# The connect string for the controller quorum +controller.quorum.voters={kafka_node_id}@{kafka_advertise_host}:{controller_port} ############################# Socket Server Settings ############################# @@ -65,7 +64,11 @@ broker.id={kafka_broker_id} # listeners = listener_name://host_name:port # EXAMPLE: # listeners = PLAINTEXT://your.host.name:9092 -listeners=PLAINTEXT://{kafka_listen_host}:{kafka_port} +listeners=PLAINTEXT://{kafka_listen_host}:{kafka_port},CONTROLLER://{kafka_listen_host}:{controller_port} + +# A comma-separated list of the names of the listeners used by the controller. +# This is required if running in KRaft mode. +controller.listener.names=CONTROLLER # Hostname and port the broker will advertise to producers and consumers. If not set, # it uses the value for "listeners" if configured. Otherwise, it will use the value @@ -73,7 +76,7 @@ listeners=PLAINTEXT://{kafka_listen_host}:{kafka_port} advertised.listeners=PLAINTEXT://{kafka_advertise_host}:{kafka_port} # Maps listener names to security protocols, the default is for them to be the same. See the config documentation for more details -#listener.security.protocol.map=PLAINTEXT:PLAINTEXT,SSL:SSL,SASL_PLAINTEXT:SASL_PLAINTEXT,SASL_SSL:SASL_SSL +listener.security.protocol.map=CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT,SSL:SSL,SASL_PLAINTEXT:SASL_PLAINTEXT,SASL_SSL:SASL_SSL # The number of threads that the server uses for receiving requests from the network and sending responses to the network num.network.threads=3 @@ -149,19 +152,6 @@ log.segment.bytes=1073741824 # to the retention policies log.retention.check.interval.ms=300000 -############################# Zookeeper ############################# - -# Zookeeper connection string (see zookeeper docs for details). -# This is a comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". -# You can also append an optional chroot string to the urls to specify the -# root directory for all kafka znodes. -zookeeper.connect={zookeeper_hosts} - -# Timeout in ms for connecting to zookeeper -zookeeper.connection.timeout.ms=18000 - - ############################# Group Coordinator Settings ############################# # The following configuration specifies the time, in milliseconds, that the GroupCoordinator will delay the initial consumer rebalance. diff --git a/src/risedevtool/src/config_gen/zookeeper_gen.rs b/src/risedevtool/src/config_gen/zookeeper_gen.rs deleted file mode 100644 index ed340d58db1c9..0000000000000 --- a/src/risedevtool/src/config_gen/zookeeper_gen.rs +++ /dev/null @@ -1,59 +0,0 @@ -// Copyright 2024 RisingWave Labs -// -// Licensed 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. - -use crate::ZooKeeperConfig; - -pub struct ZooKeeperGen; - -impl ZooKeeperGen { - pub fn gen_server_properties( - &self, - config: &ZooKeeperConfig, - zookeeper_data_dir: &str, - ) -> String { - let zookeeper_listen_host = &config.listen_address; - let zookeeper_port = &config.port; - - format!( - r#"# --- THIS FILE IS AUTO GENERATED BY RISEDEV --- - -# 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. -# the directory where the snapshot is stored. -dataDir={zookeeper_data_dir} -# the port at which the clients will connect -clientPort={zookeeper_port} -clientPortAddress={zookeeper_listen_host} -# disable the per-ip limit on the number of connections since this is a non-production config -maxClientCnxns=0 -# Disable the adminserver by default to avoid port conflicts. -# Set the port to something non-conflicting if choosing to enable this -admin.enableServer=false -# admin.serverPort=8080 -"# - ) - } -} diff --git a/src/risedevtool/src/service_config.rs b/src/risedevtool/src/service_config.rs index 27231552dfc2d..3135d7af4c009 100644 --- a/src/risedevtool/src/service_config.rs +++ b/src/risedevtool/src/service_config.rs @@ -272,11 +272,12 @@ pub struct KafkaConfig { pub address: String, #[serde(with = "string")] pub port: u16, + #[serde(with = "string")] + pub controller_port: u16, pub listen_address: String, - pub provide_zookeeper: Option>, pub persist_data: bool, - pub broker_id: u32, + pub node_id: u32, pub user_managed: bool, } @@ -294,22 +295,6 @@ pub struct PubsubConfig { pub persist_data: bool, } -#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] -#[serde(rename_all = "kebab-case")] -#[serde(deny_unknown_fields)] -pub struct ZooKeeperConfig { - #[serde(rename = "use")] - phantom_use: Option, - pub id: String, - - pub address: String, - #[serde(with = "string")] - pub port: u16, - pub listen_address: String, - - pub persist_data: bool, -} - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] #[serde(rename_all = "kebab-case")] #[serde(deny_unknown_fields)] @@ -374,7 +359,6 @@ pub enum ServiceConfig { Kafka(KafkaConfig), Pubsub(PubsubConfig), Redis(RedisConfig), - ZooKeeper(ZooKeeperConfig), RedPanda(RedPandaConfig), MySql(MySqlConfig), } @@ -393,7 +377,6 @@ impl ServiceConfig { Self::Grafana(c) => &c.id, Self::Tempo(c) => &c.id, Self::AwsS3(c) => &c.id, - Self::ZooKeeper(c) => &c.id, Self::Kafka(c) => &c.id, Self::Pubsub(c) => &c.id, Self::Redis(c) => &c.id, @@ -416,7 +399,6 @@ impl ServiceConfig { Self::Grafana(c) => Some(c.port), Self::Tempo(c) => Some(c.port), Self::AwsS3(_) => None, - Self::ZooKeeper(c) => Some(c.port), Self::Kafka(c) => Some(c.port), Self::Pubsub(c) => Some(c.port), Self::Redis(c) => Some(c.port), @@ -439,7 +421,6 @@ impl ServiceConfig { Self::Grafana(_c) => false, Self::Tempo(_c) => false, Self::AwsS3(_c) => false, - Self::ZooKeeper(_c) => false, Self::Kafka(c) => c.user_managed, Self::Pubsub(_c) => false, Self::Redis(_c) => false, diff --git a/src/risedevtool/src/task.rs b/src/risedevtool/src/task.rs index a0ce29c4d2ba6..24474d14c600f 100644 --- a/src/risedevtool/src/task.rs +++ b/src/risedevtool/src/task.rs @@ -36,7 +36,6 @@ mod task_pubsub_emu_ready_check; mod task_redis_ready_check; mod tempo_service; mod utils; -mod zookeeper_service; use std::env; use std::net::{TcpStream, ToSocketAddrs}; @@ -73,7 +72,6 @@ pub use self::task_kafka_ready_check::*; pub use self::task_pubsub_emu_ready_check::*; pub use self::task_redis_ready_check::*; pub use self::tempo_service::*; -pub use self::zookeeper_service::*; use crate::util::{complete_spin, get_program_args, get_program_name}; use crate::wait::{wait, wait_tcp_available}; diff --git a/src/risedevtool/src/task/kafka_service.rs b/src/risedevtool/src/task/kafka_service.rs index ec3a54599c509..293fcc6cecc82 100644 --- a/src/risedevtool/src/task/kafka_service.rs +++ b/src/risedevtool/src/task/kafka_service.rs @@ -41,6 +41,19 @@ impl KafkaService { fn kafka(&self) -> Result { Ok(Command::new(self.kafka_path()?)) } + + /// Format kraft storage. This is a necessary step to start a fresh Kafka service. + fn kafka_storage_format(&self) -> Result { + let prefix_bin = env::var("PREFIX_BIN")?; + let path = Path::new(&prefix_bin) + .join("kafka") + .join("bin") + .join("kafka-storage.sh"); + + let mut cmd = Command::new(path); + cmd.arg("format").arg("-t").arg("risedev-kafka").arg("-c"); // the remaining arg is the path to the config file + Ok(cmd) + } } impl Task for KafkaService { @@ -81,10 +94,18 @@ impl Task for KafkaService { KafkaGen.gen_server_properties(&self.config, &path.to_string_lossy()), )?; - let mut cmd = self.kafka()?; + // Format storage if empty. + if path.read_dir()?.next().is_none() { + let mut cmd = self.kafka_storage_format()?; + cmd.arg(&config_path); - cmd.arg(config_path); + ctx.pb.set_message("formatting storage..."); + ctx.run_command(cmd)?; + } + let mut cmd = self.kafka()?; + cmd.arg(config_path); + ctx.pb.set_message("starting kafka..."); ctx.run_command(ctx.tmux_run(cmd)?)?; ctx.pb.set_message("started"); diff --git a/src/risedevtool/src/task/zookeeper_service.rs b/src/risedevtool/src/task/zookeeper_service.rs deleted file mode 100644 index 5a94de095cab9..0000000000000 --- a/src/risedevtool/src/task/zookeeper_service.rs +++ /dev/null @@ -1,87 +0,0 @@ -// Copyright 2024 RisingWave Labs -// -// Licensed 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. - -use std::env; -use std::path::{Path, PathBuf}; -use std::process::Command; - -use anyhow::{anyhow, Result}; - -use super::{ExecuteContext, Task}; -use crate::{ZooKeeperConfig, ZooKeeperGen}; - -pub struct ZooKeeperService { - config: ZooKeeperConfig, -} - -impl ZooKeeperService { - pub fn new(config: ZooKeeperConfig) -> Result { - Ok(Self { config }) - } - - fn zookeeper_path(&self) -> Result { - let prefix_bin = env::var("PREFIX_BIN")?; - Ok(Path::new(&prefix_bin) - .join("kafka") - .join("bin") - .join("zookeeper-server-start.sh")) - } - - fn zookeeper(&self) -> Result { - Ok(Command::new(self.zookeeper_path()?)) - } -} - -impl Task for ZooKeeperService { - fn execute(&mut self, ctx: &mut ExecuteContext) -> anyhow::Result<()> { - ctx.service(self); - ctx.pb.set_message("starting..."); - - let path = self.zookeeper_path()?; - if !path.exists() { - return Err(anyhow!("ZooKeeper binary not found in {:?}\nDid you enable kafka feature in `./risedev configure`?", path)); - } - - let prefix_config = env::var("PREFIX_CONFIG")?; - - let path = if self.config.persist_data { - Path::new(&env::var("PREFIX_DATA")?).join(self.id()) - } else { - let path = Path::new("/tmp/risedev").join(self.id()); - fs_err::remove_dir_all(&path).ok(); - path - }; - fs_err::create_dir_all(&path)?; - - let config_path = Path::new(&prefix_config).join(format!("{}.properties", self.id())); - fs_err::write( - &config_path, - ZooKeeperGen.gen_server_properties(&self.config, &path.to_string_lossy()), - )?; - - let mut cmd = self.zookeeper()?; - - cmd.arg(config_path); - - ctx.run_command(ctx.tmux_run(cmd)?)?; - - ctx.pb.set_message("started"); - - Ok(()) - } - - fn id(&self) -> String { - self.config.id.clone() - } -}