Configure orchestrator's key components in builder's build method (#2802)

## Motivation and Context
Moves setting orchestrator components out of `ServiceRuntimePlugin` and
puts it in service config builders' build method.

## Description
This PR is the forth in a series of config refactoring. Here, we move
pieces of code out of `ServiceRuntimePlugin::config` method so that key
orchestrator components meant for the service-level config should only
be constructed once when a service config is created, e.g. during
builder's `build` method. Previously, those components were newly
created every time an operation is invoked.

Wherever `self.handle.conf...` is used, the PR has moved it from
`ServiceRuntimePlugin::config` to the builders' `build` method.

Note that there will be a separate PR to better handle auth resolver &
identity resolver in the context of the ongoing config refactoring.

## Testing
- [x] Passed tests in CI

----

_By submitting this pull request, I confirm that you can use, modify,
copy, and redistribute this contribution, under the terms of your
choice._

---------

Co-authored-by: Yuki Saito <awsaito@amazon.com>
This commit is contained in:
ysaito1001 2023-06-21 22:33:17 -05:00 committed by GitHub
parent e6293b2282
commit 8dc8963146
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 153 additions and 116 deletions

View File

@ -145,11 +145,11 @@ class CredentialCacheConfig(codegenContext: ClientCodegenContext) : ConfigCustom
if (runtimeMode.defaultToOrchestrator) {
rustTemplate(
"""
self.inner.store_put(
self.inner.load::<#{CredentialsCache}>()
layer.store_put(
layer.load::<#{CredentialsCache}>()
.cloned()
.unwrap_or_else({
let sleep = self.inner.load::<#{SharedAsyncSleep}>().cloned();
let sleep = layer.load::<#{SharedAsyncSleep}>().cloned();
|| match sleep {
Some(sleep) => {
#{CredentialsCache}::lazy_builder()
@ -159,7 +159,7 @@ class CredentialCacheConfig(codegenContext: ClientCodegenContext) : ConfigCustom
None => #{CredentialsCache}::lazy(),
}
})
.create_cache(self.inner.load::<#{SharedCredentialsProvider}>().cloned().unwrap_or_else(|| {
.create_cache(layer.load::<#{SharedCredentialsProvider}>().cloned().unwrap_or_else(|| {
#{SharedCredentialsProvider}::new(#{DefaultProvider})
}))
);

View File

@ -78,22 +78,6 @@ private class AuthServiceRuntimePluginCustomization(private val codegenContext:
// enable the aws-runtime `sign-eventstream` feature
addDependency(AwsCargoDependency.awsRuntime(runtimeConfig).withFeature("event-stream").toType().toSymbol())
}
section.putConfigValue(this) {
rustTemplate("#{SigningService}::from_static(self.handle.conf.signing_service())", *codegenScope)
}
rustTemplate(
"""
if let Some(region) = self.handle.conf.region() {
#{put_signing_region}
}
""",
*codegenScope,
"put_signing_region" to writable {
section.putConfigValue(this) {
rustTemplate("#{SigningRegion}::from(region.clone())", *codegenScope)
}
},
)
}
else -> {}

View File

@ -14,6 +14,7 @@ import software.amazon.smithy.model.shapes.ServiceShape
import software.amazon.smithy.model.shapes.ShapeId
import software.amazon.smithy.model.traits.OptionalAuthTrait
import software.amazon.smithy.rust.codegen.client.smithy.ClientCodegenContext
import software.amazon.smithy.rust.codegen.client.smithy.SmithyRuntimeMode
import software.amazon.smithy.rust.codegen.client.smithy.customize.ClientCodegenDecorator
import software.amazon.smithy.rust.codegen.client.smithy.generators.OperationCustomization
import software.amazon.smithy.rust.codegen.client.smithy.generators.OperationSection
@ -54,6 +55,7 @@ class SigV4SigningDecorator : ClientCodegenDecorator {
return baseCustomizations.extendIf(applies(codegenContext)) {
SigV4SigningConfig(
codegenContext.runtimeConfig,
codegenContext.smithyRuntimeMode,
codegenContext.serviceShape.hasEventStreamOperations(codegenContext.model),
codegenContext.serviceShape.expectTrait(),
)
@ -78,26 +80,48 @@ class SigV4SigningDecorator : ClientCodegenDecorator {
class SigV4SigningConfig(
private val runtimeConfig: RuntimeConfig,
private val runtimeMode: SmithyRuntimeMode,
private val serviceHasEventStream: Boolean,
private val sigV4Trait: SigV4Trait,
) : ConfigCustomization() {
private val codegenScope = arrayOf(
"Region" to AwsRuntimeType.awsTypes(runtimeConfig).resolve("region::Region"),
"SigningService" to AwsRuntimeType.awsTypes(runtimeConfig).resolve("SigningService"),
"SigningRegion" to AwsRuntimeType.awsTypes(runtimeConfig).resolve("region::SigningRegion"),
)
override fun section(section: ServiceConfig): Writable = writable {
if (section is ServiceConfig.ConfigImpl) {
if (serviceHasEventStream) {
// enable the aws-sig-auth `sign-eventstream` feature
addDependency(AwsRuntimeType.awsSigAuthEventStream(runtimeConfig).toSymbol())
}
rust(
"""
/// The signature version 4 service signing name to use in the credential scope when signing requests.
///
/// The signing service may be overridden by the `Endpoint`, or by specifying a custom
/// [`SigningService`](aws_types::SigningService) during operation construction
pub fn signing_service(&self) -> &'static str {
${sigV4Trait.name.dq()}
when (section) {
ServiceConfig.ConfigImpl -> {
if (serviceHasEventStream) {
// enable the aws-sig-auth `sign-eventstream` feature
addDependency(AwsRuntimeType.awsSigAuthEventStream(runtimeConfig).toSymbol())
}
""",
)
rust(
"""
/// The signature version 4 service signing name to use in the credential scope when signing requests.
///
/// The signing service may be overridden by the `Endpoint`, or by specifying a custom
/// [`SigningService`](aws_types::SigningService) during operation construction
pub fn signing_service(&self) -> &'static str {
${sigV4Trait.name.dq()}
}
""",
)
}
ServiceConfig.BuilderBuild -> {
if (runtimeMode.defaultToOrchestrator) {
rustTemplate(
"""
layer.put(#{SigningService}::from_static(${sigV4Trait.name.dq()}));
layer.load::<#{Region}>().cloned().map(|r| layer.put(#{SigningRegion}::from(r)));
""",
*codegenScope,
)
}
}
else -> emptySection
}
}
}

View File

@ -104,12 +104,6 @@ class UserAgentDecorator : ClientCodegenDecorator {
override fun section(section: ServiceRuntimePluginSection): Writable = writable {
when (section) {
is ServiceRuntimePluginSection.AdditionalConfig -> {
section.putConfigValue(this) {
rust("#T.clone()", ClientRustModule.Meta.toType().resolve("API_METADATA"))
}
}
is ServiceRuntimePluginSection.RegisterInterceptor -> {
section.registerInterceptor(runtimeConfig, this) {
rust("#T::new()", awsRuntime.resolve("user_agent::UserAgentInterceptor"))
@ -212,7 +206,9 @@ class UserAgentDecorator : ClientCodegenDecorator {
}
is ServiceConfig.BuilderBuild -> writable {
if (runtimeMode.defaultToMiddleware) {
if (runtimeMode.defaultToOrchestrator) {
rust("layer.put(#T.clone());", ClientRustModule.Meta.toType().resolve("API_METADATA"))
} else {
rust("app_name: self.app_name,")
}
}

View File

@ -30,6 +30,7 @@ internal class SigV4SigningDecoratorTest {
codegenContext,
SigV4SigningConfig(
codegenContext.runtimeConfig,
codegenContext.smithyRuntimeMode,
true,
SigV4Trait.builder().name("test-service").build(),
),

View File

@ -38,7 +38,12 @@ private class HttpConnectorConfigCustomization(
private val moduleUseName = codegenContext.moduleUseName()
private val codegenScope = arrayOf(
*preludeScope,
"Connection" to RuntimeType.smithyRuntimeApi(runtimeConfig).resolve("client::orchestrator::Connection"),
"ConnectorSettings" to RuntimeType.smithyClient(runtimeConfig).resolve("http_connector::ConnectorSettings"),
"DynConnectorAdapter" to RuntimeType.smithyRuntime(runtimeConfig).resolve("client::connections::adapter::DynConnectorAdapter"),
"HttpConnector" to RuntimeType.smithyClient(runtimeConfig).resolve("http_connector::HttpConnector"),
"SharedAsyncSleep" to RuntimeType.smithyAsync(runtimeConfig).resolve("rt::sleep::SharedAsyncSleep"),
"TimeoutConfig" to RuntimeType.smithyTypes(runtimeConfig).resolve("timeout::TimeoutConfig"),
)
override fun section(section: ServiceConfig): Writable {
@ -183,7 +188,29 @@ private class HttpConnectorConfigCustomization(
}
is ServiceConfig.BuilderBuild -> writable {
if (runtimeMode.defaultToMiddleware) {
if (runtimeMode.defaultToOrchestrator) {
rustTemplate(
"""
let sleep_impl = layer.load::<#{SharedAsyncSleep}>().cloned();
let timeout_config = layer.load::<#{TimeoutConfig}>().cloned().unwrap_or_else(#{TimeoutConfig}::disabled);
let connector_settings = #{ConnectorSettings}::from_timeout_config(&timeout_config);
if let Some(connection) = layer.load::<#{HttpConnector}>()
.and_then(|c| c.connector(&connector_settings, sleep_impl.clone()))
.or_else(|| #{default_connector}(&connector_settings, sleep_impl)) {
let connection: #{Box}<dyn #{Connection}> = #{Box}::new(#{DynConnectorAdapter}::new(
// TODO(enableNewSmithyRuntimeCleanup): Replace the tower-based DynConnector and remove DynConnectorAdapter when deleting the middleware implementation
connection
)) as _;
layer.set_connection(connection);
}
""",
*codegenScope,
"default_connector" to RuntimeType.smithyClient(runtimeConfig).resolve("conns::default_connector"),
)
} else {
rust("http_connector: self.http_connector,")
}
}

View File

@ -7,11 +7,8 @@ package software.amazon.smithy.rust.codegen.client.smithy.customizations
import software.amazon.smithy.rust.codegen.client.smithy.ClientCodegenContext
import software.amazon.smithy.rust.codegen.client.smithy.ClientRustModule
import software.amazon.smithy.rust.codegen.client.smithy.generators.ServiceRuntimePluginCustomization
import software.amazon.smithy.rust.codegen.client.smithy.generators.ServiceRuntimePluginSection
import software.amazon.smithy.rust.codegen.client.smithy.generators.config.ConfigCustomization
import software.amazon.smithy.rust.codegen.client.smithy.generators.config.ServiceConfig
import software.amazon.smithy.rust.codegen.core.rustlang.Writable
import software.amazon.smithy.rust.codegen.core.rustlang.rust
import software.amazon.smithy.rust.codegen.core.rustlang.rustTemplate
import software.amazon.smithy.rust.codegen.core.rustlang.writable
@ -32,6 +29,7 @@ class ResiliencyConfigCustomization(codegenContext: ClientCodegenContext) : Conf
"RetryConfig" to retryConfig.resolve("RetryConfig"),
"SharedAsyncSleep" to sleepModule.resolve("SharedAsyncSleep"),
"Sleep" to sleepModule.resolve("Sleep"),
"StandardRetryStrategy" to RuntimeType.smithyRuntime(runtimeConfig).resolve("client::retries::strategy::StandardRetryStrategy"),
"TimeoutConfig" to timeoutModule.resolve("TimeoutConfig"),
)
@ -316,7 +314,15 @@ class ResiliencyConfigCustomization(codegenContext: ClientCodegenContext) : Conf
}
ServiceConfig.BuilderBuild -> {
if (runtimeMode.defaultToMiddleware) {
if (runtimeMode.defaultToOrchestrator) {
rustTemplate(
"""
let retry_config = layer.load::<#{RetryConfig}>().cloned().unwrap_or_else(#{RetryConfig}::disabled);
layer.set_retry_strategy(#{StandardRetryStrategy}::new(&retry_config));
""",
*codegenScope,
)
} else {
rustTemplate(
// We call clone on sleep_impl because the field is used by
// initializing the credentials_cache field later in the build
@ -363,28 +369,3 @@ class ResiliencyReExportCustomization(private val runtimeConfig: RuntimeConfig)
}
}
}
class ResiliencyServiceRuntimePluginCustomization(
private val codegenContext: ClientCodegenContext,
) : ServiceRuntimePluginCustomization() {
override fun section(section: ServiceRuntimePluginSection): Writable = writable {
if (section is ServiceRuntimePluginSection.AdditionalConfig) {
rustTemplate(
"""
if let Some(sleep_impl) = self.handle.conf.sleep_impl() {
${section.newLayerName}.put(sleep_impl);
}
if let Some(timeout_config) = self.handle.conf.timeout_config() {
${section.newLayerName}.put(timeout_config.clone());
}
${section.newLayerName}.put(self.handle.conf.time_source()#{maybe_clone});
""",
"maybe_clone" to writable {
if (codegenContext.smithyRuntimeMode.defaultToMiddleware) {
rust(".clone()")
}
},
)
}
}
}

View File

@ -114,7 +114,7 @@ class TimeSourceCustomization(codegenContext: ClientCodegenContext) : ConfigCust
ServiceConfig.BuilderBuild -> {
if (runtimeMode.defaultToOrchestrator) {
rustTemplate(
"self.inner.store_put(self.inner.load::<#{SharedTimeSource}>().cloned().unwrap_or_default());",
"layer.store_put(layer.load::<#{SharedTimeSource}>().cloned().unwrap_or_default());",
*codegenScope,
)
} else {

View File

@ -15,11 +15,9 @@ import software.amazon.smithy.rust.codegen.client.smithy.customizations.Idempote
import software.amazon.smithy.rust.codegen.client.smithy.customizations.InterceptorConfigCustomization
import software.amazon.smithy.rust.codegen.client.smithy.customizations.ResiliencyConfigCustomization
import software.amazon.smithy.rust.codegen.client.smithy.customizations.ResiliencyReExportCustomization
import software.amazon.smithy.rust.codegen.client.smithy.customizations.ResiliencyServiceRuntimePluginCustomization
import software.amazon.smithy.rust.codegen.client.smithy.customizations.TimeSourceCustomization
import software.amazon.smithy.rust.codegen.client.smithy.customizations.TimeSourceOperationCustomization
import software.amazon.smithy.rust.codegen.client.smithy.generators.OperationCustomization
import software.amazon.smithy.rust.codegen.client.smithy.generators.ServiceRuntimePluginCustomization
import software.amazon.smithy.rust.codegen.client.smithy.generators.config.ConfigCustomization
import software.amazon.smithy.rust.codegen.core.rustlang.Feature
import software.amazon.smithy.rust.codegen.core.smithy.RustCrate
@ -92,10 +90,4 @@ class RequiredCustomizations : ClientCodegenDecorator {
}
}
}
override fun serviceRuntimePluginCustomizations(
codegenContext: ClientCodegenContext,
baseCustomizations: List<ServiceRuntimePluginCustomization>,
): List<ServiceRuntimePluginCustomization> =
baseCustomizations + listOf(ResiliencyServiceRuntimePluginCustomization(codegenContext))
}

View File

@ -34,6 +34,7 @@ internal class EndpointConfigCustomization(
val resolverTrait = "#{SmithyResolver}<#{Params}>"
val codegenScope = arrayOf(
*preludeScope,
"DefaultEndpointResolver" to RuntimeType.smithyRuntime(runtimeConfig).resolve("client::orchestrator::endpoints::DefaultEndpointResolver"),
"SharedEndpointResolver" to types.sharedEndpointResolver,
"SmithyResolver" to types.resolveEndpoint,
"Params" to typesGenerator.paramsStruct(),
@ -165,9 +166,11 @@ internal class EndpointConfigCustomization(
if (runtimeMode.defaultToOrchestrator) {
rustTemplate(
"""
self.inner.store_put(self.inner.load::<$sharedEndpointResolver>().cloned().unwrap_or_else(||
let endpoint_resolver = #{DefaultEndpointResolver}::<#{Params}>::new(
layer.load::<$sharedEndpointResolver>().cloned().unwrap_or_else(||
#{SharedEndpointResolver}::new(#{DefaultResolver}::new())
));
layer.set_endpoint_resolver(endpoint_resolver);
""",
*codegenScope,
"DefaultResolver" to defaultResolver,
@ -206,7 +209,11 @@ internal class EndpointConfigCustomization(
if (runtimeMode.defaultToOrchestrator) {
rustTemplate(
"""
self.inner.store_put(self.inner.load::<$sharedEndpointResolver>().cloned().unwrap_or_else(||#{SharedEndpointResolver}::new(#{FailingResolver})));
let endpoint_resolver = #{DefaultEndpointResolver}::<#{Params}>::new(
layer.load::<$sharedEndpointResolver>().cloned().unwrap_or_else(||
#{SharedEndpointResolver}::new(#{FailingResolver})
).clone());
layer.set_endpoint_resolver(endpoint_resolver);
""",
*codegenScope,
"FailingResolver" to alwaysFailsResolver,

View File

@ -56,6 +56,7 @@ open class OperationGenerator(
config_override: #{Option}<crate::config::Builder>,
) -> #{RuntimePlugins} {
let mut runtime_plugins = runtime_plugins
.with_client_plugin(handle.conf.clone())
.with_client_plugin(crate::config::ServiceRuntimePlugin::new(handle))
.with_operation_plugin(operation);
if let Some(config_override) = config_override {

View File

@ -54,6 +54,7 @@ class ServiceGenerator(
ServiceRuntimePluginGenerator(codegenContext)
.render(this, decorator.serviceRuntimePluginCustomizations(codegenContext, emptyList()))
serviceConfigGenerator.renderRuntimePluginImplForSelf(this)
serviceConfigGenerator.renderRuntimePluginImplForBuilder(this)
}
}

View File

@ -90,7 +90,6 @@ class ServiceRuntimePluginGenerator(
"ConfigBagAccessors" to runtimeApi.resolve("client::orchestrator::ConfigBagAccessors"),
"Connection" to runtimeApi.resolve("client::orchestrator::Connection"),
"ConnectorSettings" to RuntimeType.smithyClient(rc).resolve("http_connector::ConnectorSettings"),
"DefaultEndpointResolver" to runtime.resolve("client::orchestrator::endpoints::DefaultEndpointResolver"),
"DynConnectorAdapter" to runtime.resolve("client::connections::adapter::DynConnectorAdapter"),
"HttpAuthSchemes" to runtimeApi.resolve("client::auth::HttpAuthSchemes"),
"HttpConnector" to client.resolve("http_connector::HttpConnector"),
@ -103,7 +102,6 @@ class ServiceRuntimePluginGenerator(
"ResolveEndpoint" to http.resolve("endpoint::ResolveEndpoint"),
"RuntimePlugin" to runtimeApi.resolve("client::runtime_plugin::RuntimePlugin"),
"StaticAuthOptionResolver" to runtimeApi.resolve("client::auth::option_resolver::StaticAuthOptionResolver"),
"default_connector" to client.resolve("conns::default_connector"),
"require_connector" to client.resolve("conns::require_connector"),
"TimeoutConfig" to smithyTypes.resolve("timeout::TimeoutConfig"),
"RetryConfig" to smithyTypes.resolve("retry::RetryConfig"),
@ -140,37 +138,13 @@ class ServiceRuntimePluginGenerator(
// Set an empty auth option resolver to be overridden by operations that need auth.
cfg.set_auth_option_resolver(#{StaticAuthOptionResolver}::new(#{Vec}::new()));
let endpoint_resolver = #{DefaultEndpointResolver}::<#{Params}>::new(
self.handle.conf.endpoint_resolver());
cfg.set_endpoint_resolver(endpoint_resolver);
// TODO(enableNewSmithyRuntimeLaunch): Make it possible to set retry classifiers at the service level.
// Retry classifiers can also be set at the operation level and those should be added to the
// list of classifiers defined here, rather than replacing them.
let sleep_impl = self.handle.conf.sleep_impl();
let timeout_config = self.handle.conf.timeout_config().cloned().unwrap_or_else(#{TimeoutConfig}::disabled);
let retry_config = self.handle.conf.retry_config().cloned().unwrap_or_else(#{RetryConfig}::disabled);
cfg.set_retry_strategy(#{StandardRetryStrategy}::new(&retry_config));
let connector_settings = #{ConnectorSettings}::from_timeout_config(&timeout_config);
if let Some(connection) = self.handle.conf.http_connector()
.and_then(|c| c.connector(&connector_settings, sleep_impl.clone()))
.or_else(|| #{default_connector}(&connector_settings, sleep_impl)) {
let connection: #{Box}<dyn #{Connection}> = #{Box}::new(#{DynConnectorAdapter}::new(
// TODO(enableNewSmithyRuntimeCleanup): Replace the tower-based DynConnector and remove DynConnectorAdapter when deleting the middleware implementation
connection
)) as _;
cfg.set_connection(connection);
}
#{additional_config}
Some(cfg.freeze())
}
fn interceptors(&self, interceptors: &mut #{InterceptorRegistrar}) {
interceptors.extend(self.handle.conf.interceptors().cloned());
let _interceptors = interceptors;
#{additional_interceptors}
}
}
@ -183,7 +157,7 @@ class ServiceRuntimePluginGenerator(
writeCustomizations(customizations, ServiceRuntimePluginSection.AdditionalConfig("cfg"))
},
"additional_interceptors" to writable {
writeCustomizations(customizations, ServiceRuntimePluginSection.RegisterInterceptor("interceptors"))
writeCustomizations(customizations, ServiceRuntimePluginSection.RegisterInterceptor("_interceptors"))
},
)
}

View File

@ -108,7 +108,7 @@ class IdempotencyTokenProviderCustomization(codegenContext: ClientCodegenContext
ServiceConfig.BuilderBuild -> writable {
if (runtimeMode.defaultToOrchestrator) {
rustTemplate(
"self.inner.store_put(self.inner.load::<#{IdempotencyTokenProvider}>().cloned().unwrap_or_else(#{default_provider}));",
"layer.store_put(layer.load::<#{IdempotencyTokenProvider}>().cloned().unwrap_or_else(#{default_provider}));",
*codegenScope,
)
} else {

View File

@ -321,8 +321,10 @@ class ServiceConfigGenerator(
"BoxError" to runtimeApi.resolve("client::runtime_plugin::BoxError"),
"CloneableLayer" to smithyTypes.resolve("config_bag::CloneableLayer"),
"ConfigBag" to smithyTypes.resolve("config_bag::ConfigBag"),
"ConfigBagAccessors" to runtimeApi.resolve("client::orchestrator::ConfigBagAccessors"),
"FrozenLayer" to smithyTypes.resolve("config_bag::FrozenLayer"),
"InterceptorRegistrar" to runtimeApi.resolve("client::interceptors::InterceptorRegistrar"),
"Layer" to smithyTypes.resolve("config_bag::Layer"),
"RuntimePlugin" to runtimeApi.resolve("client::runtime_plugin::RuntimePlugin"),
*preludeScope,
)
@ -426,6 +428,17 @@ class ServiceConfigGenerator(
if (runtimeMode.defaultToOrchestrator) {
rust("##[allow(unused_mut)]")
rustBlock("pub fn build(mut self) -> Config") {
rustTemplate(
"""
use #{ConfigBagAccessors};
// The builder is being turned into a service config. While doing so, we'd like to avoid
// requiring that items created and stored _during_ the build method be `Clone`, since they
// will soon be part of a `FrozenLayer` owned by the service config. So we will convert the
// current `CloneableLayer` into a `Layer` that does not impose the `Clone` requirement.
let mut layer: #{Layer} = self.inner.into();
""",
*codegenScope,
)
customizations.forEach {
it.section(ServiceConfig.BuilderBuild)(this)
}
@ -433,7 +446,7 @@ class ServiceConfigGenerator(
customizations.forEach {
it.section(ServiceConfig.BuilderBuildExtras)(this)
}
rust("inner: self.inner.freeze(),")
rust("inner: layer.freeze(),")
}
}
} else {
@ -451,6 +464,25 @@ class ServiceConfigGenerator(
}
}
fun renderRuntimePluginImplForSelf(writer: RustWriter) {
writer.rustTemplate(
"""
impl #{RuntimePlugin} for Config {
fn config(&self) -> #{Option}<#{FrozenLayer}> {
#{Some}(self.inner.clone())
}
fn interceptors(&self, interceptors: &mut #{InterceptorRegistrar}) {
interceptors.extend(self.interceptors.iter().cloned());
}
}
""",
*codegenScope,
"config" to writable { writeCustomizations(customizations, ServiceConfig.RuntimePluginConfig("cfg")) },
)
}
fun renderRuntimePluginImplForBuilder(writer: RustWriter) {
writer.rustTemplate(
"""
@ -460,7 +492,7 @@ class ServiceConfigGenerator(
##[allow(unused_mut)]
let mut cfg = #{CloneableLayer}::new("service config");
#{config}
Some(cfg.freeze())
#{Some}(cfg.freeze())
}
fn interceptors(&self, _interceptors: &mut #{InterceptorRegistrar}) {

View File

@ -12,7 +12,7 @@ use aws_smithy_runtime_api::client::interceptors::InterceptorContext;
use aws_smithy_runtime_api::client::orchestrator::{
BoxError, ConfigBagAccessors, EndpointResolver, EndpointResolverParams, HttpRequest,
};
use aws_smithy_types::config_bag::ConfigBag;
use aws_smithy_types::config_bag::{ConfigBag, Storable, StoreReplace};
use aws_smithy_types::endpoint::Endpoint;
use http::header::HeaderName;
use http::{HeaderValue, Uri};
@ -65,6 +65,13 @@ pub struct DefaultEndpointResolver<Params> {
inner: SharedEndpointResolver<Params>,
}
impl<Params> Storable for DefaultEndpointResolver<Params>
where
Params: Debug + Send + Sync + 'static,
{
type Storer = StoreReplace<Self>;
}
impl<Params> DefaultEndpointResolver<Params> {
pub fn new(resolve_endpoint: SharedEndpointResolver<Params>) -> Self {
Self {

View File

@ -13,7 +13,7 @@ use aws_smithy_runtime_api::client::request_attempts::RequestAttempts;
use aws_smithy_runtime_api::client::retries::{
ClassifyRetry, RetryReason, RetryStrategy, ShouldAttempt,
};
use aws_smithy_types::config_bag::ConfigBag;
use aws_smithy_types::config_bag::{ConfigBag, Storable, StoreReplace};
use aws_smithy_types::retry::RetryConfig;
use std::sync::Mutex;
use std::time::Duration;
@ -32,6 +32,10 @@ pub struct StandardRetryStrategy {
retry_permit: Mutex<Option<OwnedSemaphorePermit>>,
}
impl Storable for StandardRetryStrategy {
type Storer = StoreReplace<Self>;
}
impl StandardRetryStrategy {
pub fn new(retry_config: &RetryConfig) -> Self {
// TODO(enableNewSmithyRuntimeLaunch) add support for `retry_config.reconnect_mode()` here or in the orchestrator flow.

View File

@ -113,6 +113,12 @@ impl Clone for CloneableLayer {
}
}
impl From<CloneableLayer> for Layer {
fn from(cloneable_layer: CloneableLayer) -> Layer {
cloneable_layer.0
}
}
// We need to "override" the mutable methods to encode the information that an item being stored
// implements `Clone`. For the immutable methods, they can just be delegated via the `Deref` trait.
impl CloneableLayer {