Skip to content
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

[HUDI-5000] Support schema evolution for Hive/presto #6989

Merged
merged 11 commits into from
Oct 28, 2022
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,9 @@ public RecordReader<NullWritable, ArrayWritable> getRecordReader(final InputSpli
return createBootstrappingRecordReader(split, job, reporter);
}

// adapt schema evolution
new SchemaEvolutionContext(split, job).doEvolutionForParquetFormat();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to fallback, if schemEvolution failed

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think it's worth paying attention, because here only a runtime exception is thrown when schema evolution is needed and the evolution fails.


if (LOG.isDebugEnabled()) {
LOG.debug("EMPLOYING DEFAULT RECORD READER - " + split);
}
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.hadoop.SchemaEvolutionContext;
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;

import org.apache.avro.Schema;
Expand Down Expand Up @@ -55,6 +57,7 @@ public abstract class AbstractRealtimeRecordReader {
private Schema writerSchema;
private Schema hiveSchema;
private HoodieTableMetaClient metaClient;
protected SchemaEvolutionContext schemaEvolutionContext;

public AbstractRealtimeRecordReader(RealtimeSplit split, JobConf job) {
this.split = split;
Expand All @@ -69,7 +72,12 @@ public AbstractRealtimeRecordReader(RealtimeSplit split, JobConf job) {
}
this.usesCustomPayload = usesCustomPayload(metaClient);
LOG.info("usesCustomPayload ==> " + this.usesCustomPayload);
init();
schemaEvolutionContext = new SchemaEvolutionContext(split, job, Option.of(metaClient));
if (schemaEvolutionContext.internalSchemaOption.isPresent()) {
schemaEvolutionContext.doEvolutionForRealtimeInputFormat(this);
} else {
init();
}
} catch (Exception e) {
throw new HoodieException("Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e);
}
Expand Down Expand Up @@ -99,7 +107,7 @@ private void init() throws Exception {
jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR), partitioningFields);

Map<String, Field> schemaFieldsMap = HoodieRealtimeRecordReaderUtils.getNameToFieldMap(writerSchema);
hiveSchema = constructHiveOrderedSchema(writerSchema, schemaFieldsMap);
hiveSchema = constructHiveOrderedSchema(writerSchema, schemaFieldsMap, jobConf.get(hive_metastoreConstants.META_TABLE_COLUMNS));
// TODO(vc): In the future, the reader schema should be updated based on log files & be able
// to null out fields not present before

Expand All @@ -108,10 +116,7 @@ private void init() throws Exception {
split.getDeltaLogPaths(), split.getPath(), projectionFields));
}

private Schema constructHiveOrderedSchema(Schema writerSchema, Map<String, Field> schemaFieldsMap) {
// Get all column names of hive table
String hiveColumnString = jobConf.get(hive_metastoreConstants.META_TABLE_COLUMNS);
LOG.info("Hive Columns : " + hiveColumnString);
public Schema constructHiveOrderedSchema(Schema writerSchema, Map<String, Field> schemaFieldsMap, String hiveColumnString) {
String[] hiveColumns = hiveColumnString.split(",");
LOG.info("Hive Columns : " + hiveColumnString);
List<Field> hiveSchemaFields = new ArrayList<>();
Expand Down Expand Up @@ -154,4 +159,16 @@ public RealtimeSplit getSplit() {
public JobConf getJobConf() {
return jobConf;
}

public void setReaderSchema(Schema readerSchema) {
this.readerSchema = readerSchema;
}

public void setWriterSchema(Schema writerSchema) {
this.writerSchema = writerSchema;
}

public void setHiveSchema(Schema hiveSchema) {
this.hiveSchema = hiveSchema;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;

Expand All @@ -44,7 +45,7 @@
import java.util.Map;
import java.util.Set;

class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader
public class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader
implements RecordReader<NullWritable, ArrayWritable> {

private static final Logger LOG = LogManager.getLogger(AbstractRealtimeRecordReader.class);
Expand Down Expand Up @@ -92,6 +93,7 @@ private HoodieMergedLogRecordScanner getMergedLogRecordScanner() throws IOExcept
.withBitCaskDiskMapCompressionEnabled(jobConf.getBoolean(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(),
HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()))
.withUseScanV2(jobConf.getBoolean(HoodieRealtimeConfig.USE_LOG_RECORD_READER_SCAN_V2, false))
.withInternalSchema(schemaEvolutionContext.internalSchemaOption.orElse(InternalSchema.getEmptyInternalSchema()))
.build();
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,155 @@
/*
* 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.functional;

import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
import org.apache.hadoop.hive.serde.serdeConstants;
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
import org.apache.hadoop.mapred.FileInputFormat;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hudi.HoodieSparkUtils;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
import org.apache.hudi.hadoop.SchemaEvolutionContext;
import org.apache.hudi.hadoop.realtime.HoodieEmptyRecordReader;
import org.apache.hudi.hadoop.realtime.HoodieRealtimeRecordReader;
import org.apache.hudi.hadoop.realtime.RealtimeCompactedRecordReader;
import org.apache.hudi.hadoop.realtime.RealtimeSplit;
import org.apache.spark.SparkConf;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.hudi.HoodieSparkSessionExtension;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import static org.junit.jupiter.api.Assertions.assertEquals;

import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat;

import java.io.File;
import java.util.Date;

@Tag("functional")
public class TestHiveTableSchemaEvolution {

private SparkSession sparkSession = null;

@BeforeEach
public void setUp() {
initSparkContexts("HiveSchemaEvolution");
}

private void initSparkContexts(String appName) {
SparkConf sparkConf = new SparkConf();
if (HoodieSparkUtils.gteqSpark3_2()) {
sparkConf.set("spark.sql.catalog.spark_catalog",
"org.apache.spark.sql.hudi.catalog.HoodieCatalog");
}
sparkSession = SparkSession.builder().appName(appName)
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.withExtensions(new HoodieSparkSessionExtension())
.config("hoodie.insert.shuffle.parallelism", "4")
.config("hoodie.upsert.shuffle.parallelism", "4")
.config("hoodie.delete.shuffle.parallelism", "4")
.config("hoodie.support.write.lock", "false")
.config("spark.sql.session.timeZone", "CTT")
.config("spark.sql.hive.convertMetastoreParquet", "false")
.config(sparkConf)
.master("local[1]").getOrCreate();
sparkSession.sparkContext().setLogLevel("ERROR");
}

@Test
public void testCopyOnWriteTableForHive() throws Exception {
String tableName = "huditest" + new Date().getTime();
File file = new File(System.getProperty("java.io.tmpdir") + tableName);
if (HoodieSparkUtils.gteqSpark3_1()) {
sparkSession.sql("set hoodie.schema.on.read.enable=true");
String path = new Path(file.getCanonicalPath()).toUri().toString();
sparkSession.sql("create table " + tableName + "(col0 int, col1 float, col2 string) using hudi options(type='cow', primaryKey='col0', preCombineField='col1') location '" + path + "'");
sparkSession.sql("insert into " + tableName + " values(1, 1.1, 'text')");
sparkSession.sql("alter table " + tableName + " alter column col1 type double");
sparkSession.sql("alter table " + tableName + " rename column col2 to aaa");

HoodieParquetInputFormat inputFormat = new HoodieParquetInputFormat();
JobConf jobConf = new JobConf();
inputFormat.setConf(jobConf);
FileInputFormat.setInputPaths(jobConf, path);
InputSplit[] splits = inputFormat.getSplits(jobConf, 1);
assertEvolutionResult("cow", splits[0], jobConf);
}
}

@Test
public void testMergeOnReadTableForHive() throws Exception {
String tableName = "huditest" + new Date().getTime();
File file = new File(System.getProperty("java.io.tmpdir") + tableName);
if (HoodieSparkUtils.gteqSpark3_1()) {
sparkSession.sql("set hoodie.schema.on.read.enable=true");
String path = new Path(file.getCanonicalPath()).toUri().toString();
sparkSession.sql("create table " + tableName + "(col0 int, col1 float, col2 string) using hudi options(type='cow', primaryKey='col0', preCombineField='col1') location '" + path + "'");
sparkSession.sql("insert into " + tableName + " values(1, 1.1, 'text')");
sparkSession.sql("insert into " + tableName + " values(2, 1.2, 'text2')");
sparkSession.sql("alter table " + tableName + " alter column col1 type double");
sparkSession.sql("alter table " + tableName + " rename column col2 to aaa");

HoodieRealtimeInputFormat inputFormat = new HoodieRealtimeInputFormat();
JobConf jobConf = new JobConf();
inputFormat.setConf(jobConf);
FileInputFormat.setInputPaths(jobConf, path);
InputSplit[] splits = inputFormat.getSplits(jobConf, 1);
assertEvolutionResult("mor", splits[0], jobConf);
}
}

private void assertEvolutionResult(String tableType, InputSplit split, JobConf jobConf) throws Exception {
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "col1,aaa");
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "6,7");
jobConf.set(serdeConstants.LIST_COLUMNS, "_hoodie_commit_time,_hoodie_commit_seqno,"
+ "_hoodie_record_key,_hoodie_partition_path,_hoodie_file_name,col0,col1,aaa");
jobConf.set(serdeConstants.LIST_COLUMN_TYPES, "string,string,string,string,string,int,double,string");

SchemaEvolutionContext schemaEvolutionContext = new SchemaEvolutionContext(split, jobConf);
if ("cow".equals(tableType)) {
schemaEvolutionContext.doEvolutionForParquetFormat();
} else {
// mot table
RealtimeSplit realtimeSplit = (RealtimeSplit) split;
RecordReader recordReader;
// for log only split, set the parquet reader as empty.
if (FSUtils.isLogFile(realtimeSplit.getPath())) {
recordReader = new HoodieRealtimeRecordReader(realtimeSplit, jobConf, new HoodieEmptyRecordReader(realtimeSplit, jobConf));
} else {
// create a RecordReader to be used by HoodieRealtimeRecordReader
recordReader = new MapredParquetInputFormat().getRecordReader(realtimeSplit, jobConf, null);
}
RealtimeCompactedRecordReader realtimeCompactedRecordReader = new RealtimeCompactedRecordReader(realtimeSplit, jobConf, recordReader);
// mor table also run with doEvolutionForParquetFormat in HoodieParquetInputFormat
schemaEvolutionContext.doEvolutionForParquetFormat();
schemaEvolutionContext.doEvolutionForRealtimeInputFormat(realtimeCompactedRecordReader);
}

assertEquals(jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR), "col1,col2");
assertEquals(jobConf.get(serdeConstants.LIST_COLUMNS), "_hoodie_commit_time,_hoodie_commit_seqno,"
+ "_hoodie_record_key,_hoodie_partition_path,_hoodie_file_name,col0,col1,col2");
assertEquals(jobConf.get(serdeConstants.LIST_COLUMN_TYPES), "string,string,string,string,string,int,double,string");
}
}
1 change: 1 addition & 0 deletions packaging/hudi-hadoop-mr-bundle/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,7 @@
<include>com.yammer.metrics:metrics-core</include>
<include>commons-io:commons-io</include>
<include>org.openjdk.jol:jol-core</include>
<include>com.github.ben-manes.caffeine:caffeine</include>
</includes>
</artifactSet>
<relocations combine.children="append">
Expand Down
1 change: 1 addition & 0 deletions packaging/hudi-presto-bundle/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@

<include>org.apache.parquet:parquet-avro</include>
<include>org.apache.avro:avro</include>
<include>com.github.ben-manes.caffeine:caffeine</include>
<include>org.codehaus.jackson:*</include>
<include>org.apache.commons:commons-lang3</include>
<include>org.apache.hbase:hbase-common</include>
Expand Down