stream2000 commented on code in PR #9558:
URL: https://github.com/apache/hudi/pull/9558#discussion_r1312487436


##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/multitable/MultiTableServiceUtils.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.hudi.utilities.multitable;
+
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.stream.Collectors;
+
+import static 
org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME;
+
+/**
+ * Utils for executing multi-table services
+ */
+public class MultiTableServiceUtils {
+
+  public static class Constants {
+    public static final String TABLES_TO_BE_SERVED_PROP = 
"hoodie.tableservice.tablesToServe";
+
+    public static final String COMMA_SEPARATOR = ",";
+
+    private static final int DEFAULT_LISTING_PARALLELISM = 1500;
+  }
+
+  public static List<String> getTablesToBeServedFromProps(TypedProperties 
properties) {
+    String combinedTablesString = 
properties.getString(Constants.TABLES_TO_BE_SERVED_PROP);
+    if (combinedTablesString == null) {
+      return new ArrayList<>();
+    }
+    String[] tablesArray = 
combinedTablesString.split(Constants.COMMA_SEPARATOR);
+    return Arrays.asList(tablesArray);
+  }
+
+  public static List<String> findHoodieTablesUnderPath(JavaSparkContext jsc, 
String pathStr) {
+    Path rootPath = new Path(pathStr);
+    SerializableConfiguration conf = new 
SerializableConfiguration(jsc.hadoopConfiguration());
+    if (isHoodieTable(rootPath, conf.get())) {
+      return Collections.singletonList(pathStr);
+    }
+
+    HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
+    List<String> hoodieTablePaths = new CopyOnWriteArrayList<>();
+    List<Path> pathsToList = new CopyOnWriteArrayList<>();
+    pathsToList.add(rootPath);
+    int listingParallelism = Math.min(Constants.DEFAULT_LISTING_PARALLELISM, 
pathsToList.size());
+
+    while (!pathsToList.isEmpty()) {
+      // List all directories in parallel
+      List<FileStatus[]> dirToFileListing = engineContext.map(pathsToList, 
path -> {
+        FileSystem fileSystem = path.getFileSystem(conf.get());
+        return fileSystem.listStatus(path);
+      }, listingParallelism);
+      pathsToList.clear();
+
+      // if current dictionary contains meta folder(.hoodie), add it to 
result. Otherwise, add it to queue
+      List<FileStatus> dirs = dirToFileListing.stream().flatMap(Arrays::stream)
+          .filter(FileStatus::isDirectory)
+          .collect(Collectors.toList());
+
+      if (!dirs.isEmpty()) {
+        List<Pair<FileStatus, Integer>> dirResults = engineContext.map(dirs, 
fileStatus -> {
+          if (isHoodieTable(fileStatus.getPath(), conf.get())) {

Review Comment:
   Nice catch~ It's not a good design that uses hard-coded magic number, 
already updated the magic number to meaningful enum constants. 
   
   Updated to: 
   ```java
     /**
      * Type of directories when searching hoodie tables under path
      */
     enum DirType {
       HOODIE_TABLE, // previous 0
       NORMAL_DIR, // previous 1 
       META_FOLDER  // previous 2 
     }
   ```
   



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/multitable/HoodieMultiTableServicesMain.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.hudi.utilities.multitable;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.utilities.HoodieCompactor;
+import org.apache.hudi.utilities.IdentitySplitter;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.streamer.HoodieStreamer;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.StringJoiner;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * Main function for executing multi-table services
+ */
+public class HoodieMultiTableServicesMain {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieStreamer.class);
+  final Config cfg;
+  final TypedProperties props;
+
+  private final JavaSparkContext jsc;
+
+  private ScheduledExecutorService executorService;
+
+  private void batchRunTableServices(List<String> tablePaths) throws 
InterruptedException, ExecutionException {
+    ExecutorService executorService = 
Executors.newFixedThreadPool(cfg.poolSize);
+    List<CompletableFuture<Void>> futures = tablePaths.stream()
+        .map(basePath -> CompletableFuture.runAsync(
+            () -> MultiTableServiceUtils.buildTableServicePipeline(jsc, 
basePath, cfg, props).execute(),

Review Comment:
   If no services is enabled, the `TableServicePipeline` will contain no task 
and the execute method will return directly. So it's OK to keep current design.



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/multitable/HoodieMultiTableServicesMain.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.hudi.utilities.multitable;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.utilities.HoodieCompactor;
+import org.apache.hudi.utilities.IdentitySplitter;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.streamer.HoodieStreamer;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.StringJoiner;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * Main function for executing multi-table services
+ */
+public class HoodieMultiTableServicesMain {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieStreamer.class);

Review Comment:
   Sure, I missed it when migrated it from `HoodieStreamer`. Will check other 
classes also.



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

Reply via email to