wangxianghu commented on a change in pull request #1827: URL: https://github.com/apache/hudi/pull/1827#discussion_r492434405
########## 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. as we discussed before, parallelDo model need a function as input parameter, Unfortunately, different engines need different type function, its hard to align them in an abstract parallelDo method. so we agree to use ` java.util.function.Function` as the unified input function. in this way, there is no need to distinguish spark and flink, and the parallelism is not needed too. ########## 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. as we discussed before, parallelDo model need a function as input parameter, Unfortunately, different engines need different type function, its hard to align them in an abstract parallelDo method. so we agreed to use ` java.util.function.Function` as the unified input function. in this way, there is no need to distinguish spark and flink, and the parallelism is not needed too. ########## 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. as we discussed before, parallelDo model need a function as input parameter, Unfortunately, different engines need different type function, its hard to align them in an abstract parallelDo method. so we agreed to use ` java.util.function.Function` as the unified input function. in this way, there is no need to distinguish spark and flink, no need to make it abstract and the parallelism is not needed too. its just java, can be implemented directly. ########## 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. Yes, this is an example to show you the bi function, if you agree with this implementation, I'll rollback them in one class ########## 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? I know what you mean. ########## 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? I know what you mean. what I am saying is that the `func` in `HoodieSparkEngineContext#map` and `HoodieEngineContext#map` is not the same type. for `HoodieEngineContext#map` it is `java.util.function.Function`, for `HoodieSparkEngineContext#map` it is `org.apache.spark.api.java.function.Function`. `HoodieSparkEngineContext#map` can not override from `HoodieEngineContext#map` ########## 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: > Is it possible to take a `java.util.function.Function` and then within `HoodieSparkEngineContext#map` wrap that into a `org.apache.spark.api.java.function.Function` ? let me try ########## 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: > when the use passes in a regular lambda, into `rdd.map()`, this is what it gets converted into The serializable issue can be solved by introducing a seriableFuncition to replace `java.util.function.Function` ``` public interface SerializableFunction<I, O> extends Serializable { O call(I v1) throws Exception; } ``` `HoodieEngineContext` can be ``` public abstract class HoodieEngineContext { public abstract <I, O> List<O> map(List<I> data, SerializableFunction<I, O> func, int parallelism) ; } ``` `HoodieSparkEngineContext` can be ``` public class HoodieSparkEngineContext extends HoodieEngineContext { private static JavaSparkContext jsc; // tmp static { SparkConf conf = new SparkConf() .setMaster("local[4]") .set("spark.driver.host","localhost") .setAppName("HoodieSparkEngineContext"); jsc = new JavaSparkContext(conf); } @Override public <I, O> List<O> map(List<I> data, SerializableFunction<I, O> func, int parallelism) { return jsc.parallelize(data, parallelism).map(func::call).collect(); } } ``` this works :) ########## 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 This is awesome. Hopefully this can reduce the amount of code you need to write for Flink significantly. `TestMarkerFiles` seems to pass, so guess the serialization etc is working as expected. > > We can go ahead with doing more files in this approach and remerge the base/child classes back as much as possible. cc @leesf @yanghua as well in case they have more things to add. > > cc @bvaradar as well as FYI Yes, it also reduce tons of code in the refactoring. I'm working on it ,hope to finish it today or tomorrow ---------------------------------------------------------------- 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: [email protected]
