jerryshao commented on code in PR #5020:
URL: https://github.com/apache/gravitino/pull/5020#discussion_r1802263646


##########
catalogs/catalog-hadoop/src/main/java/org/apache/gravitino/catalog/hadoop/fs/FileSystemProvider.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.gravitino.catalog.hadoop.fs;
+
+import java.io.IOException;
+import java.util.Map;
+import javax.annotation.Nonnull;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * FileSystemProvider is an interface for providing FileSystem instances. It 
is used by the
+ * HadoopCatalog to create FileSystem instances for accessing Hadoop 
compatible file systems.
+ */
+public interface FileSystemProvider {
+
+  /**
+   * Get the FileSystem instance according to the configuration map and file 
path.
+   *
+   * <p>Compared to the {@link FileSystem#get(Configuration)} method, this 
method allows the
+   * provider to create a FileSystem instance with a specific configuration 
and do further
+   * initialization if needed.
+   *
+   * <p>For example: 1. We can check the endpoint value validity for 
S3AFileSystem then do further
+   * actions. 2. We can also change some default behavior of the FileSystem 
initialization process
+   * 3. More...
+   *
+   * @param config The configuration for the FileSystem instance.
+   * @param path The path to the file system.
+   * @return The FileSystem instance.
+   * @throws IOException If the FileSystem instance cannot be created.
+   */
+  FileSystem getFileSystem(@Nonnull Path path, @Nonnull Map<String, String> 
config)
+      throws IOException;
+
+  /**
+   * Scheme of this FileSystem provider. The value is 'file' for 
LocalFileSystem, 'hdfs' for HDFS,
+   * etc.
+   *
+   * @return The scheme of this FileSystem provider used.
+   */
+  String scheme();
+
+  /**
+   * Name of this FileSystem provider. The value is 'LocalFileSystemProvider' 
for LocalFileSystem,
+   * 'HDFSFileSystemProvider' for HDFS, etc.

Review Comment:
   The name should be simple like "local", "hdfs", also it should be case 
insensitive, that would be better.



##########
catalogs/catalog-hadoop/src/main/java/org/apache/gravitino/catalog/hadoop/HadoopCatalogOperations.java:
##########
@@ -742,4 +752,25 @@ private boolean checkSingleFile(Fileset fileset) {
           fileset.name());
     }
   }
+
+  FileSystem getFileSystem(Path path, Map<String, String> config) throws 
IOException {
+    if (path == null) {
+      throw new IllegalArgumentException("Path should not be null");
+    }
+
+    String scheme =
+        path.toUri().getScheme() != null
+            ? path.toUri().getScheme()
+            : defaultFileSystemProvider.scheme();
+
+    FileSystemProvider provider = fileSystemProvidersMap.get(scheme);
+    if (provider == null) {
+      throw new IllegalArgumentException(
+          String.format(
+              "Unsupported scheme: %s, path: %s, all supported scheme: %s and 
provider: %s",
+              scheme, path, fileSystemProvidersMap.keySet(), 
fileSystemProvidersMap.values()));

Review Comment:
   Do you make sure that "keySet" and "values" can be correctly printed out?



##########
catalogs/catalog-hadoop/src/main/java/org/apache/gravitino/catalog/hadoop/fs/FileSystemUtils.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.gravitino.catalog.hadoop.fs;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.Set;
+
+public class FileSystemUtils {
+
+  private FileSystemUtils() {}
+
+  public static Map<String, FileSystemProvider> getFileSystemProviders(String 
fileSystemProviders) {
+    Map<String, FileSystemProvider> resultMap = Maps.newHashMap();
+    ServiceLoader<FileSystemProvider> allFileSystemProviders =
+        ServiceLoader.load(FileSystemProvider.class);

Review Comment:
   You can filter out unconfigured providers in loader's iterator, and then add 
the built-in providers, that will simplify your code.



##########
catalogs/catalog-hadoop/src/main/java/org/apache/gravitino/catalog/hadoop/fs/HDFSFileSystemProvider.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.gravitino.catalog.hadoop.fs;
+
+import static org.apache.gravitino.connector.BaseCatalog.CATALOG_BYPASS_PREFIX;
+
+import java.io.IOException;
+import java.util.Map;
+import javax.annotation.Nonnull;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+
+public class HDFSFileSystemProvider implements FileSystemProvider {
+
+  @Override
+  public FileSystem getFileSystem(@Nonnull Path path, @Nonnull Map<String, 
String> config)
+      throws IOException {
+    Configuration configuration = new Configuration();
+    config.forEach(
+        (k, v) -> {
+          configuration.set(k.replace(CATALOG_BYPASS_PREFIX, ""), v);
+        });
+    return DistributedFileSystem.newInstance(path.toUri(), configuration);
+  }
+
+  @Override
+  public String scheme() {
+    return "hdfs";
+  }
+
+  @Override
+  public String name() {
+    return "HDFSFileSystemProvider";

Review Comment:
   maybe like "builtin-hdfs", "builtin-local"?



##########
catalogs/catalog-hadoop/src/main/java/org/apache/gravitino/catalog/hadoop/fs/FileSystemUtils.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.gravitino.catalog.hadoop.fs;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.Set;
+
+public class FileSystemUtils {
+
+  private FileSystemUtils() {}
+
+  public static Map<String, FileSystemProvider> getFileSystemProviders(String 
fileSystemProviders) {
+    Map<String, FileSystemProvider> resultMap = Maps.newHashMap();
+    ServiceLoader<FileSystemProvider> allFileSystemProviders =
+        ServiceLoader.load(FileSystemProvider.class);
+
+    Set<String> providersInUses =
+        fileSystemProviders != null
+            ? Arrays.stream(fileSystemProviders.split(","))
+                .map(String::trim)
+                .collect(java.util.stream.Collectors.toSet())
+            : Sets.newHashSet();
+
+    // Always add the built-in LocalFileSystemProvider and 
HDFSFileSystemProvider to the catalog.
+    providersInUses.add(LocalFileSystemProvider.class.getSimpleName());
+    providersInUses.add(HDFSFileSystemProvider.class.getSimpleName());
+
+    allFileSystemProviders.forEach(
+        fileSystemProvider -> {
+          if 
(providersInUses.contains(fileSystemProvider.getClass().getSimpleName())) {
+            if (resultMap.containsKey(fileSystemProvider.scheme())) {
+              throw new UnsupportedOperationException(
+                  String.format(
+                      "File system provider with scheme '%s' already exists in 
the use provider list "
+                          + "Please make sure the file system provider scheme 
is unique.",
+                      fileSystemProvider.name()));
+            }
+
+            resultMap.put(fileSystemProvider.scheme(), fileSystemProvider);
+          }
+        });
+
+    return resultMap;
+  }
+
+  public static FileSystemProvider getFileSystemProviderByName(
+      Map<String, FileSystemProvider> fileSystemProviders, String 
defaultFileSystemProvider) {
+    return fileSystemProviders.entrySet().stream()
+        .filter(entry -> 
entry.getValue().name().equals(defaultFileSystemProvider))

Review Comment:
   Why it equals "defaultFileSystemProvider"?



-- 
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: commits-unsubscr...@gravitino.apache.org

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

Reply via email to