Skip to content

Commit

Permalink
Feature-gate public use of http-body and hyper within `aws-smithy…
Browse files Browse the repository at this point in the history
…-types` (#3088)

## Motivation and Context
Implements #3033

## Description
This PR hides behind cargo features the third-party types from
`http-body` and `hyper` crates that are used in`aws-smithy-types`'
public API. Customers need to opt-in by enabling a cargo feature
`http-body-0-4-x` in `aws-smithy-types` to create an `SdkBody` or
`ByteStream` using those third-party types. For more details, please see
[the upgrade
guide](#3089).

As can been seen from code changes, to reduce the surface area where we
need to feature-gate things, we have fused the
`aws_smithy_types::body::Inner::Streaming` enum variant into
`aws_smithy_types::body::Inner::Dyn` variant, thereby removing
`SdkBody::from_dyn`.

## Testing
Relied on existing tests in CI

## Checklist
- [x] I have updated `CHANGELOG.next.toml` if I made changes to the
smithy-rs codegen or runtime crates

----

_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: Zelda Hessler <[email protected]>
  • Loading branch information
ysaito1001 and Velfi authored Oct 25, 2023
1 parent 08a533f commit 2a51e0b
Show file tree
Hide file tree
Showing 28 changed files with 537 additions and 383 deletions.
12 changes: 12 additions & 0 deletions CHANGELOG.next.toml
Original file line number Diff line number Diff line change
Expand Up @@ -474,3 +474,15 @@ message = "Enable custom auth schemes to work by changing the code generated aut
references = ["smithy-rs#3034", "smithy-rs#3087"]
meta = { "breaking" = false, "tada" = false, "bug" = true, "target" = "client" }
author = "rcoh"

[[smithy-rs]]
message = "Publicly exposed types from `http-body` and `hyper` crates within `aws-smithy-types` are now feature-gated. See the [upgrade guidance](https://github.com/awslabs/smithy-rs/discussions/3089) for details."
references = ["smithy-rs#3033", "smithy-rs#3088"]
meta = { "breaking" = true, "tada" = false, "bug" = false, "target" = "all" }
author = "ysaito1001"

[[smithy-rs]]
message = "`ByteStream::poll_next` is now feature-gated. You can turn on a cargo feature `byte-stream-poll-next` in `aws-smithy-types` to use it."
references = ["smithy-rs#3033", "smithy-rs#3088"]
meta = { "breaking" = true, "tada" = false, "bug" = false, "target" = "all" }
author = "ysaito1001"
2 changes: 1 addition & 1 deletion aws/rust-runtime/aws-inlineable/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ aws-smithy-checksums = { path = "../../../rust-runtime/aws-smithy-checksums" }
aws-smithy-http = { path = "../../../rust-runtime/aws-smithy-http" }
aws-smithy-runtime = { path = "../../../rust-runtime/aws-smithy-runtime", features = ["client"] }
aws-smithy-runtime-api = { path = "../../../rust-runtime/aws-smithy-runtime-api", features = ["client"] }
aws-smithy-types = { path = "../../../rust-runtime/aws-smithy-types" }
aws-smithy-types = { path = "../../../rust-runtime/aws-smithy-types", features = ["http-body-0-4-x"] }
bytes = "1"
hex = "0.4.3"
http = "0.2.9"
Expand Down
6 changes: 3 additions & 3 deletions aws/rust-runtime/aws-inlineable/src/http_request_checksum.rs
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ use aws_smithy_runtime_api::client::interceptors::context::{
use aws_smithy_runtime_api::client::interceptors::Intercept;
use aws_smithy_runtime_api::client::orchestrator::HttpRequest;
use aws_smithy_runtime_api::client::runtime_components::RuntimeComponents;
use aws_smithy_types::body::{BoxBody, SdkBody};
use aws_smithy_types::body::SdkBody;
use aws_smithy_types::config_bag::{ConfigBag, Layer, Storable, StoreReplace};
use aws_smithy_types::error::operation::BuildError;
use http::HeaderValue;
Expand Down Expand Up @@ -173,7 +173,7 @@ fn wrap_streaming_request_body_in_checksum_calculating_body(

let body = AwsChunkedBody::new(body, aws_chunked_body_options);

SdkBody::from_dyn(BoxBody::new(body))
SdkBody::from_body_0_4(body)
})
};

Expand Down Expand Up @@ -269,7 +269,7 @@ mod tests {
let crc32c_checksum = crc32c_checksum.finalize();

let mut request = HttpRequest::new(
ByteStream::read_from()
ByteStream::read_with_body_0_4_from()
.path(&file)
.buffer_size(1024)
.build()
Expand Down
6 changes: 3 additions & 3 deletions aws/rust-runtime/aws-inlineable/src/http_response_checksum.rs
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ use aws_smithy_runtime_api::client::interceptors::context::{
};
use aws_smithy_runtime_api::client::interceptors::Intercept;
use aws_smithy_runtime_api::client::runtime_components::RuntimeComponents;
use aws_smithy_types::body::{BoxBody, SdkBody};
use aws_smithy_types::body::SdkBody;
use aws_smithy_types::config_bag::{ConfigBag, Layer, Storable, StoreReplace};
use http::HeaderValue;
use std::{fmt, mem};
Expand Down Expand Up @@ -119,11 +119,11 @@ pub(crate) fn wrap_body_with_checksum_validator(
use aws_smithy_checksums::body::validate;

body.map(move |body| {
SdkBody::from_dyn(BoxBody::new(validate::ChecksumBody::new(
SdkBody::from_body_0_4(validate::ChecksumBody::new(
body,
checksum_algorithm.into_impl(),
precalculated_checksum.clone(),
)))
))
})
}

Expand Down
6 changes: 5 additions & 1 deletion aws/sdk/integration-tests/glacier/tests/custom-headers.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,11 @@ async fn set_correct_headers() {
let _resp = client
.upload_archive()
.vault_name("vault")
.body(ByteStream::from_path("tests/test-file.txt").await.unwrap())
.body(
ByteStream::from_path_body_0_4("tests/test-file.txt")
.await
.unwrap(),
)
.send()
.await;
let req = handler.expect_request();
Expand Down
2 changes: 1 addition & 1 deletion aws/sdk/integration-tests/s3/tests/checksums.rs
Original file line number Diff line number Diff line change
Expand Up @@ -177,7 +177,7 @@ async fn test_checksum_on_streaming_request<'a>(
use std::io::Write;
file.write_all(body).unwrap();

let body = aws_sdk_s3::primitives::ByteStream::read_from()
let body = aws_sdk_s3::primitives::ByteStream::read_with_body_0_4_from()
.path(file.path())
.buffer_size(1024)
.build()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,12 +71,12 @@ class RequiredCustomizations : ClientCodegenDecorator {
override fun extras(codegenContext: ClientCodegenContext, rustCrate: RustCrate) {
val rc = codegenContext.runtimeConfig

// Add rt-tokio feature for `ByteStream::from_path`
// Add rt-tokio and http-body-0-4-x features for `ByteStream::from_path_0_4`
rustCrate.mergeFeature(
Feature(
"rt-tokio",
true,
listOf("aws-smithy-async/rt-tokio", "aws-smithy-http/rt-tokio"),
listOf("aws-smithy-async/rt-tokio", "aws-smithy-types/rt-tokio", "aws-smithy-types/http-body-0-4-x"),
),
)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,12 +29,12 @@ class ClientHttpBoundProtocolPayloadGenerator(
_cfg.interceptor_state().store_put(signer_sender);
let adapter: #{aws_smithy_http}::event_stream::MessageStreamAdapter<_, _> =
${params.outerName}.${params.memberName}.into_body_stream(marshaller, error_marshaller, signer);
let body: #{SdkBody} = #{hyper}::Body::wrap_stream(adapter).into();
body
#{SdkBody}::from_body_0_4(#{hyper}::Body::wrap_stream(adapter))
}
""",
"hyper" to CargoDependency.HyperWithStream.toType(),
"SdkBody" to RuntimeType.sdkBody(codegenContext.runtimeConfig),
"SdkBody" to CargoDependency.smithyTypes(codegenContext.runtimeConfig).withFeature("http-body-0-4-x")
.toType().resolve("body::SdkBody"),
"aws_smithy_http" to RuntimeType.smithyHttp(codegenContext.runtimeConfig),
"DeferredSigner" to RuntimeType.smithyEventStream(codegenContext.runtimeConfig).resolve("frame::DeferredSigner"),
"marshallerConstructorFn" to params.marshallerConstructorFn,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,8 +37,14 @@ class ServerRequiredCustomizations : ServerCodegenDecorator {
override fun extras(codegenContext: ServerCodegenContext, rustCrate: RustCrate) {
val rc = codegenContext.runtimeConfig

// Add rt-tokio feature for `ByteStream::from_path`
rustCrate.mergeFeature(Feature("rt-tokio", true, listOf("aws-smithy-http/rt-tokio")))
// Add rt-tokio and http-body-0-4-x features for `ByteStream::from_path_body_0_4`
rustCrate.mergeFeature(
Feature(
"rt-tokio",
true,
listOf("aws-smithy-types/rt-tokio", "aws-smithy-types/http-body-0-4-x"),
),
)

rustCrate.withModule(ServerRustModule.Types) {
pubUseSmithyPrimitives(codegenContext, codegenContext.model)(this)
Expand Down
2 changes: 1 addition & 1 deletion rust-runtime/aws-smithy-http-server-python/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ publish = true
aws-smithy-http = { path = "../aws-smithy-http" }
aws-smithy-http-server = { path = "../aws-smithy-http-server", features = ["aws-lambda"] }
aws-smithy-json = { path = "../aws-smithy-json" }
aws-smithy-types = { path = "../aws-smithy-types" }
aws-smithy-types = { path = "../aws-smithy-types", features = ["byte-stream-poll-next", "http-body-0-4-x"] }
aws-smithy-xml = { path = "../aws-smithy-xml" }
bytes = "1.2"
futures = "0.3"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -147,5 +147,5 @@ async def handler(bytestream):
fn streaming_bytestream_from_vec(chunks: Vec<&'static str>) -> ByteStream {
let stream = stream::iter(chunks.into_iter().map(Ok::<_, io::Error>));
let body = Body::wrap_stream(stream);
ByteStream::new(SdkBody::from(body))
ByteStream::new(SdkBody::from_body_0_4(body))
}
4 changes: 2 additions & 2 deletions rust-runtime/aws-smithy-http-server-python/src/types.rs
Original file line number Diff line number Diff line change
Expand Up @@ -407,7 +407,7 @@ impl ByteStream {
#[staticmethod]
pub fn from_path_blocking(py: Python, path: String) -> PyResult<Py<PyAny>> {
let byte_stream = Handle::current().block_on(async {
aws_smithy_types::byte_stream::ByteStream::from_path(path)
aws_smithy_types::byte_stream::ByteStream::from_path_body_0_4(path)
.await
.map_err(|e| PyRuntimeError::new_err(e.to_string()))
})?;
Expand All @@ -423,7 +423,7 @@ impl ByteStream {
#[staticmethod]
pub fn from_path(py: Python, path: String) -> PyResult<&PyAny> {
pyo3_asyncio::tokio::future_into_py(py, async move {
let byte_stream = aws_smithy_types::byte_stream::ByteStream::from_path(path)
let byte_stream = aws_smithy_types::byte_stream::ByteStream::from_path_body_0_4(path)
.await
.map_err(|e| PyRuntimeError::new_err(e.to_string()))?;
Ok(Self(Arc::new(Mutex::new(byte_stream))))
Expand Down
2 changes: 1 addition & 1 deletion rust-runtime/aws-smithy-http-server/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ request-id = ["dep:uuid"]
async-trait = "0.1"
aws-smithy-http = { path = "../aws-smithy-http", features = ["rt-tokio"] }
aws-smithy-json = { path = "../aws-smithy-json" }
aws-smithy-types = { path = "../aws-smithy-types" }
aws-smithy-types = { path = "../aws-smithy-types", features = ["http-body-0-4-x", "hyper-0-14-x"] }
aws-smithy-xml = { path = "../aws-smithy-xml" }
bytes = "1.1"
futures-util = { version = "0.3.16", default-features = false }
Expand Down
2 changes: 1 addition & 1 deletion rust-runtime/aws-smithy-http/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ rt-tokio = ["aws-smithy-types/rt-tokio"]

[dependencies]
aws-smithy-eventstream = { path = "../aws-smithy-eventstream", optional = true }
aws-smithy-types = { path = "../aws-smithy-types" }
aws-smithy-types = { path = "../aws-smithy-types", features = ["byte-stream-poll-next", "http-body-0-4-x"] }
bytes = "1"
bytes-utils = "0.1"
http = "0.2.3"
Expand Down
4 changes: 0 additions & 4 deletions rust-runtime/aws-smithy-http/src/body.rs
Original file line number Diff line number Diff line change
Expand Up @@ -8,10 +8,6 @@

//! Types for representing the body of an HTTP request or response

/// A boxed generic HTTP body that, when consumed, will result in [`Bytes`](bytes::Bytes) or an [`Error`](aws_smithy_types::body::Error).
#[deprecated(note = "Moved to `aws_smithy_types::body::BoxBody`.")]
pub type BoxBody = aws_smithy_types::body::BoxBody;

/// A generic, boxed error that's `Send` and `Sync`
#[deprecated(note = "`Moved to `aws_smithy_types::body::Error`.")]
pub type Error = aws_smithy_types::body::Error;
Expand Down
18 changes: 9 additions & 9 deletions rust-runtime/aws-smithy-http/src/event_stream/receiver.rs
Original file line number Diff line number Diff line change
Expand Up @@ -342,7 +342,7 @@ mod tests {
let chunks: Vec<Result<_, IOError>> =
vec![Ok(encode_message("one")), Ok(encode_message("two"))];
let chunk_stream = futures_util::stream::iter(chunks);
let body = SdkBody::from(Body::wrap_stream(chunk_stream));
let body = SdkBody::from_body_0_4(Body::wrap_stream(chunk_stream));
let mut receiver = Receiver::<TestMessage, EventStreamError>::new(Unmarshaller, body);
assert_eq!(
TestMessage("one".into()),
Expand All @@ -363,7 +363,7 @@ mod tests {
Ok(Bytes::from_static(&[])),
];
let chunk_stream = futures_util::stream::iter(chunks);
let body = SdkBody::from(Body::wrap_stream(chunk_stream));
let body = SdkBody::from_body_0_4(Body::wrap_stream(chunk_stream));
let mut receiver = Receiver::<TestMessage, EventStreamError>::new(Unmarshaller, body);
assert_eq!(
TestMessage("one".into()),
Expand All @@ -384,7 +384,7 @@ mod tests {
Ok(encode_message("three").split_to(10)),
];
let chunk_stream = futures_util::stream::iter(chunks);
let body = SdkBody::from(Body::wrap_stream(chunk_stream));
let body = SdkBody::from_body_0_4(Body::wrap_stream(chunk_stream));
let mut receiver = Receiver::<TestMessage, EventStreamError>::new(Unmarshaller, body);
assert_eq!(
TestMessage("one".into()),
Expand All @@ -410,7 +410,7 @@ mod tests {
)),
];
let chunk_stream = futures_util::stream::iter(chunks);
let body = SdkBody::from(Body::wrap_stream(chunk_stream));
let body = SdkBody::from_body_0_4(Body::wrap_stream(chunk_stream));
let mut receiver = Receiver::<TestMessage, EventStreamError>::new(Unmarshaller, body);
assert_eq!(
TestMessage("one".into()),
Expand Down Expand Up @@ -463,7 +463,7 @@ mod tests {
];

let chunk_stream = futures_util::stream::iter(chunks);
let body = SdkBody::from(Body::wrap_stream(chunk_stream));
let body = SdkBody::from_body_0_4(Body::wrap_stream(chunk_stream));
let mut receiver = Receiver::<TestMessage, EventStreamError>::new(Unmarshaller, body);
for payload in &["one", "two", "three", "four", "five", "six", "seven", "eight"] {
assert_eq!(
Expand All @@ -483,7 +483,7 @@ mod tests {
Err(IOError::new(ErrorKind::ConnectionReset, FakeError)),
];
let chunk_stream = futures_util::stream::iter(chunks);
let body = SdkBody::from(Body::wrap_stream(chunk_stream));
let body = SdkBody::from_body_0_4(Body::wrap_stream(chunk_stream));
let mut receiver = Receiver::<TestMessage, EventStreamError>::new(Unmarshaller, body);
assert_eq!(
TestMessage("one".into()),
Expand All @@ -504,7 +504,7 @@ mod tests {
Ok(Bytes::from_static(&[0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0])),
];
let chunk_stream = futures_util::stream::iter(chunks);
let body = SdkBody::from(Body::wrap_stream(chunk_stream));
let body = SdkBody::from_body_0_4(Body::wrap_stream(chunk_stream));
let mut receiver = Receiver::<TestMessage, EventStreamError>::new(Unmarshaller, body);
assert_eq!(
TestMessage("one".into()),
Expand All @@ -521,7 +521,7 @@ mod tests {
let chunks: Vec<Result<_, IOError>> =
vec![Ok(encode_initial_response()), Ok(encode_message("one"))];
let chunk_stream = futures_util::stream::iter(chunks);
let body = SdkBody::from(Body::wrap_stream(chunk_stream));
let body = SdkBody::from_body_0_4(Body::wrap_stream(chunk_stream));
let mut receiver = Receiver::<TestMessage, EventStreamError>::new(Unmarshaller, body);
assert!(receiver.try_recv_initial().await.unwrap().is_some());
assert_eq!(
Expand All @@ -535,7 +535,7 @@ mod tests {
let chunks: Vec<Result<_, IOError>> =
vec![Ok(encode_message("one")), Ok(encode_message("two"))];
let chunk_stream = futures_util::stream::iter(chunks);
let body = SdkBody::from(Body::wrap_stream(chunk_stream));
let body = SdkBody::from_body_0_4(Body::wrap_stream(chunk_stream));
let mut receiver = Receiver::<TestMessage, EventStreamError>::new(Unmarshaller, body);
assert!(receiver.try_recv_initial().await.unwrap().is_none());
assert_eq!(
Expand Down
2 changes: 1 addition & 1 deletion rust-runtime/aws-smithy-runtime/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ aws-smithy-async = { path = "../aws-smithy-async" }
aws-smithy-http = { path = "../aws-smithy-http" }
aws-smithy-protocol-test = { path = "../aws-smithy-protocol-test", optional = true }
aws-smithy-runtime-api = { path = "../aws-smithy-runtime-api" }
aws-smithy-types = { path = "../aws-smithy-types" }
aws-smithy-types = { path = "../aws-smithy-types", features = ["http-body-0-4-x"] }
bytes = "1"
fastrand = "2.0.0"
http = "0.2.8"
Expand Down
5 changes: 4 additions & 1 deletion rust-runtime/aws-smithy-runtime/src/client/http/hyper_014.rs
Original file line number Diff line number Diff line change
Expand Up @@ -358,7 +358,10 @@ where
let mut client = self.client.clone();
let fut = client.call(request);
HttpConnectorFuture::new(async move {
Ok(fut.await.map_err(downcast_error)?.map(SdkBody::from))
Ok(fut
.await
.map_err(downcast_error)?
.map(SdkBody::from_body_0_4))
})
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ fn record_body(
event_bus: Arc<Mutex<Vec<Event>>>,
) -> JoinHandle<()> {
let (sender, output_body) = hyper::Body::channel();
let real_body = std::mem::replace(body, SdkBody::from(output_body));
let real_body = std::mem::replace(body, SdkBody::from_body_0_4(output_body));
tokio::spawn(async move {
let mut real_body = real_body;
let mut sender = sender;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -294,7 +294,7 @@ impl HttpConnector for ReplayingClient {

let _initial_request = events.pop_front().unwrap();
let (sender, response_body) = hyper::Body::channel();
let body = SdkBody::from(response_body);
let body = SdkBody::from_body_0_4(response_body);
let recording = self.recorded_requests.clone();
let recorded_request = tokio::spawn(async move {
let mut data_read = vec![];
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ use aws_smithy_runtime::{ev, match_events};
use aws_smithy_runtime_api::client::interceptors::context::InterceptorContext;
use aws_smithy_runtime_api::client::orchestrator::OrchestratorError;
use aws_smithy_runtime_api::client::retries::classifiers::{ClassifyRetry, RetryAction};
use aws_smithy_types::body::{BoxBody, SdkBody};
use aws_smithy_types::body::SdkBody;
use aws_smithy_types::retry::{ErrorKind, ProvideErrorKind, ReconnectMode, RetryConfig};
use aws_smithy_types::timeout::TimeoutConfig;
use hyper::client::Builder as HyperBuilder;
Expand Down Expand Up @@ -150,7 +150,7 @@ async fn wire_level_test(
let request = http::Request::builder()
.uri(endpoint_url.clone())
// Make the body non-replayable since we don't actually want to retry
.body(SdkBody::from_dyn(BoxBody::new(SdkBody::from("body"))))
.body(SdkBody::from_body_0_4(SdkBody::from("body")))
.unwrap()
.try_into()
.unwrap();
Expand Down
7 changes: 5 additions & 2 deletions rust-runtime/aws-smithy-types/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,9 @@ license = "Apache-2.0"
repository = "https://github.com/awslabs/smithy-rs"

[features]
byte-stream-poll-next = []
http-body-0-4-x = ["dep:http-body-0-4"]
hyper-0-14-x = ["dep:hyper-0-14"]
rt-tokio = ["dep:tokio-util", "dep:tokio", "tokio?/rt", "tokio?/fs", "tokio?/io-util", "tokio-util?/io"]
test-util = []
serde-serialize = []
Expand All @@ -21,8 +24,8 @@ base64-simd = "0.8"
bytes = "1"
bytes-utils = "0.1"
http = "0.2.3"
http-body = "0.4.4"
hyper = "0.14.26"
http-body-0-4 = { package = "http-body", version = "0.4.4", optional = true }
hyper-0-14 = { package = "hyper", version = "0.14.26", optional = true }
itoa = "1.0.0"
num-integer = "0.1.44"
pin-project-lite = "0.2.9"
Expand Down
7 changes: 3 additions & 4 deletions rust-runtime/aws-smithy-types/external-types.toml
Original file line number Diff line number Diff line change
Expand Up @@ -2,14 +2,13 @@ allowed_external_types = [
"bytes::bytes::Bytes",
"bytes::buf::buf_impl::Buf",

# TODO(https://github.com/awslabs/smithy-rs/issues/3033): Feature gate based on unstable versions
# TODO(https://github.com/awslabs/smithy-rs/issues/2412): Support cargo-features for cargo-check-external-types
"http_body::Body",
"http_body::combinators::box_body::BoxBody",
"hyper::body::body::Body",

# TODO(https://github.com/awslabs/smithy-rs/issues/1193): Feature gate Tokio `AsyncRead`
# TODO(https://github.com/awslabs/smithy-rs/issues/2412): Support cargo-features for cargo-check-external-types
"tokio::io::async_read::AsyncRead",

# TODO(https://github.com/awslabs/smithy-rs/issues/1193): Feature gate references to Tokio `File`
# TODO(https://github.com/awslabs/smithy-rs/issues/2412): Support cargo-features for cargo-check-external-types
"tokio::fs::file::File",
]
Loading

0 comments on commit 2a51e0b

Please sign in to comment.