-
Notifications
You must be signed in to change notification settings - Fork 2.5k
fix: Enable schema merging for incremental and dfs sources #18385
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
8938ef3
32cfdc9
cfb3309
b641584
9acf533
0bea318
91d98a0
b81c5a5
f609bff
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,51 @@ | ||
| /* | ||
| * 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.config; | ||
|
|
||
| import org.apache.hudi.common.config.ConfigClassProperty; | ||
| import org.apache.hudi.common.config.ConfigGroups; | ||
| import org.apache.hudi.common.config.ConfigProperty; | ||
| import org.apache.hudi.common.config.HoodieConfig; | ||
|
|
||
| import javax.annotation.concurrent.Immutable; | ||
|
|
||
| import static org.apache.hudi.common.util.ConfigUtils.STREAMER_CONFIG_PREFIX; | ||
|
|
||
| /** | ||
| * ORC DFS Source Configs | ||
| */ | ||
| @Immutable | ||
| @ConfigClassProperty(name = "ORC DFS Source Configs", | ||
| groupName = ConfigGroups.Names.HUDI_STREAMER, | ||
| subGroupName = ConfigGroups.SubGroupNames.DELTA_STREAMER_SOURCE, | ||
| description = "Configurations controlling the behavior of ORC DFS source in Hudi Streamer.") | ||
| public class ORCDFSSourceConfig extends HoodieConfig { | ||
|
|
||
| public static final ConfigProperty<Boolean> ORC_DFS_MERGE_SCHEMA = ConfigProperty | ||
| .key(STREAMER_CONFIG_PREFIX + "source.orc.dfs.merge.schema.enable") | ||
| .defaultValue(true) | ||
| .markAdvanced() | ||
| .sinceVersion("1.2.0") | ||
| .withDocumentation("Whether to merge schema across ORC files within a single read. " | ||
| + "Defaults to true: heterogeneous-schema source files (e.g. during bootstrap or " | ||
| + "evolving producers) get a unioned schema instead of silently dropping columns " | ||
| + "that exist only in some files. Requires spark.sql.orc.impl=native (default since " | ||
| + "Spark 2.4); the option is silently ignored under spark.sql.orc.impl=hive."); | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -71,6 +71,7 @@ | |
| import static org.apache.hudi.common.util.ConfigUtils.containsConfigProperty; | ||
| import static org.apache.hudi.common.util.ConfigUtils.getBooleanWithAltKeys; | ||
| import static org.apache.hudi.common.util.ConfigUtils.getStringWithAltKeys; | ||
| import static org.apache.hudi.utilities.config.CloudSourceConfig.CLOUD_INCREMENTAL_MERGE_SCHEMA; | ||
| import static org.apache.hudi.utilities.config.CloudSourceConfig.CLOUD_DATAFILE_EXTENSION; | ||
| import static org.apache.hudi.utilities.config.CloudSourceConfig.IGNORE_RELATIVE_PATH_PREFIX; | ||
| import static org.apache.hudi.utilities.config.CloudSourceConfig.IGNORE_RELATIVE_PATH_SUBSTR; | ||
|
|
@@ -281,7 +282,7 @@ public Option<Dataset<Row>> loadAsDataset(SparkSession spark, List<CloudObjectMe | |
| if (isNullOrEmpty(cloudObjectMetadata)) { | ||
| return Option.empty(); | ||
| } | ||
| DataFrameReader reader = spark.read().format(fileFormat); | ||
| DataFrameReader reader = applyMergeSchemaOption(spark.read().format(fileFormat), fileFormat); | ||
| String datasourceOpts = getStringWithAltKeys(properties, CloudSourceConfig.SPARK_DATASOURCE_OPTIONS, true); | ||
|
|
||
| StructType rowSchema = null; | ||
|
|
@@ -546,6 +547,34 @@ private static Option<String> getPropVal(TypedProperties props, ConfigProperty<S | |
| return Option.empty(); | ||
| } | ||
|
|
||
| /** | ||
| * Enables Spark {@code mergeSchema} for cloud object batches of Parquet or ORC files when configured, so | ||
| * heterogeneous files in one sync round share a merged struct type. Applied before user | ||
| * {@link CloudSourceConfig#SPARK_DATASOURCE_OPTIONS} so explicit reader options can override. | ||
| * | ||
| * <p>Spark's native Parquet reader honors {@code mergeSchema} on all supported versions. Spark's native ORC | ||
| * reader honors it on Spark 3.0+ (the native ORC impl is the default since Spark 2.4); on older runtimes the | ||
| * option is silently ignored, which is harmless. | ||
| */ | ||
| private DataFrameReader applyMergeSchemaOption(DataFrameReader reader, String fileFormat) { | ||
| if (!isParquetOrOrcFileFormat(fileFormat)) { | ||
| return reader; | ||
| } | ||
| if (!getBooleanWithAltKeys(properties, CLOUD_INCREMENTAL_MERGE_SCHEMA)) { | ||
| return reader; | ||
| } | ||
| return reader.option("mergeSchema", "true"); | ||
| } | ||
|
|
||
| // Package-private for unit testing — see TestCloudObjectsSelectorCommon. | ||
| static boolean isParquetOrOrcFileFormat(String fileFormat) { | ||
| if (fileFormat == null) { | ||
| return false; | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 🤖 nit: could you rename - AI-generated; verify before applying. React 👍/👎 to flag quality. |
||
| String f = fileFormat.trim(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 🤖 nit: the single-character name - AI-generated; verify before applying. React 👍/👎 to flag quality. |
||
| return "parquet".equalsIgnoreCase(f) || "orc".equalsIgnoreCase(f); | ||
| } | ||
|
|
||
| public enum Type { | ||
| S3, | ||
| GCS | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.