vinothchandar commented on a change in pull request #1827:
URL: https://github.com/apache/hudi/pull/1827#discussion_r492262964



##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/HoodieSparkEngineContext.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.common;
+
+import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SQLContext;
+
+/**
+ * A Spark engine implementation of HoodieEngineContext.
+ */
+public class HoodieSparkEngineContext extends HoodieEngineContext {

Review comment:
       can we implement versiosn of `map`, `flatMap`, `forEach` here which use 
`javaSparkContext.parallelize()` ? It would be good to keep this PR free of any 
changes in terms of whether we are executing the deletes/lists in parallel or 
in serial. 

##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/SparkMarkerFiles.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.table;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hudi.bifunction.wrapper.ThrowingFunction;
+import org.apache.hudi.common.HoodieEngineContext;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.hudi.bifunction.wrapper.BiFunctionWrapper.throwingConsumerWrapper;
+import static 
org.apache.hudi.bifunction.wrapper.BiFunctionWrapper.throwingFlatMapWrapper;
+
+public class SparkMarkerFiles extends BaseMarkerFiles {
+
+  private static final Logger LOG = 
LogManager.getLogger(SparkMarkerFiles.class);
+
+  public SparkMarkerFiles(HoodieTable table, String instantTime) {
+    super(table, instantTime);
+  }
+
+  public SparkMarkerFiles(FileSystem fs, String basePath, String 
markerFolderPath, String instantTime) {
+    super(fs, basePath, markerFolderPath, instantTime);
+  }
+
+  @Override
+  public boolean deleteMarkerDir(HoodieEngineContext context, int parallelism) 
{
+    try {
+      if (fs.exists(markerDirPath)) {
+        FileStatus[] fileStatuses = fs.listStatus(markerDirPath);
+        List<String> markerDirSubPaths = Arrays.stream(fileStatuses)
+            .map(fileStatus -> fileStatus.getPath().toString())
+            .collect(Collectors.toList());
+
+        if (markerDirSubPaths.size() > 0) {
+          SerializableConfiguration conf = new 
SerializableConfiguration(fs.getConf());
+          context.foreach(markerDirSubPaths, 
throwingConsumerWrapper(subPathStr -> {
+            Path subPath = new Path(subPathStr);
+            FileSystem fileSystem = subPath.getFileSystem(conf.get());
+            fileSystem.delete(subPath, true);
+          }));
+        }
+
+        boolean result = fs.delete(markerDirPath, true);
+        LOG.info("Removing marker directory at " + markerDirPath);
+        return result;
+      }
+    } catch (IOException ioe) {
+      throw new HoodieIOException(ioe.getMessage(), ioe);
+    }
+    return false;
+  }
+
+  @Override
+  public Set<String> createdAndMergedDataPaths(HoodieEngineContext context, 
int parallelism) throws IOException {

Review comment:
       we are not using parallelism here. This will lead to a perf regression 
w.r.t master.

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/HoodieEngineContext.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.hudi.common;
+
+import org.apache.hudi.client.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+
+import java.util.List;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Base class contains the context information needed by the engine at 
runtime. It will be extended by different
+ * engine implementation if needed.
+ */
+public class HoodieEngineContext {
+  /**
+   * A wrapped hadoop configuration which can be serialized.
+   */
+  private SerializableConfiguration hadoopConf;
+
+  private TaskContextSupplier taskContextSupplier;
+
+  public HoodieEngineContext(SerializableConfiguration hadoopConf, 
TaskContextSupplier taskContextSupplier) {
+    this.hadoopConf = hadoopConf;
+    this.taskContextSupplier = taskContextSupplier;
+  }
+
+  public SerializableConfiguration getHadoopConf() {
+    return hadoopConf;
+  }
+
+  public TaskContextSupplier getTaskContextSupplier() {
+    return taskContextSupplier;
+  }
+
+  public <I, O> List<O> map(List<I> data, Function<I, O> func) {

Review comment:
       Also these APIs should take in a `parallelism` parameter, no?

##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/SparkMarkerFiles.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.table;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hudi.bifunction.wrapper.ThrowingFunction;
+import org.apache.hudi.common.HoodieEngineContext;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.hudi.bifunction.wrapper.BiFunctionWrapper.throwingConsumerWrapper;
+import static 
org.apache.hudi.bifunction.wrapper.BiFunctionWrapper.throwingFlatMapWrapper;
+
+public class SparkMarkerFiles extends BaseMarkerFiles {

Review comment:
       Given this file is now free of Spark, we dont have the need of breaking 
these into base and child classes right.
   
   

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/HoodieEngineContext.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.hudi.common;
+
+import org.apache.hudi.client.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+
+import java.util.List;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Base class contains the context information needed by the engine at 
runtime. It will be extended by different
+ * engine implementation if needed.
+ */
+public class HoodieEngineContext {
+  /**
+   * A wrapped hadoop configuration which can be serialized.
+   */
+  private SerializableConfiguration hadoopConf;
+
+  private TaskContextSupplier taskContextSupplier;
+
+  public HoodieEngineContext(SerializableConfiguration hadoopConf, 
TaskContextSupplier taskContextSupplier) {
+    this.hadoopConf = hadoopConf;
+    this.taskContextSupplier = taskContextSupplier;
+  }
+
+  public SerializableConfiguration getHadoopConf() {
+    return hadoopConf;
+  }
+
+  public TaskContextSupplier getTaskContextSupplier() {
+    return taskContextSupplier;
+  }
+
+  public <I, O> List<O> map(List<I> data, Function<I, O> func) {

Review comment:
       I think we should leave this abstract and let the engines implement 
this?  even for Java. Its better to have a `HoodieJavaEngineContext`. From what 
I can see, this is not overridden in `HoodieSparkEngineContext` and thus we 
lose the parallel execution that we currently have with Spark with this change. 

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/HoodieEngineContext.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.hudi.common;
+
+import org.apache.hudi.client.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+
+import java.util.List;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Base class contains the context information needed by the engine at 
runtime. It will be extended by different
+ * engine implementation if needed.
+ */
+public class HoodieEngineContext {
+  /**
+   * A wrapped hadoop configuration which can be serialized.
+   */
+  private SerializableConfiguration hadoopConf;
+
+  private TaskContextSupplier taskContextSupplier;
+
+  public HoodieEngineContext(SerializableConfiguration hadoopConf, 
TaskContextSupplier taskContextSupplier) {
+    this.hadoopConf = hadoopConf;
+    this.taskContextSupplier = taskContextSupplier;
+  }
+
+  public SerializableConfiguration getHadoopConf() {
+    return hadoopConf;
+  }
+
+  public TaskContextSupplier getTaskContextSupplier() {
+    return taskContextSupplier;
+  }
+
+  public <I, O> List<O> map(List<I> data, Function<I, O> func) {

Review comment:
       @wangxianghu functionality wise, you are correct. it can be implemented 
just using Java. but, we do parallelization of different pieces of code e.g 
deletion of files in parallel using spark for a reason. It significantly speeds 
these up, for large tables. 
   
   All I am saying is to implement the `HoodieSparkEngineContext#map` like below
   
   ```
    public <I, O> List<O> map(List<I> data, Function<I, O> func, int 
parallelism) {
       return javaSparkContext.parallelize(data, 
parallelism).map(func).collect();
    }
   ```
   
   similarly for the other two methods. I don't see any issues with this. do 
you?




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

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


Reply via email to