1
0

[HUDI-4178] Addressing performance regressions in Spark DataSourceV2 Integration (#5737)

There are multiple issues with our current DataSource V2 integrations: b/c we advertise Hudi tables as V2, Spark expects it to implement certain APIs which are not implemented at the moment, instead we're using custom Resolution rule (in HoodieSpark3Analysis) to instead manually fallback to V1 APIs.  This commit fixes the issue by reverting DSv2 APIs and making Spark use V1, except for schema evaluation logic.
This commit is contained in:
Alexey Kudinkin
2022-06-07 16:30:46 -07:00
committed by GitHub
parent 1349b596a1
commit 35afdb4316
28 changed files with 374 additions and 256 deletions

View File

@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hudi.HoodieConversionUtils;
import org.apache.hudi.avro.model.HoodieActionInstant;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
@@ -73,12 +74,14 @@ import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.WorkloadStat;
import org.apache.hudi.timeline.service.TimelineService;
import org.apache.hudi.util.JFunction;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SQLContext;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.SparkSessionExtensions;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
@@ -98,6 +101,7 @@ import java.util.Properties;
import java.util.Random;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.stream.Collectors;
@@ -145,6 +149,10 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
FileSystem.closeAll();
}
protected Option<Consumer<SparkSessionExtensions>> getSparkSessionExtensionsInjector() {
return Option.empty();
}
@BeforeEach
public void setTestMethodName(TestInfo testInfo) {
if (testInfo.getTestMethod().isPresent()) {
@@ -186,16 +194,32 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
* @param appName The specified application name.
*/
protected void initSparkContexts(String appName) {
Option<Consumer<SparkSessionExtensions>> sparkSessionExtensionsInjector =
getSparkSessionExtensionsInjector();
if (sparkSessionExtensionsInjector.isPresent()) {
// In case we need to inject extensions into Spark Session, we have
// to stop any session that might still be active and since Spark will try
// to re-use it
HoodieConversionUtils.toJavaOption(SparkSession.getActiveSession())
.ifPresent(SparkSession::stop);
}
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest(appName + "#" + testMethodName));
jsc.setLogLevel("ERROR");
hadoopConf = jsc.hadoopConfiguration();
// SQLContext stuff
sqlContext = new SQLContext(jsc);
hadoopConf = jsc.hadoopConfiguration();
context = new HoodieSparkEngineContext(jsc);
hadoopConf = context.getHadoopConf().get();
sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate();
sparkSession = SparkSession.builder()
.withExtensions(JFunction.toScala(sparkSessionExtensions -> {
sparkSessionExtensionsInjector.ifPresent(injector -> injector.accept(sparkSessionExtensions));
return null;
}))
.config(jsc.getConf())
.getOrCreate();
sqlContext = new SQLContext(sparkSession);
}
/**