[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:
@@ -175,11 +175,6 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
.withDocumentation("Schema string representing the latest schema of the table. Hudi passes this to "
|
||||
+ "implementations of evolution of schema");
|
||||
|
||||
public static final ConfigProperty<Boolean> SCHEMA_EVOLUTION_ENABLE = ConfigProperty
|
||||
.key("hoodie.schema.on.read.enable")
|
||||
.defaultValue(false)
|
||||
.withDocumentation("enable full schema evolution for hoodie");
|
||||
|
||||
public static final ConfigProperty<Boolean> ENABLE_INTERNAL_SCHEMA_CACHE = ConfigProperty
|
||||
.key("hoodie.schema.cache.enable")
|
||||
.defaultValue(false)
|
||||
@@ -929,11 +924,11 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
}
|
||||
|
||||
public boolean getSchemaEvolutionEnable() {
|
||||
return getBoolean(SCHEMA_EVOLUTION_ENABLE);
|
||||
return getBoolean(HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE);
|
||||
}
|
||||
|
||||
public void setSchemaEvolutionEnable(boolean enable) {
|
||||
setValue(SCHEMA_EVOLUTION_ENABLE, String.valueOf(enable));
|
||||
setValue(HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE, String.valueOf(enable));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -2175,7 +2170,7 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
}
|
||||
|
||||
public Builder withSchemaEvolutionEnable(boolean enable) {
|
||||
writeConfig.setValue(SCHEMA_EVOLUTION_ENABLE, String.valueOf(enable));
|
||||
writeConfig.setValue(HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE, String.valueOf(enable));
|
||||
return this;
|
||||
}
|
||||
|
||||
|
||||
@@ -0,0 +1,33 @@
|
||||
/*
|
||||
* 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.util
|
||||
|
||||
/**
|
||||
* Utility allowing for seamless conversion b/w Java/Scala functional primitives
|
||||
*/
|
||||
object JFunction {
|
||||
|
||||
def toScala[T, R](f: java.util.function.Function[T, R]): T => R =
|
||||
(t: T) => f.apply(t)
|
||||
|
||||
def toJava[T](f: T => Unit): java.util.function.Consumer[T] =
|
||||
new java.util.function.Consumer[T] {
|
||||
override def accept(t: T): Unit = f.apply(t)
|
||||
}
|
||||
|
||||
}
|
||||
@@ -28,14 +28,13 @@ import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.catalyst.parser.ParserInterface
|
||||
import org.apache.spark.sql.catalyst.plans.JoinType
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, SubqueryAlias}
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.execution.datasources.{FilePartition, LogicalRelation, PartitionedFile, SparkParsePartitionUtil}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.types.DataType
|
||||
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, Row, SparkSession}
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.rules.Rule
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
|
||||
import java.util.Locale
|
||||
|
||||
@@ -141,8 +140,8 @@ trait SparkAdapter extends Serializable {
|
||||
maxSplitBytes: Long): Seq[FilePartition]
|
||||
|
||||
def isHoodieTable(table: LogicalPlan, spark: SparkSession): Boolean = {
|
||||
tripAlias(table) match {
|
||||
case LogicalRelation(_, _, Some(tbl), _) => isHoodieTable(tbl)
|
||||
unfoldSubqueryAliases(table) match {
|
||||
case LogicalRelation(_, _, Some(table), _) => isHoodieTable(table)
|
||||
case relation: UnresolvedRelation =>
|
||||
isHoodieTable(toTableIdentifier(relation), spark)
|
||||
case _=> false
|
||||
@@ -162,20 +161,15 @@ trait SparkAdapter extends Serializable {
|
||||
isHoodieTable(table)
|
||||
}
|
||||
|
||||
def tripAlias(plan: LogicalPlan): LogicalPlan = {
|
||||
protected def unfoldSubqueryAliases(plan: LogicalPlan): LogicalPlan = {
|
||||
plan match {
|
||||
case SubqueryAlias(_, relation: LogicalPlan) =>
|
||||
tripAlias(relation)
|
||||
unfoldSubqueryAliases(relation)
|
||||
case other =>
|
||||
other
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create customresolutionRule to deal with alter command for hudi.
|
||||
*/
|
||||
def createResolveHudiAlterTableCommand(sparkSession: SparkSession): Rule[LogicalPlan]
|
||||
|
||||
/**
|
||||
* Create instance of [[ParquetFileFormat]]
|
||||
*/
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
Reference in New Issue
Block a user