luoyuxia commented on code in PR #1941: URL: https://github.com/apache/fluss/pull/1941#discussion_r2618607098
########## fluss-filesystems/fluss-fs-abfs/src/main/java/org/apache/fluss/fs/abfs/AzureFileSystem.java: ########## @@ -0,0 +1,66 @@ +/* + * 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. + */ + +package org.apache.fluss.fs.abfs; + +import org.apache.fluss.fs.FileSystem; +import org.apache.fluss.fs.abfs.token.AzureDelegationTokenProvider; +import org.apache.fluss.fs.hdfs.HadoopFileSystem; +import org.apache.fluss.fs.token.ObtainedSecurityToken; + +import org.apache.hadoop.conf.Configuration; + +import java.io.IOException; + +/** + * Implementation of the Fluss {@link FileSystem} interface for Azure Blob Storage. This class + * implements the common behavior implemented directly by Fluss and delegates common calls to an + * implementation of Hadoop's filesystem abstraction. + */ +public class AzureFileSystem extends HadoopFileSystem { + + private final String scheme; + private final Configuration conf; + + private AzureDelegationTokenProvider delegationTokenProvider; Review Comment: ```suggestion private volatile AzureDelegationTokenProvider delegationTokenProvider; ``` ########## fluss-filesystems/fluss-fs-abfs/src/main/java/org/apache/fluss/fs/abfs/token/AzureDelegationTokenReceiver.java: ########## @@ -0,0 +1,93 @@ +/* + * 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. + */ + +package org.apache.fluss.fs.abfs.token; + +import org.apache.fluss.fs.token.Credentials; +import org.apache.fluss.fs.token.CredentialsJsonSerde; +import org.apache.fluss.fs.token.ObtainedSecurityToken; +import org.apache.fluss.fs.token.SecurityTokenReceiver; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +/** Security token receiver for the abfs filesystem. */ +public class AzureDelegationTokenReceiver implements SecurityTokenReceiver { + public static final String PROVIDER_CONFIG_NAME = "fs.azure.account.oauth.provider.type"; + + private static final Logger LOG = LoggerFactory.getLogger(AzureDelegationTokenReceiver.class); + + static volatile Credentials credentials; + static volatile Map<String, String> additionInfos; + + public static void updateHadoopConfig(org.apache.hadoop.conf.Configuration hadoopConfig) { Review Comment: updateHadoopConfig is never called. ########## fluss-filesystems/fluss-fs-abfs/src/main/java/org/apache/fluss/fs/abfs/token/AzureDelegationTokenReceiver.java: ########## @@ -0,0 +1,93 @@ +/* + * 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. + */ + +package org.apache.fluss.fs.abfs.token; + +import org.apache.fluss.fs.token.Credentials; +import org.apache.fluss.fs.token.CredentialsJsonSerde; +import org.apache.fluss.fs.token.ObtainedSecurityToken; +import org.apache.fluss.fs.token.SecurityTokenReceiver; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +/** Security token receiver for the abfs filesystem. */ +public class AzureDelegationTokenReceiver implements SecurityTokenReceiver { + public static final String PROVIDER_CONFIG_NAME = "fs.azure.account.oauth.provider.type"; + + private static final Logger LOG = LoggerFactory.getLogger(AzureDelegationTokenReceiver.class); + + static volatile Credentials credentials; + static volatile Map<String, String> additionInfos; + + public static void updateHadoopConfig(org.apache.hadoop.conf.Configuration hadoopConfig) { Review Comment: You can refer `S3FileSystemPlugin` for where to call this method. In `S3FileSystemPlugin`, it will check `ACCESS_KEY_ID` set or not. - if set, do no thing. will use provide ak/sk - if not set, it means the FileSystemPlugin is initialized in client side, it will need to get security token from Fluss to access s3. Then it'll call `S3DelegationTokenReceiver#updateHadoopConfig` to set the credential provider, whose token is from Fluss via method getSecurityToken ########## fluss-filesystems/fluss-fs-abfs/src/main/java/org/apache/fluss/fs/abfs/token/DynamicTemporaryAzureCredentialsProvider.java: ########## @@ -0,0 +1,82 @@ +/* + * 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. + */ + +package org.apache.fluss.fs.abfs.token; + +import org.apache.fluss.fs.token.Credentials; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException; +import org.apache.hadoop.fs.azurebfs.extensions.CustomTokenProviderAdaptee; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.AzureADToken; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Date; + +/** + * Support dynamic token for authenticating with Azure. Please note that users may reference this + * class name from configuration property fs.azure.account.oauth.provider.type. Therefore, changing + * the class name would be a backward-incompatible change. This credential provider must not fail in + * creation because that will break a chain of credential providers. + */ +public class DynamicTemporaryAzureCredentialsProvider extends AccessTokenProvider + implements CustomTokenProviderAdaptee { + + public static final String NAME = DynamicTemporaryAzureCredentialsProvider.class.getName(); + + public static final String COMPONENT = "Dynamic session credentials for Fluss"; + + private volatile AzureADToken azureADToken; Review Comment: Seems azureADToken is not used? ########## fluss-filesystems/fluss-fs-abfs/src/main/java/org/apache/fluss/fs/abfs/token/AzureDelegationTokenProvider.java: ########## @@ -0,0 +1,93 @@ +/* + * 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. + */ + +package org.apache.fluss.fs.abfs.token; + +import org.apache.fluss.fs.token.CredentialsJsonSerde; +import org.apache.fluss.fs.token.ObtainedSecurityToken; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.oauth2.AzureADAuthenticator; +import org.apache.hadoop.fs.azurebfs.oauth2.AzureADToken; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +/** Token provider for abfs Hadoop filesystems. */ +public class AzureDelegationTokenProvider { + + private static final Logger LOG = LoggerFactory.getLogger(AzureDelegationTokenProvider.class); + + private static final String CLIENT_ID = "fs.azure.account.oauth2.client.id"; + private static final String CLIENT_SECRET = "fs.azure.account.oauth2.client.secret"; + + private static final String ENDPOINT_KEY = "fs.azure.account.oauth2.client.endpoint"; + + private final String scheme; + private final String clientId; + private final String clientSecret; + + private final String authEndpoint; + private final Map<String, String> additionInfos; + + public AzureDelegationTokenProvider(String scheme, Configuration conf) { + this.scheme = scheme; + + this.clientId = conf.get(CLIENT_ID); + this.clientSecret = conf.get(CLIENT_SECRET); + this.authEndpoint = conf.get(ENDPOINT_KEY); + this.additionInfos = new HashMap<>(); + + for (String key : Arrays.asList(ENDPOINT_KEY)) { Review Comment: ```suggestion for (String key : Collections.singleton(ENDPOINT_KEY)) ``` -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
