Skip to content

Commit

Permalink
Add cluster provision functionality
Browse files Browse the repository at this point in the history
After starting the metdata store service and the grpc server, the node will
try to initialize itself by joining an existing cluster or provisioning the
a new cluster. Either of the two actions will return the NodeId of the node.
Moreover, the initialization procedure makes sure that the latest metadata
is fetched from the metadata store.

The cluster provisioning will ensure that all relevant metadata has been written
to the metadata store. This includes the initial NodesConfiguration, a PartitionTable
that includes the number of partitions and an empty Logs that contains the default
log provider configuration.

This fixes restatedev#2409.
  • Loading branch information
tillrohrmann committed Dec 20, 2024
1 parent 3da5356 commit 0e58b00
Show file tree
Hide file tree
Showing 19 changed files with 1,109 additions and 231 deletions.
1 change: 1 addition & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

30 changes: 7 additions & 23 deletions crates/admin/src/cluster_controller/logs_controller.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,10 +25,9 @@ use xxhash_rust::xxh3::Xxh3Builder;

use restate_bifrost::{Bifrost, BifrostAdmin, Error as BifrostError};
use restate_core::metadata_store::{
retry_on_network_error, MetadataStoreClient, Precondition, ReadWriteError, WriteError,
MetadataStoreClient, Precondition, ReadWriteError, WriteError,
};
use restate_core::{Metadata, MetadataWriter, ShutdownError, TaskCenterFutureExt};
use restate_types::config::Configuration;
use restate_types::errors::GenericError;
use restate_types::identifiers::PartitionId;
use restate_types::live::Pinned;
Expand Down Expand Up @@ -642,9 +641,9 @@ struct LogsControllerInner {
}

impl LogsControllerInner {
fn new(configuration: LogsConfiguration, retry_policy: RetryPolicy) -> Self {
fn new(current_logs: Arc<Logs>, retry_policy: RetryPolicy) -> Self {
Self {
current_logs: Arc::new(Logs::with_logs_configuration(configuration)),
current_logs,
logs_state: HashMap::with_hasher(Xxh3Builder::default()),
logs_write_in_progress: None,
retry_policy,
Expand Down Expand Up @@ -928,26 +927,11 @@ pub struct LogsController {
}

impl LogsController {
pub async fn init(
configuration: &Configuration,
pub fn new(
bifrost: Bifrost,
metadata_store_client: MetadataStoreClient,
metadata_writer: MetadataWriter,
) -> Result<Self> {
// obtain the latest logs or init it with an empty logs variant
let logs = retry_on_network_error(
configuration.common.network_error_retry_policy.clone(),
|| {
metadata_store_client.get_or_insert(BIFROST_CONFIG_KEY.clone(), || {
Logs::from_configuration(configuration)
})
},
)
.await?;

let logs_configuration = logs.configuration().clone();
metadata_writer.update(Arc::new(logs)).await?;

) -> Self {
//todo(azmy): make configurable
let retry_policy = RetryPolicy::exponential(
Duration::from_millis(10),
Expand All @@ -958,7 +942,7 @@ impl LogsController {

let mut this = Self {
effects: Some(Vec::new()),
inner: LogsControllerInner::new(logs_configuration, retry_policy),
inner: LogsControllerInner::new(Metadata::with_current(|m| m.logs_snapshot()), retry_policy),
bifrost,
metadata_store_client,
metadata_writer,
Expand All @@ -967,7 +951,7 @@ impl LogsController {
};

this.find_logs_tail();
Ok(this)
this
}

pub fn find_logs_tail(&mut self) {
Expand Down
32 changes: 1 addition & 31 deletions crates/admin/src/cluster_controller/service.rs
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ use tonic::codec::CompressionEncoding;
use tracing::{debug, info};

use restate_bifrost::{Bifrost, BifrostAdmin, SealedSegment};
use restate_core::metadata_store::{retry_on_network_error, MetadataStoreClient};
use restate_core::metadata_store::MetadataStoreClient;
use restate_core::network::rpc_router::RpcRouter;
use restate_core::network::tonic_service_filter::{TonicServiceFilter, WaitForReady};
use restate_core::network::{
Expand Down Expand Up @@ -295,8 +295,6 @@ impl<T: TransportConnect> Service<T> {
}

pub async fn run(mut self) -> anyhow::Result<()> {
self.init_partition_table().await?;

let mut config_watcher = Configuration::watcher();
let mut cluster_state_watcher = self.cluster_state_refresher.cluster_state_watcher();

Expand Down Expand Up @@ -352,34 +350,6 @@ impl<T: TransportConnect> Service<T> {
}
}

/// creates partition table iff it does not exist
async fn init_partition_table(&mut self) -> anyhow::Result<()> {
let configuration = self.configuration.live_load();

let partition_table = retry_on_network_error(
configuration.common.network_error_retry_policy.clone(),
|| {
self.metadata_store_client
.get_or_insert(PARTITION_TABLE_KEY.clone(), || {
let partition_table = PartitionTable::with_equally_sized_partitions(
Version::MIN,
configuration.common.bootstrap_num_partitions.get(),
);

debug!("Initializing the partition table with '{partition_table:?}'");

partition_table
})
},
)
.await?;

self.metadata_writer
.update(Arc::new(partition_table))
.await?;

Ok(())
}
/// Triggers a snapshot creation for the given partition by issuing an RPC
/// to the node hosting the active leader.
async fn create_partition_snapshot(
Expand Down
6 changes: 2 additions & 4 deletions crates/admin/src/cluster_controller/service/state.rs
Original file line number Diff line number Diff line change
Expand Up @@ -161,13 +161,11 @@ where
)
.await?;

let logs_controller = LogsController::init(
&configuration,
let logs_controller = LogsController::new(
service.bifrost.clone(),
service.metadata_store_client.clone(),
service.metadata_writer.clone(),
)
.await?;
);

let (log_trim_interval, log_trim_threshold) =
create_log_trim_interval(&configuration.admin);
Expand Down
8 changes: 4 additions & 4 deletions crates/bifrost/src/service.rs
Original file line number Diff line number Diff line change
Expand Up @@ -78,10 +78,10 @@ impl BifrostService {
pub fn handle(&self) -> Bifrost {
self.bifrost.clone()
}
/// Runs initialization phase, then returns a handle to join on shutdown.
/// In this phase the system should wait until this is completed before
/// continuing. For instance, a worker mark itself as `STARTING_UP` and not
/// accept any requests until this is completed.

/// Runs initialization phase. In this phase the system should wait until this is completed
/// before continuing. For instance, a worker mark itself as `STARTING_UP` and not accept any
/// requests until this is completed.
///
/// This requires to run within a task_center context.
pub async fn start(self) -> anyhow::Result<()> {
Expand Down
1 change: 1 addition & 0 deletions crates/core/build.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ fn main() -> Result<(), Box<dyn std::error::Error>> {
.protoc_arg("--experimental_allow_proto3_optional")
.extern_path(".restate.node", "::restate_types::protobuf::node")
.extern_path(".restate.common", "::restate_types::protobuf::common")
.extern_path(".restate.cluster", "::restate_types::protobuf::cluster")
.compile_protos(
&["./protobuf/node_ctl_svc.proto"],
&["protobuf", "../types/protobuf"],
Expand Down
25 changes: 25 additions & 0 deletions crates/core/protobuf/node_ctl_svc.proto
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@
syntax = "proto3";

import "google/protobuf/empty.proto";
import "restate/cluster.proto";
import "restate/common.proto";
import "restate/node.proto";

Expand All @@ -20,6 +21,30 @@ service NodeCtlSvc {
rpc GetIdent(google.protobuf.Empty) returns (IdentResponse);

rpc GetMetadata(GetMetadataRequest) returns (GetMetadataResponse);

// Provision the Restate cluster on this node.
rpc ProvisionCluster(ProvisionClusterRequest) returns (ProvisionClusterResponse);
}

message ProvisionClusterRequest {
bool dry_run = 1;
optional uint32 num_partitions = 2;
optional restate.cluster.ReplicationStrategy placement_strategy = 3;
optional restate.cluster.DefaultProvider log_provider = 4;
}

enum ProvisionClusterResponseKind {
ProvisionClusterResponseType_UNKNOWN = 0;
ERROR = 1;
SUCCESS = 2;
DRY_RUN = 3;
}

message ProvisionClusterResponse {
ProvisionClusterResponseKind kind = 1;
// If there is an error, this field will be set. All other fields will be empty.
optional string error = 2;
optional restate.cluster.ClusterConfiguration cluster_configuration = 3;
}

message IdentResponse {
Expand Down
28 changes: 28 additions & 0 deletions crates/core/src/network/protobuf.rs
Original file line number Diff line number Diff line change
Expand Up @@ -9,10 +9,38 @@
// by the Apache License, Version 2.0.

pub mod node_ctl_svc {
use restate_types::protobuf::cluster::ClusterConfiguration;

tonic::include_proto!("restate.node_ctl_svc");

pub const FILE_DESCRIPTOR_SET: &[u8] =
tonic::include_file_descriptor_set!("node_ctl_svc_descriptor");

impl ProvisionClusterResponse {
pub fn dry_run(cluster_configuration: ClusterConfiguration) -> Self {
ProvisionClusterResponse {
kind: ProvisionClusterResponseKind::DryRun.into(),
cluster_configuration: Some(cluster_configuration),
..Default::default()
}
}

pub fn err(message: impl ToString) -> Self {
ProvisionClusterResponse {
kind: ProvisionClusterResponseKind::Error.into(),
error: Some(message.to_string()),
..Default::default()
}
}

pub fn success(cluster_configuration: ClusterConfiguration) -> Self {
ProvisionClusterResponse {
kind: ProvisionClusterResponseKind::Success.into(),
cluster_configuration: Some(cluster_configuration),
..Default::default()
}
}
}
}

pub mod core_node_svc {
Expand Down
1 change: 1 addition & 0 deletions crates/metadata-store/src/local/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ use restate_types::{

use crate::local::grpc::client::LocalMetadataStoreClient;
pub use service::{BuildError, Error, LocalMetadataStoreService};
pub use store::{ProvisionError, ProvisionHandle};

/// Creates a [`MetadataStoreClient`].
pub async fn create_client(
Expand Down
6 changes: 5 additions & 1 deletion crates/metadata-store/src/local/service.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ use restate_types::protobuf::common::MetadataServerStatus;
use crate::grpc_svc;
use crate::grpc_svc::metadata_store_svc_server::MetadataStoreSvcServer;
use crate::local::grpc::handler::LocalMetadataStoreHandler;
use crate::local::store::LocalMetadataStore;
use crate::local::store::{LocalMetadataStore, ProvisionHandle};

pub struct LocalMetadataStoreService {
health_status: HealthStatus<MetadataServerStatus>,
Expand Down Expand Up @@ -63,6 +63,10 @@ impl LocalMetadataStoreService {
})
}

pub fn provision_handle(&self) -> ProvisionHandle {
self.store.provision_handle()
}

pub async fn run(self) -> Result<(), Error> {
let LocalMetadataStoreService {
health_status,
Expand Down
Loading

0 comments on commit 0e58b00

Please sign in to comment.