You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/07/27 21:20:57 UTC

[GitHub] [arrow-rs] alamb opened a new pull request, #2204: Add Builder style config objects

alamb opened a new pull request, #2204:
URL: https://github.com/apache/arrow-rs/pull/2204

   # Which issue does this PR close?
   
   Closes https://github.com/apache/arrow-rs/issues/2203
   
   # Rationale for this change
    
   Current config is both a pain to use as well as will require backwards incompatible changes to extend (e.g. add a new parameter)
   
   # What changes are included in this PR?
   Add builder style configs:
   - [ ] `AmazonS3Config`
   - [ ] `AzureConfig` (TODO check name)
   - [ ] `GcsConfig`  (TODO check name)
   
   # Are there any user-facing changes?
   Yes, the configuration system is better
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] alamb commented on a diff in pull request #2204: Add Builder style config objects

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2204:
URL: https://github.com/apache/arrow-rs/pull/2204#discussion_r932153362


##########
object_store/src/azure.rs:
##########
@@ -570,73 +579,126 @@ fn url_from_env(env_name: &str, default_url: &str) -> Result<Url> {
     Ok(url)
 }
 
-/// Configure a connection to container with given name on Microsoft Azure
-/// Blob store.
+/// Configure a connection to Mirosoft Azure Blob Storage bucket using
+/// the specified credentials.
 ///
-/// The credentials `account` and `access_key` must provide access to the
-/// store.
-pub fn new_azure(
-    account: impl Into<String>,
-    access_key: impl Into<String>,
-    container_name: impl Into<String>,
+/// # Example
+/// ```
+/// # let ACCOUNT = "foo";
+/// # let BUCKET_NAME = "foo";
+/// # let ACCESS_KEY = "foo";
+/// # let CONTAINER_NAME = "foo";
+/// let azure = object_store::azure::MicrosoftAzureBuilder::new()
+///  .with_account(ACCOUNT)
+///  .with_access_key(ACCESS_KEY)
+///  .with_container_name(CONTAINER_NAME)
+///  .build();
+/// ```
+#[derive(Debug, Default)]
+pub struct MicrosoftAzureBuilder {
+    account: Option<String>,
+    access_key: Option<String>,
+    container_name: Option<String>,

Review Comment:
   not sure if I should also call this 'bucket_name` (generic term) or `container_name` (Azure specific term). I kept the existing naming convention



##########
object_store/src/aws.rs:
##########
@@ -1067,17 +1185,9 @@ mod tests {
 
     const NON_EXISTENT_NAME: &str = "nonexistentname";
 
-    #[derive(Debug)]
-    struct AwsConfig {

Review Comment:
   Amusingly (at least to me) the tests had created a partial config struct anyways -- so this PR can be thought of "exposing these configuration options to uers" 😆 



##########
object_store/src/aws.rs:
##########
@@ -584,100 +592,210 @@ fn convert_object_meta(object: rusoto_s3::Object, bucket: &str) -> Result<Object
 
 /// Configure a connection to Amazon S3 using the specified credentials in
 /// the specified Amazon region and bucket.
-#[allow(clippy::too_many_arguments)]
-pub fn new_s3(
-    access_key_id: Option<impl Into<String>>,
-    secret_access_key: Option<impl Into<String>>,
-    region: impl Into<String>,
-    bucket_name: impl Into<String>,
-    endpoint: Option<impl Into<String>>,
-    session_token: Option<impl Into<String>>,
+///
+/// # Example
+/// ```
+/// # let REGION = "foo";
+/// # let BUCKET_NAME = "foo";
+/// # let ACCESS_KEY_ID = "foo";
+/// # let SECRET_KEY = "foo";
+/// let s3 = object_store::aws::AmazonS3Builder::new()
+///  .with_region(REGION)
+///  .with_bucket_name(BUCKET_NAME)
+///  .with_access_key_id(ACCESS_KEY_ID)
+///  .with_secret_access_key(SECRET_KEY)
+///  .build();
+/// ```
+#[derive(Debug)]
+pub struct AmazonS3Builder {
+    access_key_id: Option<String>,
+    secret_access_key: Option<String>,
+    region: Option<String>,
+    bucket_name: Option<String>,
+    endpoint: Option<String>,
+    token: Option<String>,
     max_connections: NonZeroUsize,
     allow_http: bool,
-) -> Result<AmazonS3> {
-    let region = region.into();
-    let region: rusoto_core::Region = match endpoint {
-        None => region.parse().context(InvalidRegionSnafu { region })?,
-        Some(endpoint) => rusoto_core::Region::Custom {
-            name: region,
-            endpoint: endpoint.into(),
-        },
-    };
+}
 
-    let mut builder = HyperBuilder::default();
-    builder.pool_max_idle_per_host(max_connections.get());
-
-    let connector = if allow_http {
-        hyper_rustls::HttpsConnectorBuilder::new()
-            .with_webpki_roots()
-            .https_or_http()
-            .enable_http1()
-            .enable_http2()
-            .build()
-    } else {
-        hyper_rustls::HttpsConnectorBuilder::new()
-            .with_webpki_roots()
-            .https_only()
-            .enable_http1()
-            .enable_http2()
-            .build()
-    };
+impl Default for AmazonS3Builder {
+    fn default() -> Self {
+        Self {
+            access_key_id: None,
+            secret_access_key: None,
+            region: None,
+            bucket_name: None,
+            endpoint: None,
+            token: None,
+            max_connections: NonZeroUsize::new(16).unwrap(),
+            allow_http: false,
+        }
+    }
+}
 
-    let http_client = rusoto_core::request::HttpClient::from_builder(builder, connector);
+impl AmazonS3Builder {
+    /// Create a new [`AmazonS3Builder`] with default values.
+    pub fn new() -> Self {
+        Default::default()
+    }
 
-    let client = match (access_key_id, secret_access_key, session_token) {
-        (Some(access_key_id), Some(secret_access_key), Some(session_token)) => {
-            let credentials_provider = StaticProvider::new(
-                access_key_id.into(),
-                secret_access_key.into(),
-                Some(session_token.into()),
-                None,
-            );
-            rusoto_s3::S3Client::new_with(http_client, credentials_provider, region)
-        }
-        (Some(access_key_id), Some(secret_access_key), None) => {
-            let credentials_provider = StaticProvider::new_minimal(
-                access_key_id.into(),
-                secret_access_key.into(),
-            );
-            rusoto_s3::S3Client::new_with(http_client, credentials_provider, region)
-        }
-        (None, Some(_), _) => return Err(Error::MissingAccessKey.into()),
-        (Some(_), None, _) => return Err(Error::MissingSecretAccessKey.into()),
-        _ if std::env::var_os("AWS_WEB_IDENTITY_TOKEN_FILE").is_some() => {
-            rusoto_s3::S3Client::new_with(
+    /// Set the AWS Access Key (required)
+    pub fn with_access_key_id(mut self, access_key_id: impl Into<String>) -> Self {
+        self.access_key_id = Some(access_key_id.into());
+        self
+    }
+
+    /// Set the AWS Secret Access Key (required)
+    pub fn with_secret_access_key(
+        mut self,
+        secret_access_key: impl Into<String>,
+    ) -> Self {
+        self.secret_access_key = Some(secret_access_key.into());
+        self
+    }
+
+    /// Set the region (e.g. `us-east-1`) (required)
+    pub fn with_region(mut self, region: impl Into<String>) -> Self {
+        self.region = Some(region.into());
+        self
+    }
+
+    /// Set the bucket_name (required)
+    pub fn with_bucket_name(mut self, bucket_name: impl Into<String>) -> Self {
+        self.bucket_name = Some(bucket_name.into());
+        self
+    }
+
+    /// Sets the endpoint for communicating with AWS S3. Default value
+    /// is based on region.
+    ///
+    /// For example, this might be set to `"http://localhost:4566:`
+    /// for testing against a localstack instance.
+    pub fn with_endpoint(mut self, endpoint: impl Into<String>) -> Self {
+        self.endpoint = Some(endpoint.into());
+        self
+    }
+
+    /// Set the token to use for requests (passed to underlying provider)
+    pub fn with_token(mut self, token: impl Into<String>) -> Self {
+        self.token = Some(token.into());
+        self
+    }
+
+    /// Sets the maximum number of concurrent outstanding
+    /// connectons. Default is `16`.
+    pub fn with_max_connections(mut self, max_connections: NonZeroUsize) -> Self {
+        self.max_connections = max_connections;
+        self
+    }
+
+    /// Sets what protocol is allowed. If `allow_http` is :
+    /// * false (default):  Only HTTPS are allowed
+    /// * true:  HTTP and HTTPS are allowed
+    pub fn with_allow_http(mut self, allow_http: bool) -> Self {
+        self.allow_http = allow_http;
+        self
+    }
+
+    /// Create a [`AmazonS3`] instance from the provided values,
+    /// consuming `self`.
+    pub fn build(self) -> Result<AmazonS3> {
+        let Self {
+            access_key_id,
+            secret_access_key,
+            region,
+            bucket_name,
+            endpoint,
+            token,
+            max_connections,
+            allow_http,
+        } = self;
+
+        let region = region.ok_or(Error::MissingRegion {})?;
+        let bucket_name = bucket_name.ok_or(Error::MissingBucketName {})?;
+
+        let region: rusoto_core::Region = match endpoint {
+            None => region.parse().context(InvalidRegionSnafu { region })?,
+            Some(endpoint) => rusoto_core::Region::Custom {
+                name: region,
+                endpoint,
+            },
+        };
+
+        let mut builder = HyperBuilder::default();
+        builder.pool_max_idle_per_host(max_connections.get());
+
+        let connector = if allow_http {
+            hyper_rustls::HttpsConnectorBuilder::new()
+                .with_webpki_roots()
+                .https_or_http()
+                .enable_http1()
+                .enable_http2()
+                .build()
+        } else {
+            hyper_rustls::HttpsConnectorBuilder::new()
+                .with_webpki_roots()
+                .https_only()
+                .enable_http1()
+                .enable_http2()
+                .build()
+        };
+
+        let http_client =
+            rusoto_core::request::HttpClient::from_builder(builder, connector);
+
+        let client = match (access_key_id, secret_access_key, token) {
+            (Some(access_key_id), Some(secret_access_key), Some(token)) => {
+                let credentials_provider = StaticProvider::new(
+                    access_key_id,
+                    secret_access_key,
+                    Some(token),
+                    None,
+                );
+                rusoto_s3::S3Client::new_with(http_client, credentials_provider, region)
+            }
+            (Some(access_key_id), Some(secret_access_key), None) => {
+                let credentials_provider =
+                    StaticProvider::new_minimal(access_key_id, secret_access_key);
+                rusoto_s3::S3Client::new_with(http_client, credentials_provider, region)
+            }
+            (None, Some(_), _) => return Err(Error::MissingAccessKey.into()),
+            (Some(_), None, _) => return Err(Error::MissingSecretAccessKey.into()),
+            _ if std::env::var_os("AWS_WEB_IDENTITY_TOKEN_FILE").is_some() => {
+                rusoto_s3::S3Client::new_with(
+                    http_client,
+                    WebIdentityProvider::from_k8s_env(),
+                    region,
+                )
+            }
+            _ => rusoto_s3::S3Client::new_with(
                 http_client,
-                WebIdentityProvider::from_k8s_env(),
+                InstanceMetadataProvider::new(),
                 region,
-            )
-        }
-        _ => rusoto_s3::S3Client::new_with(
-            http_client,
-            InstanceMetadataProvider::new(),
-            region,
-        ),
-    };
+            ),
+        };
 
-    Ok(AmazonS3 {
-        client_unrestricted: client,
-        connection_semaphore: Arc::new(Semaphore::new(max_connections.get())),
-        bucket_name: bucket_name.into(),
-    })
+        Ok(AmazonS3 {
+            client_unrestricted: client,
+            connection_semaphore: Arc::new(Semaphore::new(max_connections.get())),
+            bucket_name,
+        })
+    }
 }
 
-/// Create a new [`AmazonS3`] that always errors
-pub fn new_failing_s3() -> Result<AmazonS3> {
-    new_s3(
-        Some("foo"),
-        Some("bar"),
-        "us-east-1",
-        "bucket",
-        None as Option<&str>,
-        None as Option<&str>,
-        NonZeroUsize::new(16).unwrap(),
-        true,
-    )
-}
+// /// Create a new [`AmazonS3`] that always errors

Review Comment:
   I am not sure what this is used for -- I think perhaps we should remove it. Maybe @tustvold or @carols10cents 
    remembers what it is for ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] tustvold commented on a diff in pull request #2204: Add Builder style config objects

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2204:
URL: https://github.com/apache/arrow-rs/pull/2204#discussion_r933370682


##########
object_store/src/gcp.rs:
##########
@@ -779,55 +785,116 @@ fn reader_credentials_file(
     Ok(serde_json::from_reader(reader).context(DecodeCredentialsSnafu)?)
 }
 
-/// Configure a connection to Google Cloud Storage.
-pub fn new_gcs(
-    service_account_path: impl AsRef<std::path::Path>,
-    bucket_name: impl Into<String>,
-) -> Result<GoogleCloudStorage> {
-    new_gcs_with_client(service_account_path, bucket_name, Client::new())
+/// Configure a connection to Google Cloud Storage using the specified
+/// credentials.
+///
+/// # Example
+/// ```
+/// # let BUCKET_NAME = "foo";
+/// # let SERVICE_ACCOUNT_PATH = "/tmp/foo.json";
+/// let gcs = object_store::gcp::GoogleCloudStorageBuilder::new()
+///  .with_service_account_path(SERVICE_ACCOUNT_PATH)
+///  .with_bucket_name(BUCKET_NAME)
+///  .build();
+/// ```
+#[derive(Debug, Default)]
+pub struct GoogleCloudStorageBuilder {
+    bucket_name: Option<String>,
+    service_account_path: Option<String>,
+    client: Option<Client>,
 }
 
-/// Configure a connection to Google Cloud Storage with the specified HTTP client.
-pub fn new_gcs_with_client(
-    service_account_path: impl AsRef<std::path::Path>,
-    bucket_name: impl Into<String>,
-    client: Client,
-) -> Result<GoogleCloudStorage> {
-    let credentials = reader_credentials_file(service_account_path)?;
-
-    // TODO: https://cloud.google.com/storage/docs/authentication#oauth-scopes
-    let scope = "https://www.googleapis.com/auth/devstorage.full_control";
-    let audience = "https://www.googleapis.com/oauth2/v4/token".to_string();
-
-    let oauth_provider = (!credentials.disable_oauth)
-        .then(|| {
-            OAuthProvider::new(
-                credentials.client_email,
-                credentials.private_key,
-                scope.to_string(),
-                audience,
-            )
-        })
-        .transpose()?;
+impl GoogleCloudStorageBuilder {
+    /// Create a new [`GoogleCloudStorageBuilder`] with default values.
+    pub fn new() -> Self {
+        Default::default()
+    }
 
-    let bucket_name = bucket_name.into();
-    let encoded_bucket_name =
-        percent_encode(bucket_name.as_bytes(), NON_ALPHANUMERIC).to_string();
+    /// Set the bucket name (required)
+    pub fn with_bucket_name(mut self, bucket_name: impl Into<String>) -> Self {
+        self.bucket_name = Some(bucket_name.into());
+        self
+    }
 
-    // The cloud storage crate currently only supports authentication via
-    // environment variables. Set the environment variable explicitly so
-    // that we can optionally accept command line arguments instead.
-    Ok(GoogleCloudStorage {
-        client: Arc::new(GoogleCloudStorageClient {
-            client,
-            base_url: credentials.gcs_base_url,
-            oauth_provider,
-            token_cache: Default::default(),
+    /// Set the path to the service account file (required). Example
+    /// `"/tmp/gcs.json"`
+    ///
+    /// Example contents of `gcs.json`:
+    ///
+    /// ```json
+    /// {
+    ///    "gcs_base_url": "https://localhost:4443",
+    ///    "disable_oauth": true,
+    ///    "client_email": "",
+    ///    "private_key": ""
+    /// }
+    /// ```
+    pub fn with_service_account_path(
+        mut self,
+        service_account_path: impl Into<String>,
+    ) -> Self {
+        self.service_account_path = Some(service_account_path.into());
+        self
+    }
+
+    /// Use the specified http [`Client`] (defaults to [`Client::new`])
+    ///
+    /// This allows you to set custom client options such as allowing
+    /// non secure connections or custom headers.
+    pub fn with_client(mut self, client: Client) -> Self {

Review Comment:
   I wonder if we should keep this `pub(crate)` at least for now, in case we want to change the client down the line or something...



##########
object_store/src/aws.rs:
##########
@@ -584,100 +592,210 @@ fn convert_object_meta(object: rusoto_s3::Object, bucket: &str) -> Result<Object
 
 /// Configure a connection to Amazon S3 using the specified credentials in
 /// the specified Amazon region and bucket.
-#[allow(clippy::too_many_arguments)]
-pub fn new_s3(
-    access_key_id: Option<impl Into<String>>,
-    secret_access_key: Option<impl Into<String>>,
-    region: impl Into<String>,
-    bucket_name: impl Into<String>,
-    endpoint: Option<impl Into<String>>,
-    session_token: Option<impl Into<String>>,
+///
+/// # Example
+/// ```
+/// # let REGION = "foo";
+/// # let BUCKET_NAME = "foo";
+/// # let ACCESS_KEY_ID = "foo";
+/// # let SECRET_KEY = "foo";
+/// let s3 = object_store::aws::AmazonS3Builder::new()
+///  .with_region(REGION)
+///  .with_bucket_name(BUCKET_NAME)
+///  .with_access_key_id(ACCESS_KEY_ID)
+///  .with_secret_access_key(SECRET_KEY)
+///  .build();
+/// ```
+#[derive(Debug)]
+pub struct AmazonS3Builder {
+    access_key_id: Option<String>,
+    secret_access_key: Option<String>,
+    region: Option<String>,
+    bucket_name: Option<String>,
+    endpoint: Option<String>,
+    token: Option<String>,
     max_connections: NonZeroUsize,
     allow_http: bool,
-) -> Result<AmazonS3> {
-    let region = region.into();
-    let region: rusoto_core::Region = match endpoint {
-        None => region.parse().context(InvalidRegionSnafu { region })?,
-        Some(endpoint) => rusoto_core::Region::Custom {
-            name: region,
-            endpoint: endpoint.into(),
-        },
-    };
+}
 
-    let mut builder = HyperBuilder::default();
-    builder.pool_max_idle_per_host(max_connections.get());
-
-    let connector = if allow_http {
-        hyper_rustls::HttpsConnectorBuilder::new()
-            .with_webpki_roots()
-            .https_or_http()
-            .enable_http1()
-            .enable_http2()
-            .build()
-    } else {
-        hyper_rustls::HttpsConnectorBuilder::new()
-            .with_webpki_roots()
-            .https_only()
-            .enable_http1()
-            .enable_http2()
-            .build()
-    };
+impl Default for AmazonS3Builder {
+    fn default() -> Self {
+        Self {
+            access_key_id: None,
+            secret_access_key: None,
+            region: None,
+            bucket_name: None,
+            endpoint: None,
+            token: None,
+            max_connections: NonZeroUsize::new(16).unwrap(),
+            allow_http: false,
+        }
+    }
+}
 
-    let http_client = rusoto_core::request::HttpClient::from_builder(builder, connector);
+impl AmazonS3Builder {
+    /// Create a new [`AmazonS3Builder`] with default values.
+    pub fn new() -> Self {
+        Default::default()
+    }
 
-    let client = match (access_key_id, secret_access_key, session_token) {
-        (Some(access_key_id), Some(secret_access_key), Some(session_token)) => {
-            let credentials_provider = StaticProvider::new(
-                access_key_id.into(),
-                secret_access_key.into(),
-                Some(session_token.into()),
-                None,
-            );
-            rusoto_s3::S3Client::new_with(http_client, credentials_provider, region)
-        }
-        (Some(access_key_id), Some(secret_access_key), None) => {
-            let credentials_provider = StaticProvider::new_minimal(
-                access_key_id.into(),
-                secret_access_key.into(),
-            );
-            rusoto_s3::S3Client::new_with(http_client, credentials_provider, region)
-        }
-        (None, Some(_), _) => return Err(Error::MissingAccessKey.into()),
-        (Some(_), None, _) => return Err(Error::MissingSecretAccessKey.into()),
-        _ if std::env::var_os("AWS_WEB_IDENTITY_TOKEN_FILE").is_some() => {
-            rusoto_s3::S3Client::new_with(
+    /// Set the AWS Access Key (required)
+    pub fn with_access_key_id(mut self, access_key_id: impl Into<String>) -> Self {
+        self.access_key_id = Some(access_key_id.into());
+        self
+    }
+
+    /// Set the AWS Secret Access Key (required)
+    pub fn with_secret_access_key(
+        mut self,
+        secret_access_key: impl Into<String>,
+    ) -> Self {
+        self.secret_access_key = Some(secret_access_key.into());
+        self
+    }
+
+    /// Set the region (e.g. `us-east-1`) (required)
+    pub fn with_region(mut self, region: impl Into<String>) -> Self {
+        self.region = Some(region.into());
+        self
+    }
+
+    /// Set the bucket_name (required)
+    pub fn with_bucket_name(mut self, bucket_name: impl Into<String>) -> Self {
+        self.bucket_name = Some(bucket_name.into());
+        self
+    }
+
+    /// Sets the endpoint for communicating with AWS S3. Default value
+    /// is based on region.
+    ///
+    /// For example, this might be set to `"http://localhost:4566:`
+    /// for testing against a localstack instance.
+    pub fn with_endpoint(mut self, endpoint: impl Into<String>) -> Self {
+        self.endpoint = Some(endpoint.into());
+        self
+    }
+
+    /// Set the token to use for requests (passed to underlying provider)
+    pub fn with_token(mut self, token: impl Into<String>) -> Self {
+        self.token = Some(token.into());
+        self
+    }
+
+    /// Sets the maximum number of concurrent outstanding
+    /// connectons. Default is `16`.
+    pub fn with_max_connections(mut self, max_connections: NonZeroUsize) -> Self {
+        self.max_connections = max_connections;
+        self
+    }
+
+    /// Sets what protocol is allowed. If `allow_http` is :
+    /// * false (default):  Only HTTPS are allowed
+    /// * true:  HTTP and HTTPS are allowed
+    pub fn with_allow_http(mut self, allow_http: bool) -> Self {
+        self.allow_http = allow_http;
+        self
+    }
+
+    /// Create a [`AmazonS3`] instance from the provided values,
+    /// consuming `self`.
+    pub fn build(self) -> Result<AmazonS3> {
+        let Self {
+            access_key_id,
+            secret_access_key,
+            region,
+            bucket_name,
+            endpoint,
+            token,
+            max_connections,
+            allow_http,
+        } = self;
+
+        let region = region.ok_or(Error::MissingRegion {})?;
+        let bucket_name = bucket_name.ok_or(Error::MissingBucketName {})?;
+
+        let region: rusoto_core::Region = match endpoint {
+            None => region.parse().context(InvalidRegionSnafu { region })?,
+            Some(endpoint) => rusoto_core::Region::Custom {
+                name: region,
+                endpoint,
+            },
+        };
+
+        let mut builder = HyperBuilder::default();
+        builder.pool_max_idle_per_host(max_connections.get());
+
+        let connector = if allow_http {
+            hyper_rustls::HttpsConnectorBuilder::new()
+                .with_webpki_roots()
+                .https_or_http()
+                .enable_http1()
+                .enable_http2()
+                .build()
+        } else {
+            hyper_rustls::HttpsConnectorBuilder::new()
+                .with_webpki_roots()
+                .https_only()
+                .enable_http1()
+                .enable_http2()
+                .build()
+        };
+
+        let http_client =
+            rusoto_core::request::HttpClient::from_builder(builder, connector);
+
+        let client = match (access_key_id, secret_access_key, token) {
+            (Some(access_key_id), Some(secret_access_key), Some(token)) => {
+                let credentials_provider = StaticProvider::new(
+                    access_key_id,
+                    secret_access_key,
+                    Some(token),
+                    None,
+                );
+                rusoto_s3::S3Client::new_with(http_client, credentials_provider, region)
+            }
+            (Some(access_key_id), Some(secret_access_key), None) => {
+                let credentials_provider =
+                    StaticProvider::new_minimal(access_key_id, secret_access_key);
+                rusoto_s3::S3Client::new_with(http_client, credentials_provider, region)
+            }
+            (None, Some(_), _) => return Err(Error::MissingAccessKey.into()),
+            (Some(_), None, _) => return Err(Error::MissingSecretAccessKey.into()),
+            _ if std::env::var_os("AWS_WEB_IDENTITY_TOKEN_FILE").is_some() => {
+                rusoto_s3::S3Client::new_with(
+                    http_client,
+                    WebIdentityProvider::from_k8s_env(),
+                    region,
+                )
+            }
+            _ => rusoto_s3::S3Client::new_with(
                 http_client,
-                WebIdentityProvider::from_k8s_env(),
+                InstanceMetadataProvider::new(),
                 region,
-            )
-        }
-        _ => rusoto_s3::S3Client::new_with(
-            http_client,
-            InstanceMetadataProvider::new(),
-            region,
-        ),
-    };
+            ),
+        };
 
-    Ok(AmazonS3 {
-        client_unrestricted: client,
-        connection_semaphore: Arc::new(Semaphore::new(max_connections.get())),
-        bucket_name: bucket_name.into(),
-    })
+        Ok(AmazonS3 {
+            client_unrestricted: client,
+            connection_semaphore: Arc::new(Semaphore::new(max_connections.get())),
+            bucket_name,
+        })
+    }
 }
 
-/// Create a new [`AmazonS3`] that always errors
-pub fn new_failing_s3() -> Result<AmazonS3> {
-    new_s3(
-        Some("foo"),
-        Some("bar"),
-        "us-east-1",
-        "bucket",
-        None as Option<&str>,
-        None as Option<&str>,
-        NonZeroUsize::new(16).unwrap(),
-        true,
-    )
-}
+// /// Create a new [`AmazonS3`] that always errors

Review Comment:
   I seem to remember IOx used it for failing tests, now that ObjectStore is object-safe I'm fairly confident it can go



##########
object_store/src/azure.rs:
##########
@@ -570,73 +579,126 @@ fn url_from_env(env_name: &str, default_url: &str) -> Result<Url> {
     Ok(url)
 }
 
-/// Configure a connection to container with given name on Microsoft Azure
-/// Blob store.
+/// Configure a connection to Mirosoft Azure Blob Storage bucket using
+/// the specified credentials.
 ///
-/// The credentials `account` and `access_key` must provide access to the
-/// store.
-pub fn new_azure(
-    account: impl Into<String>,
-    access_key: impl Into<String>,
-    container_name: impl Into<String>,
+/// # Example
+/// ```
+/// # let ACCOUNT = "foo";
+/// # let BUCKET_NAME = "foo";
+/// # let ACCESS_KEY = "foo";
+/// # let CONTAINER_NAME = "foo";
+/// let azure = object_store::azure::MicrosoftAzureBuilder::new()
+///  .with_account(ACCOUNT)
+///  .with_access_key(ACCESS_KEY)
+///  .with_container_name(CONTAINER_NAME)
+///  .build();
+/// ```
+#[derive(Debug, Default)]
+pub struct MicrosoftAzureBuilder {
+    account: Option<String>,
+    access_key: Option<String>,
+    container_name: Option<String>,

Review Comment:
   I think we should match the convention for Azure, i.e. container_name



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] ursabot commented on pull request #2204: Add Builder style config objects for object_store

Posted by GitBox <gi...@apache.org>.
ursabot commented on PR #2204:
URL: https://github.com/apache/arrow-rs/pull/2204#issuecomment-1199786536

   Benchmark runs are scheduled for baseline = 393f0069be6d50cd896e5e7124a3b779f5dbe2ea and contender = 41d96b21628a9035623b72e80d7ea06e5fb904f9. 41d96b21628a9035623b72e80d7ea06e5fb904f9 is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Skipped :warning: Benchmarking of arrow-rs-commits is not supported on ec2-t3-xlarge-us-east-2] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/94310ddcc4ca4ee0b0636d0723772628...09f8a04f0f59426f84f6f98126c50f14/)
   [Skipped :warning: Benchmarking of arrow-rs-commits is not supported on test-mac-arm] [test-mac-arm](https://conbench.ursa.dev/compare/runs/3a81879821f04af3bdc7524d2723dc25...8a67d965aea543649ea13c07f5c169e6/)
   [Skipped :warning: Benchmarking of arrow-rs-commits is not supported on ursa-i9-9960x] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/e2b85c8612c94fa5ad697ca586e384c7...49d95c88c79a42cea38b7263af687e15/)
   [Skipped :warning: Benchmarking of arrow-rs-commits is not supported on ursa-thinkcentre-m75q] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/c0eca22f570c4596a14db642d39befa2...b306f3eb08944713950ef01e0b060f7a/)
   Buildkite builds:
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] alamb commented on a diff in pull request #2204: Add Builder style config objects

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2204:
URL: https://github.com/apache/arrow-rs/pull/2204#discussion_r932153362


##########
object_store/src/azure.rs:
##########
@@ -570,73 +579,126 @@ fn url_from_env(env_name: &str, default_url: &str) -> Result<Url> {
     Ok(url)
 }
 
-/// Configure a connection to container with given name on Microsoft Azure
-/// Blob store.
+/// Configure a connection to Mirosoft Azure Blob Storage bucket using
+/// the specified credentials.
 ///
-/// The credentials `account` and `access_key` must provide access to the
-/// store.
-pub fn new_azure(
-    account: impl Into<String>,
-    access_key: impl Into<String>,
-    container_name: impl Into<String>,
+/// # Example
+/// ```
+/// # let ACCOUNT = "foo";
+/// # let BUCKET_NAME = "foo";
+/// # let ACCESS_KEY = "foo";
+/// # let CONTAINER_NAME = "foo";
+/// let azure = object_store::azure::MicrosoftAzureBuilder::new()
+///  .with_account(ACCOUNT)
+///  .with_access_key(ACCESS_KEY)
+///  .with_container_name(CONTAINER_NAME)
+///  .build();
+/// ```
+#[derive(Debug, Default)]
+pub struct MicrosoftAzureBuilder {
+    account: Option<String>,
+    access_key: Option<String>,
+    container_name: Option<String>,

Review Comment:
   not sure if I should also call this `bucket_name` (generic term) or `container_name` (Azure specific term). I kept the existing naming convention



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] alamb commented on a diff in pull request #2204: Add Builder style config objects for object_store

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2204:
URL: https://github.com/apache/arrow-rs/pull/2204#discussion_r933454130


##########
object_store/src/gcp.rs:
##########
@@ -779,55 +785,116 @@ fn reader_credentials_file(
     Ok(serde_json::from_reader(reader).context(DecodeCredentialsSnafu)?)
 }
 
-/// Configure a connection to Google Cloud Storage.
-pub fn new_gcs(
-    service_account_path: impl AsRef<std::path::Path>,
-    bucket_name: impl Into<String>,
-) -> Result<GoogleCloudStorage> {
-    new_gcs_with_client(service_account_path, bucket_name, Client::new())
+/// Configure a connection to Google Cloud Storage using the specified
+/// credentials.
+///
+/// # Example
+/// ```
+/// # let BUCKET_NAME = "foo";
+/// # let SERVICE_ACCOUNT_PATH = "/tmp/foo.json";
+/// let gcs = object_store::gcp::GoogleCloudStorageBuilder::new()
+///  .with_service_account_path(SERVICE_ACCOUNT_PATH)
+///  .with_bucket_name(BUCKET_NAME)
+///  .build();
+/// ```
+#[derive(Debug, Default)]
+pub struct GoogleCloudStorageBuilder {
+    bucket_name: Option<String>,
+    service_account_path: Option<String>,
+    client: Option<Client>,
 }
 
-/// Configure a connection to Google Cloud Storage with the specified HTTP client.
-pub fn new_gcs_with_client(
-    service_account_path: impl AsRef<std::path::Path>,
-    bucket_name: impl Into<String>,
-    client: Client,
-) -> Result<GoogleCloudStorage> {
-    let credentials = reader_credentials_file(service_account_path)?;
-
-    // TODO: https://cloud.google.com/storage/docs/authentication#oauth-scopes
-    let scope = "https://www.googleapis.com/auth/devstorage.full_control";
-    let audience = "https://www.googleapis.com/oauth2/v4/token".to_string();
-
-    let oauth_provider = (!credentials.disable_oauth)
-        .then(|| {
-            OAuthProvider::new(
-                credentials.client_email,
-                credentials.private_key,
-                scope.to_string(),
-                audience,
-            )
-        })
-        .transpose()?;
+impl GoogleCloudStorageBuilder {
+    /// Create a new [`GoogleCloudStorageBuilder`] with default values.
+    pub fn new() -> Self {
+        Default::default()
+    }
 
-    let bucket_name = bucket_name.into();
-    let encoded_bucket_name =
-        percent_encode(bucket_name.as_bytes(), NON_ALPHANUMERIC).to_string();
+    /// Set the bucket name (required)
+    pub fn with_bucket_name(mut self, bucket_name: impl Into<String>) -> Self {
+        self.bucket_name = Some(bucket_name.into());
+        self
+    }
 
-    // The cloud storage crate currently only supports authentication via
-    // environment variables. Set the environment variable explicitly so
-    // that we can optionally accept command line arguments instead.
-    Ok(GoogleCloudStorage {
-        client: Arc::new(GoogleCloudStorageClient {
-            client,
-            base_url: credentials.gcs_base_url,
-            oauth_provider,
-            token_cache: Default::default(),
+    /// Set the path to the service account file (required). Example
+    /// `"/tmp/gcs.json"`
+    ///
+    /// Example contents of `gcs.json`:
+    ///
+    /// ```json
+    /// {
+    ///    "gcs_base_url": "https://localhost:4443",
+    ///    "disable_oauth": true,
+    ///    "client_email": "",
+    ///    "private_key": ""
+    /// }
+    /// ```
+    pub fn with_service_account_path(
+        mut self,
+        service_account_path: impl Into<String>,
+    ) -> Self {
+        self.service_account_path = Some(service_account_path.into());
+        self
+    }
+
+    /// Use the specified http [`Client`] (defaults to [`Client::new`])
+    ///
+    /// This allows you to set custom client options such as allowing
+    /// non secure connections or custom headers.
+    pub fn with_client(mut self, client: Client) -> Self {

Review Comment:
   Since it is only in `test` I will make it `#[cfg(test)]` and we can always make it pub later



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] alamb merged pull request #2204: Add Builder style config objects for object_store

Posted by GitBox <gi...@apache.org>.
alamb merged PR #2204:
URL: https://github.com/apache/arrow-rs/pull/2204


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org