Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -211,7 +211,7 @@ private static void setPropsForInputFormat(FileInputFormat inputFormat, JobConf
conf.set(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "datestr");
conf.set(hive_metastoreConstants.META_TABLE_COLUMN_TYPES, hiveColumnTypesWithDatestr);
conf.set(IOConstants.COLUMNS, hiveColumnNames);
conf.get(IOConstants.COLUMNS_TYPES, hiveColumnTypesWithDatestr);
conf.set(IOConstants.COLUMNS_TYPES, hiveColumnTypesWithDatestr);

// Hoodie Input formats are also configurable
Configurable configurable = (Configurable)inputFormat;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -831,8 +831,9 @@ private FlinkOptions() {
.key("hive_sync.support_timestamp")
.booleanType()
.defaultValue(true)
.withDescription("INT64 with original type TIMESTAMP_MICROS is converted to hive timestamp type.\n"
+ "Disabled by default for backward compatibility.");
.withDescription("'INT64' with original type TIMESTAMP_MICROS is converted to hive 'timestamp' type. "
+ "From 0.13.0, 'timestamp' type will be supported and also can be disabled by this variable. "
+ "Previous versions keep being disabled by default.");

public static final ConfigOption<String> HIVE_SYNC_TABLE_PROPERTIES = ConfigOptions
.key("hive_sync.table_properties")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -340,8 +340,9 @@ public class FlinkStreamerConfig extends Configuration {
public Boolean hiveSyncSkipRoSuffix = false;

@Parameter(names = {"--hive-sync-support-timestamp"}, description = "INT64 with original type TIMESTAMP_MICROS is converted to hive timestamp type.\n"
+ "Disabled by default for backward compatibility.")
public Boolean hiveSyncSupportTimestamp = false;
+ "From 0.13.0, 'timestamp' type will be supported and also can be disabled by this variable. "
+ "Previous versions keep being disabled by default.")
public Boolean hiveSyncSupportTimestamp = true;


/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,4 +109,22 @@ public static List<Pair<String,String>> getIOColumnNameAndTypes(Configuration co
.collect(Collectors.toList());
}

/**
* if schema contains timestamp columns, this method is used for compatibility when there is no timestamp fields
* We expect 3 cases to use parquet-avro reader to read timestamp column:
* 1. read columns contain timestamp type
* 2. no read columns and exists original columns contain timestamp type
* 3. no read columns and no original columns, but avro schema contains type
*/
public static boolean supportTimestamp(Configuration conf) {
List<String> reads = Arrays.asList(getReadColumnNames(conf));
if (reads.isEmpty()) {
return getIOColumnTypes(conf).contains("timestamp");
}
List<String> names = getIOColumns(conf);
List<String> types = getIOColumnTypes(conf);
return types.isEmpty() || IntStream.range(0, names.size()).filter(i -> reads.contains(names.get(i)))
Copy link
Member

Choose a reason for hiding this comment

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

not quite sure about types.isEmpty() being true then support timestamp?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@xushiyan
I was trying to think of a worst case scenario if I couldn't find the columns. I had encountered a similar problem with some of the test cases when running the azure test cases, so I've added this treatment here.

.anyMatch(i -> types.get(i).equals("timestamp"));
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.hudi.hadoop;

import org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper;
import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
import org.apache.hadoop.hive.ql.plan.TableScanDesc;
import org.apache.hadoop.io.ArrayWritable;
Expand All @@ -29,11 +30,16 @@
import org.apache.hadoop.mapred.Reporter;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.hadoop.avro.HoodieTimestampAwareParquetInputFormat;
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.hadoop.ParquetInputFormat;

import java.io.IOException;
import java.lang.reflect.Constructor;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
Expand All @@ -49,12 +55,36 @@ public class HoodieParquetInputFormat extends HoodieParquetInputFormatBase {

private static final Logger LOG = LogManager.getLogger(HoodieParquetInputFormat.class);

private boolean supportAvroRead = false;

public HoodieParquetInputFormat() {
super(new HoodieCopyOnWriteTableInputFormat());
initAvroInputFormat();
}

protected HoodieParquetInputFormat(HoodieCopyOnWriteTableInputFormat delegate) {
super(delegate);
initAvroInputFormat();
}

/**
* Spark2 use `parquet.hadoopParquetInputFormat` in `com.twitter:parquet-hadoop-bundle`.
* So that we need to distinguish the constructions of classes with
* `parquet.hadoopParquetInputFormat` or `org.apache.parquet.hadoop.ParquetInputFormat`.
* If we use `org.apache.parquet:parquet-hadoop`, we can use `HudiAvroParquetInputFormat`
* in Hive or Spark3 to get timestamp with correct type.
*/
private void initAvroInputFormat() {
try {
Constructor[] constructors = ParquetRecordReaderWrapper.class.getConstructors();
if (Arrays.stream(constructors)
.anyMatch(c -> c.getParameterCount() > 0 && c.getParameterTypes()[0]
.getName().equals(ParquetInputFormat.class.getName()))) {
supportAvroRead = true;
}
} catch (SecurityException e) {
throw new HoodieException("Failed to check if support avro reader: " + e.getMessage(), e);
}
}

@Override
Expand Down Expand Up @@ -89,7 +119,15 @@ public RecordReader<NullWritable, ArrayWritable> getRecordReader(final InputSpli
private RecordReader<NullWritable, ArrayWritable> getRecordReaderInternal(InputSplit split,
JobConf job,
Reporter reporter) throws IOException {
return super.getRecordReader(split, job, reporter);
try {
if (supportAvroRead && HoodieColumnProjectionUtils.supportTimestamp(job)) {
return new ParquetRecordReaderWrapper(new HoodieTimestampAwareParquetInputFormat(), split, job, reporter);
} else {
return super.getRecordReader(split, job, reporter);
}
} catch (final InterruptedException | IOException e) {
throw new RuntimeException("Cannot create a RecordReaderWrapper", e);
}
}

private RecordReader<NullWritable, ArrayWritable> createBootstrappingRecordReader(InputSplit split,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
/*
* 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.hadoop.avro;

import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.ArrayWritable;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hudi.hadoop.HoodieColumnProjectionUtils;
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.format.converter.ParquetMetadataConverter;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.hadoop.ParquetInputSplit;
import org.apache.parquet.hadoop.ParquetRecordReader;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
import org.apache.parquet.schema.MessageType;

import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;

import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter;

public class HoodieAvroParquetReader extends RecordReader<Void, ArrayWritable> {
Copy link
Contributor

Choose a reason for hiding this comment

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

@cdmikechen can you please help me understand why we need custom ParquetReader?


private final ParquetRecordReader<GenericData.Record> parquetRecordReader;

public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf) throws IOException {
AvroReadSupport avroReadSupport = new AvroReadSupport<>();
// if exists read columns, we need to filter columns.
List<String> readColNames = Arrays.asList(HoodieColumnProjectionUtils.getReadColumnNames(conf));
if (!readColNames.isEmpty()) {
// get base schema
ParquetMetadata fileFooter =
ParquetFileReader.readFooter(conf, ((ParquetInputSplit) inputSplit).getPath(), ParquetMetadataConverter.NO_FILTER);
MessageType messageType = fileFooter.getFileMetaData().getSchema();
Schema baseSchema = new AvroSchemaConverter(conf).convert(messageType);
// filter schema for reading
final Schema filterSchema = Schema.createRecord(baseSchema.getName(),
baseSchema.getDoc(), baseSchema.getNamespace(), baseSchema.isError(),
baseSchema.getFields().stream()
.filter(f -> readColNames.contains(f.name()))
.map(f -> new Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal()))
.collect(Collectors.toList()));
avroReadSupport.setAvroReadSchema(conf, filterSchema);
avroReadSupport.setRequestedProjection(conf, filterSchema);
}
parquetRecordReader = new ParquetRecordReader<>(avroReadSupport, getFilter(conf));
}

@Override
public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
parquetRecordReader.initialize(split, context);
}

@Override
public boolean nextKeyValue() throws IOException, InterruptedException {
return parquetRecordReader.nextKeyValue();
}

@Override
public Void getCurrentKey() throws IOException, InterruptedException {
return parquetRecordReader.getCurrentKey();
}

@Override
public ArrayWritable getCurrentValue() throws IOException, InterruptedException {
GenericRecord record = parquetRecordReader.getCurrentValue();
return (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(record, record.getSchema(), true);
}

@Override
public float getProgress() throws IOException, InterruptedException {
return parquetRecordReader.getProgress();
}

@Override
public void close() throws IOException {
parquetRecordReader.close();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
/*
* 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.hadoop.avro;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.ArrayWritable;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.parquet.hadoop.ParquetInputFormat;
import org.apache.parquet.hadoop.util.ContextUtil;

import java.io.IOException;

/**
* To resolve issue <a href="https://issues.apache.org/jira/browse/HUDI-83">Fix Timestamp/Date type read by Hive3</a>,
* we need to handle timestamp types separately based on the parquet-avro approach
*/
public class HoodieTimestampAwareParquetInputFormat extends ParquetInputFormat<ArrayWritable> {

@Override
public RecordReader<Void, ArrayWritable> createRecordReader(
InputSplit inputSplit,
TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
Configuration conf = ContextUtil.getConfiguration(taskAttemptContext);
return new HoodieAvroParquetReader(inputSplit, conf);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.hadoop.HoodieColumnProjectionUtils;
import org.apache.hudi.hadoop.SchemaEvolutionContext;
import org.apache.hudi.hadoop.utils.HiveAvroSerializer;
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
Expand Down Expand Up @@ -72,6 +73,8 @@ public abstract class AbstractRealtimeRecordReader {
protected boolean supportPayload = true;
// handle hive type to avro record
protected HiveAvroSerializer serializer;
// support timestamp
private final boolean supportTimestamp;

public AbstractRealtimeRecordReader(RealtimeSplit split, JobConf job) {
this.split = split;
Expand All @@ -80,6 +83,8 @@ public AbstractRealtimeRecordReader(RealtimeSplit split, JobConf job) {
LOG.info("columnIds ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR));
LOG.info("partitioningColumns ==> " + job.get(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, ""));
this.supportPayload = Boolean.parseBoolean(job.get("hoodie.support.payload", "true"));
// get timestamp columns
supportTimestamp = HoodieColumnProjectionUtils.supportTimestamp(jobConf);
try {
metaClient = HoodieTableMetaClient.builder().setConf(jobConf).setBasePath(split.getBasePath()).build();
if (metaClient.getTableConfig().getPreCombineField() != null) {
Expand Down Expand Up @@ -219,4 +224,8 @@ public void setWriterSchema(Schema writerSchema) {
public void setHiveSchema(Schema hiveSchema) {
this.hiveSchema = hiveSchema;
}

public boolean isSupportTimestamp() {
return supportTimestamp;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,7 @@ private void setUpWritable(Option<GenericRecord> rec, ArrayWritable arrayWritabl
// we assume, a later safe record in the log, is newer than what we have in the map &
// replace it. Since we want to return an arrayWritable which is the same length as the elements in the latest
// schema, we use writerSchema to create the arrayWritable from the latest generic record
ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(recordToReturn, getHiveSchema());
ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(recordToReturn, getHiveSchema(), isSupportTimestamp());
Writable[] replaceValue = aWritable.get();
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("key %s, base values: %s, log values: %s", key, HoodieRealtimeRecordReaderUtils.arrayWritableToString(arrayWritable),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,7 @@ private List<HoodieProducer<ArrayWritable>> getParallelProducers(
scannerBuilder.withLogRecordScannerCallback(record -> {
// convert Hoodie log record to Hadoop AvroWritable and buffer
GenericRecord rec = (GenericRecord) record.getData().getInsertValue(getReaderSchema(), payloadProps).get();
ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(rec, getHiveSchema());
ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(rec, getHiveSchema(), isSupportTimestamp());
queue.insertRecord(aWritable);
})
.build();
Expand Down
Loading