You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "chitralverma (via GitHub)" <gi...@apache.org> on 2023/04/13 10:35:29 UTC

[GitHub] [arrow-rs] chitralverma opened a new pull request, #4077: parse_url prototype

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

   # Which issue does this PR close?
   
   Closes https://github.com/apache/arrow-rs/issues/4047.
   
   # Rationale for this change
    
   This PR proposes a standardized implementation to create an object store from provided URL and options. It would make things significantly simple for developers using the crate.
   
   # What changes are included in this PR?
   
   **Check list**
   
   - [x] Add a `parse_url( ... )`
   - [ ] Tests for all object stores X `parse_url( ... )`
   - [ ] Examples
   - [ ] Documentation
   
   # Are there any user-facing changes?
   
   Yes, `parse_url( ... )` is user-facing.
   No breaking changes.
   


-- 
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 #4077: (object_store): Instantiate object store from provided url

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1165545023


##########
object_store/src/lib.rs:
##########
@@ -717,6 +726,139 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url<
+    I: IntoIterator<Item = (impl AsRef<str>, impl Into<String> + Clone)> + Clone,
+>(
+    url: impl AsRef<str>,
+    options: I,
+) -> Result<Box<DynObjectStore>> {
+    let storage_url = url.as_ref();
+
+    if let Ok(url) = Url::parse(storage_url) {
+        let opts = options
+            .clone()
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value));
+
+        let allow_http: bool = options.into_iter().any(|(key, value)| {
+            key.as_ref().to_ascii_lowercase().contains("allow_http")
+                & value.into().eq_ignore_ascii_case("true")
+        });
+
+        match url.scheme() {
+            #[cfg(any(feature = "aws", feature = "aws_profile"))]
+            "s3" | "s3a" => {
+                let opts = opts

Review Comment:
   I guess my concern is that a typo-ed option can result in a very poor debugging UX if it doesn't return an error, I'm curious why we would want to accept options that aren't valid



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1167110158


##########
object_store/src/lib.rs:
##########
@@ -717,6 +726,139 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url<
+    I: IntoIterator<Item = (impl AsRef<str>, impl Into<String> + Clone)> + Clone,
+>(
+    url: impl AsRef<str>,
+    options: I,
+) -> Result<Box<DynObjectStore>> {
+    let storage_url = url.as_ref();
+
+    if let Ok(url) = Url::parse(storage_url) {
+        let opts = options
+            .clone()
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value));
+
+        let allow_http: bool = options.into_iter().any(|(key, value)| {

Review Comment:
   @tustvold I have added a different approach for this via `options.rs` which has conditional compilation on `ClientOptions` to not only support `allow_http` but all other options available to the user



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1165660875


##########
object_store/src/lib.rs:
##########
@@ -717,6 +726,139 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url<
+    I: IntoIterator<Item = (impl AsRef<str>, impl Into<String> + Clone)> + Clone,
+>(
+    url: impl AsRef<str>,
+    options: I,
+) -> Result<Box<DynObjectStore>> {
+    let storage_url = url.as_ref();
+
+    if let Ok(url) = Url::parse(storage_url) {
+        let opts = options
+            .clone()
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value));
+
+        let allow_http: bool = options.into_iter().any(|(key, value)| {
+            key.as_ref().to_ascii_lowercase().contains("allow_http")
+                & value.into().eq_ignore_ascii_case("true")
+        });
+
+        match url.scheme() {
+            #[cfg(any(feature = "aws", feature = "aws_profile"))]
+            "s3" | "s3a" => {
+                let opts = opts

Review Comment:
   >  guess my concern is that a typo-ed option can result in a very poor debugging UX if it doesn't return an error
   
   makes sense, I'd would change it



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1178638590


##########
object_store/src/options.rs:
##########
@@ -0,0 +1,166 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+pub use std::str::FromStr;
+
+#[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+use crate::client::ClientOptions;
+
+#[cfg(any(feature = "aws"))]
+use crate::aws::AmazonS3ConfigKey;
+#[cfg(feature = "azure")]
+use crate::azure::AzureConfigKey;
+#[cfg(feature = "gcp")]
+use crate::gcp::GoogleConfigKey;
+
+/// Options used for configuring backend store
+#[derive(Clone, Debug, Default)]
+pub struct StoreOptions {
+    /// Store specific options like key, secret, region etc.
+    _store_options: HashMap<String, String>,
+
+    /// Options specific for the internal client
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    client_options: ClientOptions,
+}
+
+impl StoreOptions {
+    /// Create a new instance of [`StorageOptions`]
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub fn new(
+        store_options: HashMap<String, String>,
+        client_options: ClientOptions,
+    ) -> Self {
+        Self {
+            _store_options: store_options,
+            client_options,
+        }
+    }
+
+    #[cfg(not(any(
+        feature = "gcp",
+        feature = "aws",
+        feature = "azure",
+        feature = "http"
+    )))]
+    pub fn new(store_options: HashMap<String, String>) -> Self {
+        Self {
+            _store_options: store_options,
+        }
+    }
+
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub fn from_iterable<I: IntoIterator<Item = (impl AsRef<str>, impl Into<String>)>>(
+        iter: I,
+        client_options: ClientOptions,
+    ) -> Self {
+        let store_options: HashMap<String, String> = iter
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value.into()))
+            .collect();
+
+        Self {
+            _store_options: store_options,
+            client_options,
+        }
+    }
+
+    #[cfg(not(any(
+        feature = "gcp",
+        feature = "aws",
+        feature = "azure",
+        feature = "http"
+    )))]
+    pub fn from_iterable<I: IntoIterator<Item = (impl AsRef<str>, impl Into<String>)>>(
+        iter: I,
+    ) -> Self {
+        let store_options: HashMap<String, String> = iter
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value.into()))
+            .collect();
+
+        Self {
+            _store_options: store_options,
+        }
+    }
+
+    /// Gets an instance of ClientOptions
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub fn get_client_options(&self) -> ClientOptions {
+        self.client_options.clone()
+    }
+
+    /// Ensures that provided options are compatible with Azure
+    #[cfg(feature = "azure")]
+    pub fn get_azure_options(&self) -> HashMap<AzureConfigKey, String> {
+        self._store_options
+            .iter()
+            .map(|(key, value)| {
+                let conf_key =
+                    AzureConfigKey::from_str(&key.to_ascii_lowercase()).unwrap();

Review Comment:
   @roeap Thanks for the review. 
   I'm not sure what exactly you meant here, but based on my understanding and a little searching, I found something called error propagation and have made some changes for this.
   
   Can you please check if this is what you meant and if this is on the right track?



-- 
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 #4077: (object_store): Instantiate object store from provided url

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1165406088


##########
object_store/src/lib.rs:
##########
@@ -717,6 +726,139 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url<
+    I: IntoIterator<Item = (impl AsRef<str>, impl Into<String> + Clone)> + Clone,
+>(
+    url: impl AsRef<str>,
+    options: I,
+) -> Result<Box<DynObjectStore>> {
+    let storage_url = url.as_ref();
+
+    if let Ok(url) = Url::parse(storage_url) {
+        let opts = options
+            .clone()
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value));
+
+        let allow_http: bool = options.into_iter().any(|(key, value)| {

Review Comment:
   I think this should probably be made into a config option where relevant, as opposed to special casing it



##########
object_store/src/lib.rs:
##########
@@ -717,6 +726,139 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url<
+    I: IntoIterator<Item = (impl AsRef<str>, impl Into<String> + Clone)> + Clone,
+>(
+    url: impl AsRef<str>,
+    options: I,
+) -> Result<Box<DynObjectStore>> {
+    let storage_url = url.as_ref();
+
+    if let Ok(url) = Url::parse(storage_url) {
+        let opts = options
+            .clone()
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value));
+
+        let allow_http: bool = options.into_iter().any(|(key, value)| {
+            key.as_ref().to_ascii_lowercase().contains("allow_http")
+                & value.into().eq_ignore_ascii_case("true")
+        });
+
+        match url.scheme() {
+            #[cfg(any(feature = "aws", feature = "aws_profile"))]
+            "s3" | "s3a" => {
+                let opts = opts

Review Comment:
   What is the motivation for filtering the options? IMO if you provide an invalid option it should return an error?



##########
object_store/src/lib.rs:
##########
@@ -705,6 +707,13 @@ pub enum Error {
         store
     ))]
     UnknownConfigurationKey { store: &'static str, key: String },
+
+    #[snafu(display(
+        "object_store must be built with feature '{}' to support loading from '{}'.",
+        feature,
+        url
+    ))]
+    MissingFeature { feature: &'static str, url: String },

Review Comment:
   :+1:



##########
object_store/src/lib.rs:
##########
@@ -717,6 +726,139 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url<
+    I: IntoIterator<Item = (impl AsRef<str>, impl Into<String> + Clone)> + Clone,
+>(
+    url: impl AsRef<str>,
+    options: I,
+) -> Result<Box<DynObjectStore>> {
+    let storage_url = url.as_ref();
+
+    if let Ok(url) = Url::parse(storage_url) {

Review Comment:
   I'd support returning an error instead



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1165520700


##########
object_store/src/lib.rs:
##########
@@ -717,6 +726,139 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url<
+    I: IntoIterator<Item = (impl AsRef<str>, impl Into<String> + Clone)> + Clone,
+>(
+    url: impl AsRef<str>,
+    options: I,
+) -> Result<Box<DynObjectStore>> {
+    let storage_url = url.as_ref();
+
+    if let Ok(url) = Url::parse(storage_url) {

Review Comment:
   The rationale behind this was to allow `memory` or `/` as valid inputs for `parse_url`. if we return Err on `Url::parse(storage_url)` then for memory users will have to pass `memory://` and `file://` 
   
   So there are pros/cons of either side.
   



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1175167909


##########
object_store/src/options.rs:
##########
@@ -0,0 +1,166 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+pub use std::str::FromStr;
+
+#[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+use crate::client::ClientOptions;
+
+#[cfg(any(feature = "aws"))]
+use crate::aws::AmazonS3ConfigKey;
+#[cfg(feature = "azure")]
+use crate::azure::AzureConfigKey;
+#[cfg(feature = "gcp")]
+use crate::gcp::GoogleConfigKey;
+
+/// Options used for configuring backend store
+#[derive(Clone, Debug, Default)]
+pub struct StoreOptions {
+    /// Store specific options like key, secret, region etc.
+    _store_options: HashMap<String, String>,
+
+    /// Options specific for the internal client
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    client_options: ClientOptions,
+}
+
+impl StoreOptions {
+    /// Create a new instance of [`StorageOptions`]
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub fn new(
+        store_options: HashMap<String, String>,
+        client_options: ClientOptions,

Review Comment:
   good catch. done!



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1181068286


##########
object_store/src/lib.rs:
##########
@@ -717,6 +727,131 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url(
+    url: impl AsRef<str>,
+    store_options: Option<impl Into<StoreOptions>>,
+    _from_env: bool,

Review Comment:
   yes, I thought about that as well. let me do something about this and then we can merge our ideas on this.
   
   goals - 
    - Allow URL based obejct_store instantiation
    - store options are either explicitly passed or picked from env or both (preference given to explicit over env)
    - optionally allow internal `ClientOptions` to be passed
    - Should work for all object stores like local, HTTP, AWS, GCS, Azure, mem
   



##########
object_store/src/lib.rs:
##########
@@ -717,6 +727,131 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url(
+    url: impl AsRef<str>,
+    store_options: Option<impl Into<StoreOptions>>,
+    _from_env: bool,

Review Comment:
   yes, I thought about that as well. let me do something about this and then we can merge our ideas on this.
   
   goals - 
    - allow URL based obejct_store instantiation
    - store options are either explicitly passed or picked from env or both (preference given to explicit over env)
    - optionally allow internal `ClientOptions` to be passed
    - should work for all object stores like local, HTTP, AWS, GCS, Azure, mem
   



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1181068286


##########
object_store/src/lib.rs:
##########
@@ -717,6 +727,131 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url(
+    url: impl AsRef<str>,
+    store_options: Option<impl Into<StoreOptions>>,
+    _from_env: bool,

Review Comment:
   yes, I thought about that as well. let me do something about this and then we can merge our ideas on this.
   
   goals - 
    - allow URL based obejct_store instantiation
    - store options are either explicitly passed or picked from env or both (preference given to explicit over env)
    - optionally allow internal `ClientOptions` to be passed
    - should work for all object stores like local, HTTP, AWS, GCS, Azure, mem
    - user facing API should be simple and natural
   



-- 
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] chitralverma commented on pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#issuecomment-1506978141

   @tustvold one question I had regarding the tests - I want to test parse_url for each supported store. so shall I put them separately in each module or create 1 big test in the same file?


-- 
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 pull request #4077: (object_store): Instantiate object store from provided url

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#issuecomment-1523214949

   I've marked this as a draft as it doesn't appear to be ready for review


-- 
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] chitralverma commented on pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#issuecomment-1524989988

   @tustvold @roeap The implementation and the test cases are ready for review.
   If there arent any other changes then I will add examples and docs and then mark this PR as ready for review


-- 
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 closed pull request #4077: (object_store): Instantiate object store from provided url

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold closed pull request #4077: (object_store): Instantiate object store from provided url 
URL: https://github.com/apache/arrow-rs/pull/4077


-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1175151986


##########
object_store/src/options.rs:
##########
@@ -0,0 +1,166 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+pub use std::str::FromStr;
+
+#[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+use crate::client::ClientOptions;
+
+#[cfg(any(feature = "aws"))]
+use crate::aws::AmazonS3ConfigKey;
+#[cfg(feature = "azure")]
+use crate::azure::AzureConfigKey;
+#[cfg(feature = "gcp")]
+use crate::gcp::GoogleConfigKey;
+
+/// Options used for configuring backend store
+#[derive(Clone, Debug, Default)]
+pub struct StoreOptions {
+    /// Store specific options like key, secret, region etc.
+    _store_options: HashMap<String, String>,
+
+    /// Options specific for the internal client
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    client_options: ClientOptions,
+}
+
+impl StoreOptions {
+    /// Create a new instance of [`StorageOptions`]
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub fn new(
+        store_options: HashMap<String, String>,
+        client_options: ClientOptions,
+    ) -> Self {
+        Self {
+            _store_options: store_options,
+            client_options,
+        }
+    }
+
+    #[cfg(not(any(
+        feature = "gcp",
+        feature = "aws",
+        feature = "azure",
+        feature = "http"
+    )))]
+    pub fn new(store_options: HashMap<String, String>) -> Self {
+        Self {
+            _store_options: store_options,
+        }
+    }
+
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub fn from_iterable<I: IntoIterator<Item = (impl AsRef<str>, impl Into<String>)>>(
+        iter: I,
+        client_options: ClientOptions,
+    ) -> Self {
+        let store_options: HashMap<String, String> = iter
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value.into()))
+            .collect();
+
+        Self {
+            _store_options: store_options,
+            client_options,
+        }
+    }
+
+    #[cfg(not(any(
+        feature = "gcp",
+        feature = "aws",
+        feature = "azure",
+        feature = "http"
+    )))]
+    pub fn from_iterable<I: IntoIterator<Item = (impl AsRef<str>, impl Into<String>)>>(
+        iter: I,
+    ) -> Self {
+        let store_options: HashMap<String, String> = iter
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value.into()))
+            .collect();
+
+        Self {
+            _store_options: store_options,
+        }
+    }
+
+    /// Gets an instance of ClientOptions
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub fn get_client_options(&self) -> ClientOptions {
+        self.client_options.clone()
+    }
+
+    /// Ensures that provided options are compatible with Azure
+    #[cfg(feature = "azure")]
+    pub fn get_azure_options(&self) -> HashMap<AzureConfigKey, String> {
+        self._store_options
+            .iter()
+            .map(|(key, value)| {
+                let conf_key =
+                    AzureConfigKey::from_str(&key.to_ascii_lowercase()).unwrap();

Review Comment:
   @roeap this is how it was done before, but based on @tustvold's [comments](https://github.com/apache/arrow-rs/pull/4077#discussion_r1165408572) I changed it.



-- 
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] roeap commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "roeap (via GitHub)" <gi...@apache.org>.
roeap commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1168434636


##########
object_store/src/options.rs:
##########
@@ -0,0 +1,166 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+pub use std::str::FromStr;
+
+#[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+use crate::client::ClientOptions;
+
+#[cfg(any(feature = "aws"))]
+use crate::aws::AmazonS3ConfigKey;
+#[cfg(feature = "azure")]
+use crate::azure::AzureConfigKey;
+#[cfg(feature = "gcp")]
+use crate::gcp::GoogleConfigKey;
+
+/// Options used for configuring backend store
+#[derive(Clone, Debug, Default)]
+pub struct StoreOptions {
+    /// Store specific options like key, secret, region etc.
+    _store_options: HashMap<String, String>,
+
+    /// Options specific for the internal client
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    client_options: ClientOptions,
+}
+
+impl StoreOptions {
+    /// Create a new instance of [`StorageOptions`]
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub fn new(
+        store_options: HashMap<String, String>,
+        client_options: ClientOptions,

Review Comment:
   Not feeling too strongly about this, but the `from_iterator` method also will accept a HashMap as it should fit the type constraints as well. So maybe it makes sense to have the generic type in `new` and drop the `from_iterator` method?



##########
object_store/src/options.rs:
##########
@@ -0,0 +1,166 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+pub use std::str::FromStr;
+
+#[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+use crate::client::ClientOptions;
+
+#[cfg(any(feature = "aws"))]
+use crate::aws::AmazonS3ConfigKey;
+#[cfg(feature = "azure")]
+use crate::azure::AzureConfigKey;
+#[cfg(feature = "gcp")]
+use crate::gcp::GoogleConfigKey;
+
+/// Options used for configuring backend store
+#[derive(Clone, Debug, Default)]
+pub struct StoreOptions {
+    /// Store specific options like key, secret, region etc.
+    _store_options: HashMap<String, String>,
+
+    /// Options specific for the internal client
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    client_options: ClientOptions,
+}
+
+impl StoreOptions {
+    /// Create a new instance of [`StorageOptions`]
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub fn new(
+        store_options: HashMap<String, String>,
+        client_options: ClientOptions,
+    ) -> Self {
+        Self {
+            _store_options: store_options,
+            client_options,
+        }
+    }
+
+    #[cfg(not(any(
+        feature = "gcp",
+        feature = "aws",
+        feature = "azure",
+        feature = "http"
+    )))]
+    pub fn new(store_options: HashMap<String, String>) -> Self {
+        Self {
+            _store_options: store_options,
+        }
+    }
+
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub fn from_iterable<I: IntoIterator<Item = (impl AsRef<str>, impl Into<String>)>>(
+        iter: I,
+        client_options: ClientOptions,
+    ) -> Self {
+        let store_options: HashMap<String, String> = iter
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value.into()))
+            .collect();
+
+        Self {
+            _store_options: store_options,
+            client_options,
+        }
+    }
+
+    #[cfg(not(any(
+        feature = "gcp",
+        feature = "aws",
+        feature = "azure",
+        feature = "http"
+    )))]
+    pub fn from_iterable<I: IntoIterator<Item = (impl AsRef<str>, impl Into<String>)>>(
+        iter: I,
+    ) -> Self {
+        let store_options: HashMap<String, String> = iter
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value.into()))
+            .collect();
+
+        Self {
+            _store_options: store_options,
+        }
+    }
+
+    /// Gets an instance of ClientOptions
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub fn get_client_options(&self) -> ClientOptions {
+        self.client_options.clone()
+    }
+
+    /// Ensures that provided options are compatible with Azure
+    #[cfg(feature = "azure")]
+    pub fn get_azure_options(&self) -> HashMap<AzureConfigKey, String> {
+        self._store_options
+            .iter()
+            .map(|(key, value)| {
+                let conf_key =
+                    AzureConfigKey::from_str(&key.to_ascii_lowercase()).unwrap();

Review Comment:
   should we either return an error or omit the keys that will panic here?



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1165520700


##########
object_store/src/lib.rs:
##########
@@ -717,6 +726,139 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url<
+    I: IntoIterator<Item = (impl AsRef<str>, impl Into<String> + Clone)> + Clone,
+>(
+    url: impl AsRef<str>,
+    options: I,
+) -> Result<Box<DynObjectStore>> {
+    let storage_url = url.as_ref();
+
+    if let Ok(url) = Url::parse(storage_url) {

Review Comment:
   The rationale bind this was to allow `memory` or `/` as valid inputs for `parse_url`. if we return Err on `Url::parse(storage_url)` then for memory users will have to pass `memory://` and `file://` 



##########
object_store/src/lib.rs:
##########
@@ -717,6 +726,139 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url<
+    I: IntoIterator<Item = (impl AsRef<str>, impl Into<String> + Clone)> + Clone,
+>(
+    url: impl AsRef<str>,
+    options: I,
+) -> Result<Box<DynObjectStore>> {
+    let storage_url = url.as_ref();
+
+    if let Ok(url) = Url::parse(storage_url) {

Review Comment:
   The rationale behind this was to allow `memory` or `/` as valid inputs for `parse_url`. if we return Err on `Url::parse(storage_url)` then for memory users will have to pass `memory://` and `file://` 



-- 
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 pull request #4077: (object_store): Instantiate object store from provided url

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#issuecomment-1531489180

   Apologies I have a bit of an interrupted week, and want to give some thought into how this would integrate with things like DataFusion's ObjectStoreRegistry or downstream ObjectStore implementations like HDFS. Thank you for sticking with this, I'll get back to you as soon as I am able


-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1167110791


##########
object_store/src/lib.rs:
##########
@@ -717,6 +726,139 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url<
+    I: IntoIterator<Item = (impl AsRef<str>, impl Into<String> + Clone)> + Clone,
+>(
+    url: impl AsRef<str>,
+    options: I,
+) -> Result<Box<DynObjectStore>> {
+    let storage_url = url.as_ref();
+
+    if let Ok(url) = Url::parse(storage_url) {
+        let opts = options
+            .clone()
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value));
+
+        let allow_http: bool = options.into_iter().any(|(key, value)| {
+            key.as_ref().to_ascii_lowercase().contains("allow_http")
+                & value.into().eq_ignore_ascii_case("true")
+        });
+
+        match url.scheme() {
+            #[cfg(any(feature = "aws", feature = "aws_profile"))]
+            "s3" | "s3a" => {
+                let opts = opts

Review Comment:
   done.



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1167151217


##########
object_store/src/lib.rs:
##########
@@ -717,6 +726,139 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url<
+    I: IntoIterator<Item = (impl AsRef<str>, impl Into<String> + Clone)> + Clone,
+>(
+    url: impl AsRef<str>,
+    options: I,
+) -> Result<Box<DynObjectStore>> {
+    let storage_url = url.as_ref();
+
+    if let Ok(url) = Url::parse(storage_url) {

Review Comment:
   done.



-- 
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 pull request #4077: (object_store): Instantiate object store from provided url

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#issuecomment-1506987557

   > so shall I put them separately in each module
   
   I suspect this will make the feature flags easier


-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1167576637


##########
object_store/src/options.rs:
##########
@@ -0,0 +1,129 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+pub use std::str::FromStr;
+
+#[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+use crate::client::ClientOptions;
+
+#[cfg(any(feature = "aws"))]
+use crate::aws::AmazonS3ConfigKey;
+#[cfg(feature = "azure")]
+use crate::azure::AzureConfigKey;
+#[cfg(feature = "gcp")]
+use crate::gcp::GoogleConfigKey;
+
+/// Options used for configuring backend store
+#[derive(Clone, Debug, Default)]
+pub struct StoreOptions {
+    /// Store specific options like key, secret, region etc.
+    pub store_options: HashMap<String, String>,
+
+    /// Options specific for the internal client
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub client_options: ClientOptions,

Review Comment:
   done.



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1165531596


##########
object_store/src/lib.rs:
##########
@@ -717,6 +726,139 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url<
+    I: IntoIterator<Item = (impl AsRef<str>, impl Into<String> + Clone)> + Clone,
+>(
+    url: impl AsRef<str>,
+    options: I,
+) -> Result<Box<DynObjectStore>> {
+    let storage_url = url.as_ref();
+
+    if let Ok(url) = Url::parse(storage_url) {
+        let opts = options
+            .clone()
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value));
+
+        let allow_http: bool = options.into_iter().any(|(key, value)| {
+            key.as_ref().to_ascii_lowercase().contains("allow_http")
+                & value.into().eq_ignore_ascii_case("true")
+        });
+
+        match url.scheme() {
+            #[cfg(any(feature = "aws", feature = "aws_profile"))]
+            "s3" | "s3a" => {
+                let opts = opts

Review Comment:
   its more of a "use configs that are valid" instead of an "all configs must be valid".



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1167569655


##########
object_store/src/lib.rs:
##########
@@ -717,6 +727,113 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///

Review Comment:
   still a WIP 😂
   
   there are some open items in my checklist



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1182197065


##########
object_store/src/lib.rs:
##########
@@ -717,6 +727,131 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url(
+    url: impl AsRef<str>,
+    store_options: Option<impl Into<StoreOptions>>,
+    _from_env: bool,

Review Comment:
   Created a builder pattern for this.
   
   https://github.com/apache/arrow-rs/pull/4077/files#r1182195434



-- 
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] chitralverma commented on pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#issuecomment-1527087240

   also one more question - Since each object store has its own `parse_url` already, do you think we should have a different name for the functionality in this PR ?
   


-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1167576610


##########
object_store/src/options.rs:
##########
@@ -0,0 +1,129 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+pub use std::str::FromStr;
+
+#[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+use crate::client::ClientOptions;
+
+#[cfg(any(feature = "aws"))]
+use crate::aws::AmazonS3ConfigKey;
+#[cfg(feature = "azure")]
+use crate::azure::AzureConfigKey;
+#[cfg(feature = "gcp")]
+use crate::gcp::GoogleConfigKey;
+
+/// Options used for configuring backend store
+#[derive(Clone, Debug, Default)]
+pub struct StoreOptions {
+    /// Store specific options like key, secret, region etc.
+    pub store_options: HashMap<String, String>,

Review Comment:
   done.



-- 
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] roeap commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "roeap (via GitHub)" <gi...@apache.org>.
roeap commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1175461236


##########
object_store/src/options.rs:
##########
@@ -0,0 +1,166 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+pub use std::str::FromStr;
+
+#[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+use crate::client::ClientOptions;
+
+#[cfg(any(feature = "aws"))]
+use crate::aws::AmazonS3ConfigKey;
+#[cfg(feature = "azure")]
+use crate::azure::AzureConfigKey;
+#[cfg(feature = "gcp")]
+use crate::gcp::GoogleConfigKey;
+
+/// Options used for configuring backend store
+#[derive(Clone, Debug, Default)]
+pub struct StoreOptions {
+    /// Store specific options like key, secret, region etc.
+    _store_options: HashMap<String, String>,
+
+    /// Options specific for the internal client
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    client_options: ClientOptions,
+}
+
+impl StoreOptions {
+    /// Create a new instance of [`StorageOptions`]
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub fn new(
+        store_options: HashMap<String, String>,
+        client_options: ClientOptions,
+    ) -> Self {
+        Self {
+            _store_options: store_options,
+            client_options,
+        }
+    }
+
+    #[cfg(not(any(
+        feature = "gcp",
+        feature = "aws",
+        feature = "azure",
+        feature = "http"
+    )))]
+    pub fn new(store_options: HashMap<String, String>) -> Self {
+        Self {
+            _store_options: store_options,
+        }
+    }
+
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub fn from_iterable<I: IntoIterator<Item = (impl AsRef<str>, impl Into<String>)>>(
+        iter: I,
+        client_options: ClientOptions,
+    ) -> Self {
+        let store_options: HashMap<String, String> = iter
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value.into()))
+            .collect();
+
+        Self {
+            _store_options: store_options,
+            client_options,
+        }
+    }
+
+    #[cfg(not(any(
+        feature = "gcp",
+        feature = "aws",
+        feature = "azure",
+        feature = "http"
+    )))]
+    pub fn from_iterable<I: IntoIterator<Item = (impl AsRef<str>, impl Into<String>)>>(
+        iter: I,
+    ) -> Self {
+        let store_options: HashMap<String, String> = iter
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value.into()))
+            .collect();
+
+        Self {
+            _store_options: store_options,
+        }
+    }
+
+    /// Gets an instance of ClientOptions
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub fn get_client_options(&self) -> ClientOptions {
+        self.client_options.clone()
+    }
+
+    /// Ensures that provided options are compatible with Azure
+    #[cfg(feature = "azure")]
+    pub fn get_azure_options(&self) -> HashMap<AzureConfigKey, String> {
+        self._store_options
+            .iter()
+            .map(|(key, value)| {
+                let conf_key =
+                    AzureConfigKey::from_str(&key.to_ascii_lowercase()).unwrap();

Review Comment:
   I meant making the function fallible and return the error, rather then unwrapping and thus panicing ... 



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1182195434


##########
object_store/src/builder.rs:
##########
@@ -0,0 +1,301 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Implementation of `ObjectStoreBuilder` which allows the creation of object store
+//! from provided url and options.
+//!
+use std::collections::HashMap;
+pub use std::str::FromStr;
+use url::Url;
+
+#[cfg(feature = "aws")]
+use crate::aws;
+#[cfg(feature = "azure")]
+use crate::azure;
+#[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+use crate::client::ClientOptions;
+#[cfg(feature = "gcp")]
+use crate::gcp;
+#[cfg(feature = "http")]
+use crate::http;
+#[cfg(not(target_arch = "wasm32"))]
+use crate::local;
+
+use crate::{memory, DynObjectStore, Error, Result};
+
+#[allow(dead_code)]
+#[derive(Debug, Clone)]
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+/// ```
+/// # let BUCKET_NAME = "foo";
+/// # let ACCESS_KEY_ID = "foo";
+/// # let SECRET_KEY = "foo";
+/// # use object_store::builder::ObjectStoreBuilder;///
+///
+/// // Instantiate Local FS
+/// # let LOCAL_URL = "file:///";
+/// let local = ObjectStoreBuilder::new(LOCAL_URL).build();
+///
+/// // Instantiate S3
+/// # let S3_URL = "s3://foo/";
+/// let s3_opts =
+///    vec![
+///       ("AWS_ACCESS_KEY_ID", "abc"),
+///       ("AWS_SECRET_ACCESS_KEY", "xyz")
+///    ];
+/// let s3 = ObjectStoreBuilder::new(S3_URL).with_store_options(s3_opts).build();
+///
+/// // Instantiate Azure
+/// # let AZURE_URL = "az://foo/";
+/// let azure = ObjectStoreBuilder::new(AZURE_URL).with_env_variables(true).build();
+///
+/// // Instantiate GCS
+/// # let GCS_URL = "gs://foo/";
+/// let gs_opts = vec![("GOOGLE_SERVICE_ACCOUNT", "/tmp/foo.json")];
+/// let gs = ObjectStoreBuilder::new(AZURE_URL)
+///    .with_store_options(gs_opts)
+///    .with_env_variables(true)
+///    .build();
+/// ```
+pub struct ObjectStoreBuilder {

Review Comment:
   @tustvold Based on your comment I have created a builder pattern for object store.
   
   If this looks good, the we can get rid of the `options.rs` I add as it contains duplicate code.
   
   Also added better examples :)



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1182197065


##########
object_store/src/lib.rs:
##########
@@ -717,6 +727,131 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url(
+    url: impl AsRef<str>,
+    store_options: Option<impl Into<StoreOptions>>,
+    _from_env: bool,

Review Comment:
   Created a builder pattern for this. 
   let me know if you had something like this in mind?
   
   https://github.com/apache/arrow-rs/pull/4077/files#r1182195434



-- 
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 #4077: (object_store): Instantiate object store from provided url

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1180573123


##########
object_store/src/lib.rs:
##########
@@ -717,6 +727,131 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url(
+    url: impl AsRef<str>,
+    store_options: Option<impl Into<StoreOptions>>,
+    _from_env: bool,

Review Comment:
   I wonder if we could instead use a builder pattern here :thinking: I'll have a play over the weekend and see what I can come up with



-- 
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 #4077: (object_store): Instantiate object store from provided url

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1167480241


##########
object_store/src/options.rs:
##########
@@ -0,0 +1,129 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+pub use std::str::FromStr;
+
+#[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+use crate::client::ClientOptions;
+
+#[cfg(any(feature = "aws"))]
+use crate::aws::AmazonS3ConfigKey;
+#[cfg(feature = "azure")]
+use crate::azure::AzureConfigKey;
+#[cfg(feature = "gcp")]
+use crate::gcp::GoogleConfigKey;
+
+/// Options used for configuring backend store
+#[derive(Clone, Debug, Default)]
+pub struct StoreOptions {
+    /// Store specific options like key, secret, region etc.
+    pub store_options: HashMap<String, String>,
+
+    /// Options specific for the internal client
+    #[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+    pub client_options: ClientOptions,

Review Comment:
   ```suggestion
       client_options: ClientOptions,
   ```



##########
object_store/src/lib.rs:
##########
@@ -717,6 +727,113 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///

Review Comment:
   Brilliant example :smile: 



##########
object_store/src/options.rs:
##########
@@ -0,0 +1,129 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::collections::HashMap;
+pub use std::str::FromStr;
+
+#[cfg(any(feature = "gcp", feature = "aws", feature = "azure", feature = "http"))]
+use crate::client::ClientOptions;
+
+#[cfg(any(feature = "aws"))]
+use crate::aws::AmazonS3ConfigKey;
+#[cfg(feature = "azure")]
+use crate::azure::AzureConfigKey;
+#[cfg(feature = "gcp")]
+use crate::gcp::GoogleConfigKey;
+
+/// Options used for configuring backend store
+#[derive(Clone, Debug, Default)]
+pub struct StoreOptions {
+    /// Store specific options like key, secret, region etc.
+    pub store_options: HashMap<String, String>,

Review Comment:
   ```suggestion
       store_options: HashMap<String, String>,
   ```
   
   In case we want to extend this further 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] tustvold commented on pull request #4077: (object_store): Instantiate object store from provided url

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#issuecomment-1547649788

   Closing this as https://github.com/apache/arrow-rs/pull/4200 has now been merged, thank you for your work on this


-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1165526450


##########
object_store/src/lib.rs:
##########
@@ -717,6 +726,139 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url<
+    I: IntoIterator<Item = (impl AsRef<str>, impl Into<String> + Clone)> + Clone,
+>(
+    url: impl AsRef<str>,
+    options: I,
+) -> Result<Box<DynObjectStore>> {
+    let storage_url = url.as_ref();
+
+    if let Ok(url) = Url::parse(storage_url) {
+        let opts = options
+            .clone()
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value));
+
+        let allow_http: bool = options.into_iter().any(|(key, value)| {

Review Comment:
   yes, this is based on what @roeap  mentioned before on the issue.
   
   I actually have to find a better way for `allow_http` and the `ClientOptions` which cannot be `serde`'d. HTTP_store and others need it.



-- 
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] chitralverma commented on a diff in pull request #4077: (object_store): Instantiate object store from provided url

Posted by "chitralverma (via GitHub)" <gi...@apache.org>.
chitralverma commented on code in PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#discussion_r1167110158


##########
object_store/src/lib.rs:
##########
@@ -717,6 +726,139 @@ impl From<Error> for std::io::Error {
     }
 }
 
+/// Creates object store from provided url and options
+///
+/// The scheme of the provided url is used to instantiate the store. If the url
+/// scheme is cannot be mapped to a store, [`NotImplemented`] is raised. For invalid
+/// input, e.g. url with no scheme the default behaviour is to return
+/// [`local::LocalFileSystem`].
+///
+/// # Examples
+///
+/// ```
+///
+/// ```
+pub fn parse_url<
+    I: IntoIterator<Item = (impl AsRef<str>, impl Into<String> + Clone)> + Clone,
+>(
+    url: impl AsRef<str>,
+    options: I,
+) -> Result<Box<DynObjectStore>> {
+    let storage_url = url.as_ref();
+
+    if let Ok(url) = Url::parse(storage_url) {
+        let opts = options
+            .clone()
+            .into_iter()
+            .map(|(key, value)| (key.as_ref().to_ascii_lowercase(), value));
+
+        let allow_http: bool = options.into_iter().any(|(key, value)| {

Review Comment:
   @tustvold I have add a different approach for this via `options.rs` which has conditional compilation on `ClientOptions` to not only support `allow_http` but all other options available to the user



-- 
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 pull request #4077: (object_store): Instantiate object store from provided url

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on PR #4077:
URL: https://github.com/apache/arrow-rs/pull/4077#issuecomment-1540239506

   Thank you for your work on this, I've raised https://github.com/apache/arrow-rs/pull/4184 with a proposal inspired by this. PTAL and let me know what you think


-- 
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