[HUDI-960] Implementation of the HFile base and log file format. (#1804)
* [HUDI-960] Implementation of the HFile base and log file format. 1. Includes HFileWriter and HFileReader 2. Includes HFileInputFormat for both snapshot and realtime input format for Hive 3. Unit test for new code 4. IT for using HFile format and querying using Hive (Presto and SparkSQL are not supported) Advantage: HFile file format saves data as binary key-value pairs. This implementation chooses the following values: 1. Key = Hoodie Record Key (as bytes) 2. Value = Avro encoded GenericRecord (as bytes) HFile allows efficient lookup of a record by key or range of keys. Hence, this base file format is well suited to applications like RFC-15, RFC-08 which will benefit from the ability to lookup records by key or search in a range of keys without having to read the entire data/log format. Limitations: HFile storage format has certain limitations when used as a general purpose data storage format. 1. Does not have a implemented reader for Presto and SparkSQL 2. Is not a columnar file format and hence may lead to lower compression levels and greater IO on query side due to lack of column pruning Other changes: - Remove databricks/avro from pom - Fix HoodieClientTestUtils from not using scala imports/reflection based conversion etc - Breaking up limitFileSize(), per parquet and hfile base files - Added three new configs for HoodieHFileConfig - prefetchBlocksOnOpen, cacheDataInL1, dropBehindCacheCompaction - Throw UnsupportedException in HFileReader.getRecordKeys() - Updated HoodieCopyOnWriteTable to create the correct merge handle (HoodieSortedMergeHandle for HFile and HoodieMergeHandle otherwise) * Fixing checkstyle Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -0,0 +1,163 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import org.apache.hadoop.conf.Configurable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
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.hadoop.mapred.Reporter;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* HoodieInputFormat for HUDI datasets which store data in HFile base file format.
|
||||
*/
|
||||
@UseFileSplitsFromInputFormat
|
||||
public class HoodieHFileInputFormat extends FileInputFormat<NullWritable, ArrayWritable> implements Configurable {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieHFileInputFormat.class);
|
||||
|
||||
protected Configuration conf;
|
||||
|
||||
protected HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) {
|
||||
return HoodieInputFormatUtils.filterInstantsTimeline(timeline);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(JobConf job) throws IOException {
|
||||
// Segregate inputPaths[] to incremental, snapshot and non hoodie paths
|
||||
List<String> incrementalTables = HoodieHiveUtils.getIncrementalTableNames(Job.getInstance(job));
|
||||
InputPathHandler inputPathHandler = new InputPathHandler(conf, getInputPaths(job), incrementalTables);
|
||||
List<FileStatus> returns = new ArrayList<>();
|
||||
|
||||
Map<String, HoodieTableMetaClient> tableMetaClientMap = inputPathHandler.getTableMetaClientMap();
|
||||
// process incremental pulls first
|
||||
for (String table : incrementalTables) {
|
||||
HoodieTableMetaClient metaClient = tableMetaClientMap.get(table);
|
||||
if (metaClient == null) {
|
||||
/* This can happen when the INCREMENTAL mode is set for a table but there were no InputPaths
|
||||
* in the jobConf
|
||||
*/
|
||||
continue;
|
||||
}
|
||||
List<Path> inputPaths = inputPathHandler.getGroupedIncrementalPaths().get(metaClient);
|
||||
List<FileStatus> result = listStatusForIncrementalMode(job, metaClient, inputPaths);
|
||||
if (result != null) {
|
||||
returns.addAll(result);
|
||||
}
|
||||
}
|
||||
|
||||
// process non hoodie Paths next.
|
||||
List<Path> nonHoodiePaths = inputPathHandler.getNonHoodieInputPaths();
|
||||
if (nonHoodiePaths.size() > 0) {
|
||||
setInputPaths(job, nonHoodiePaths.toArray(new Path[nonHoodiePaths.size()]));
|
||||
FileStatus[] fileStatuses = super.listStatus(job);
|
||||
returns.addAll(Arrays.asList(fileStatuses));
|
||||
}
|
||||
|
||||
// process snapshot queries next.
|
||||
List<Path> snapshotPaths = inputPathHandler.getSnapshotPaths();
|
||||
if (snapshotPaths.size() > 0) {
|
||||
setInputPaths(job, snapshotPaths.toArray(new Path[snapshotPaths.size()]));
|
||||
FileStatus[] fileStatuses = super.listStatus(job);
|
||||
Map<HoodieTableMetaClient, List<FileStatus>> groupedFileStatus =
|
||||
HoodieInputFormatUtils.groupFileStatusForSnapshotPaths(fileStatuses, HoodieFileFormat.HFILE.getFileExtension(),
|
||||
tableMetaClientMap.values());
|
||||
LOG.info("Found a total of " + groupedFileStatus.size() + " groups");
|
||||
for (Map.Entry<HoodieTableMetaClient, List<FileStatus>> entry : groupedFileStatus.entrySet()) {
|
||||
List<FileStatus> result = HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, entry.getKey(), entry.getValue());
|
||||
if (result != null) {
|
||||
returns.addAll(result);
|
||||
}
|
||||
}
|
||||
}
|
||||
return returns.toArray(new FileStatus[returns.size()]);
|
||||
}
|
||||
|
||||
/**
|
||||
* Achieves listStatus functionality for an incrementally queried table. Instead of listing all
|
||||
* partitions and then filtering based on the commits of interest, this logic first extracts the
|
||||
* partitions touched by the desired commits and then lists only those partitions.
|
||||
*/
|
||||
private List<FileStatus> listStatusForIncrementalMode(
|
||||
JobConf job, HoodieTableMetaClient tableMetaClient, List<Path> inputPaths) throws IOException {
|
||||
String tableName = tableMetaClient.getTableConfig().getTableName();
|
||||
Job jobContext = Job.getInstance(job);
|
||||
Option<HoodieTimeline> timeline = HoodieInputFormatUtils.getFilteredCommitsTimeline(jobContext, tableMetaClient);
|
||||
if (!timeline.isPresent()) {
|
||||
return null;
|
||||
}
|
||||
Option<List<HoodieInstant>> commitsToCheck = HoodieInputFormatUtils.getCommitsForIncrementalQuery(jobContext, tableName, timeline.get());
|
||||
if (!commitsToCheck.isPresent()) {
|
||||
return null;
|
||||
}
|
||||
Option<String> incrementalInputPaths = HoodieInputFormatUtils.getAffectedPartitions(commitsToCheck.get(), tableMetaClient, timeline.get(), inputPaths);
|
||||
// Mutate the JobConf to set the input paths to only partitions touched by incremental pull.
|
||||
if (!incrementalInputPaths.isPresent()) {
|
||||
return null;
|
||||
}
|
||||
setInputPaths(job, incrementalInputPaths.get());
|
||||
FileStatus[] fileStatuses = super.listStatus(job);
|
||||
return HoodieInputFormatUtils.filterIncrementalFileStatus(jobContext, tableMetaClient, timeline.get(), fileStatuses, commitsToCheck.get());
|
||||
}
|
||||
|
||||
@Override
|
||||
public RecordReader<NullWritable, ArrayWritable> getRecordReader(final InputSplit split, final JobConf job,
|
||||
final Reporter reporter) throws IOException {
|
||||
return new HoodieHFileRecordReader(conf, split, job);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean isSplitable(FileSystem fs, Path filename) {
|
||||
// This file isn't splittable.
|
||||
return false;
|
||||
}
|
||||
|
||||
public void setConf(Configuration conf) {
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConf() {
|
||||
return conf;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,101 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
|
||||
import org.apache.hudi.io.storage.HoodieHFileReader;
|
||||
|
||||
public class HoodieHFileRecordReader implements RecordReader<NullWritable, ArrayWritable> {
|
||||
|
||||
private long count = 0;
|
||||
private ArrayWritable valueObj;
|
||||
private HoodieHFileReader reader;
|
||||
private Iterator<GenericRecord> recordIterator;
|
||||
private Schema schema;
|
||||
|
||||
public HoodieHFileRecordReader(Configuration conf, InputSplit split, JobConf job) throws IOException {
|
||||
FileSplit fileSplit = (FileSplit) split;
|
||||
Path path = fileSplit.getPath();
|
||||
reader = new HoodieHFileReader(conf, path, new CacheConfig(conf));
|
||||
|
||||
schema = reader.getSchema();
|
||||
valueObj = new ArrayWritable(Writable.class, new Writable[schema.getFields().size()]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean next(NullWritable key, ArrayWritable value) throws IOException {
|
||||
if (recordIterator == null) {
|
||||
recordIterator = reader.getRecordIterator(schema);
|
||||
}
|
||||
|
||||
if (!recordIterator.hasNext()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
GenericRecord record = recordIterator.next();
|
||||
ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(record, schema);
|
||||
value.set(aWritable.get());
|
||||
count++;
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NullWritable createKey() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayWritable createValue() {
|
||||
return valueObj;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPos() throws IOException {
|
||||
// TODO Auto-generated method stub
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (reader != null) {
|
||||
reader.close();
|
||||
reader = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getProgress() throws IOException {
|
||||
return 1.0f * count / reader.getTotalRecords();
|
||||
}
|
||||
}
|
||||
@@ -51,7 +51,7 @@ public class InputPathHandler {
|
||||
private final List<Path> snapshotPaths;
|
||||
private final List<Path> nonHoodieInputPaths;
|
||||
|
||||
InputPathHandler(Configuration conf, Path[] inputPaths, List<String> incrementalTables) throws IOException {
|
||||
public InputPathHandler(Configuration conf, Path[] inputPaths, List<String> incrementalTables) throws IOException {
|
||||
this.conf = conf;
|
||||
tableMetaClientMap = new HashMap<>();
|
||||
snapshotPaths = new ArrayList<>();
|
||||
|
||||
@@ -0,0 +1,110 @@
|
||||
/*
|
||||
* 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.realtime;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hadoop.mapred.Reporter;
|
||||
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.hadoop.HoodieHFileInputFormat;
|
||||
import org.apache.hudi.hadoop.UseFileSplitsFromInputFormat;
|
||||
import org.apache.hudi.hadoop.UseRecordReaderFromInputFormat;
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
|
||||
import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* HoodieRealtimeInputFormat for HUDI datasets which store data in HFile base file format.
|
||||
*/
|
||||
@UseRecordReaderFromInputFormat
|
||||
@UseFileSplitsFromInputFormat
|
||||
public class HoodieHFileRealtimeInputFormat extends HoodieHFileInputFormat {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieHFileRealtimeInputFormat.class);
|
||||
|
||||
@Override
|
||||
public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
|
||||
Stream<FileSplit> fileSplits = Arrays.stream(super.getSplits(job, numSplits)).map(is -> (FileSplit) is);
|
||||
return HoodieRealtimeInputFormatUtils.getRealtimeSplits(job, fileSplits);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(JobConf job) throws IOException {
|
||||
// Call the HoodieInputFormat::listStatus to obtain all latest hfiles, based on commit timeline.
|
||||
return super.listStatus(job);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) {
|
||||
// no specific filtering for Realtime format
|
||||
return timeline;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RecordReader<NullWritable, ArrayWritable> getRecordReader(final InputSplit split, final JobConf jobConf,
|
||||
final Reporter reporter) throws IOException {
|
||||
// Hive on Spark invokes multiple getRecordReaders from different threads in the same spark task (and hence the
|
||||
// same JVM) unlike Hive on MR. Due to this, accesses to JobConf, which is shared across all threads, is at the
|
||||
// risk of experiencing race conditions. Hence, we synchronize on the JobConf object here. There is negligible
|
||||
// latency incurred here due to the synchronization since get record reader is called once per spilt before the
|
||||
// actual heavy lifting of reading the parquet files happen.
|
||||
if (jobConf.get(HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP) == null) {
|
||||
synchronized (jobConf) {
|
||||
LOG.info(
|
||||
"Before adding Hoodie columns, Projections :" + jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)
|
||||
+ ", Ids :" + jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR));
|
||||
if (jobConf.get(HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP) == null) {
|
||||
// Hive (across all versions) fails for queries like select count(`_hoodie_commit_time`) from table;
|
||||
// In this case, the projection fields gets removed. Looking at HiveInputFormat implementation, in some cases
|
||||
// hoodie additional projection columns are reset after calling setConf and only natural projections
|
||||
// (one found in select queries) are set. things would break because of this.
|
||||
// For e:g _hoodie_record_key would be missing and merge step would throw exceptions.
|
||||
// TO fix this, hoodie columns are appended late at the time record-reader gets built instead of construction
|
||||
// time.
|
||||
HoodieRealtimeInputFormatUtils.cleanProjectionColumnIds(jobConf);
|
||||
HoodieRealtimeInputFormatUtils.addRequiredProjectionFields(jobConf);
|
||||
|
||||
this.conf = jobConf;
|
||||
this.conf.set(HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP, "true");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
LOG.info("Creating record reader with readCols :" + jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)
|
||||
+ ", Ids :" + jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR));
|
||||
// sanity check
|
||||
ValidationUtils.checkArgument(split instanceof HoodieRealtimeFileSplit,
|
||||
"HoodieRealtimeRecordReader can only work on HoodieRealtimeFileSplit and not with " + split);
|
||||
|
||||
return new HoodieRealtimeRecordReader((HoodieRealtimeFileSplit) split, jobConf,
|
||||
super.getRecordReader(split, jobConf, reporter));
|
||||
}
|
||||
}
|
||||
@@ -18,10 +18,8 @@
|
||||
|
||||
package org.apache.hudi.hadoop.realtime;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.hadoop.HoodieColumnProjectionUtils;
|
||||
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
||||
import org.apache.hudi.hadoop.UseFileSplitsFromInputFormat;
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
|
||||
@@ -43,9 +41,7 @@ import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
@@ -83,68 +79,6 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
|
||||
return timeline;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a field to the existing fields projected.
|
||||
*/
|
||||
private static Configuration addProjectionField(Configuration conf, String fieldName, int fieldIndex) {
|
||||
String readColNames = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "");
|
||||
String readColIds = conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "");
|
||||
|
||||
String readColNamesPrefix = readColNames + ",";
|
||||
if (readColNames == null || readColNames.isEmpty()) {
|
||||
readColNamesPrefix = "";
|
||||
}
|
||||
String readColIdsPrefix = readColIds + ",";
|
||||
if (readColIds == null || readColIds.isEmpty()) {
|
||||
readColIdsPrefix = "";
|
||||
}
|
||||
|
||||
if (!readColNames.contains(fieldName)) {
|
||||
// If not already in the list - then add it
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, readColNamesPrefix + fieldName);
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, readColIdsPrefix + fieldIndex);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format("Adding extra column " + fieldName + ", to enable log merging cols (%s) ids (%s) ",
|
||||
conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
|
||||
conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)));
|
||||
}
|
||||
}
|
||||
return conf;
|
||||
}
|
||||
|
||||
private static void addRequiredProjectionFields(Configuration configuration) {
|
||||
List<Integer> projectedIds = new ArrayList<>(HoodieColumnProjectionUtils.getReadColumnIDs(configuration));
|
||||
List<String> projectedNames = new ArrayList<>(Arrays.asList(HoodieColumnProjectionUtils.getReadColumnNames(configuration)));
|
||||
projectedIds.addAll(Arrays.asList(
|
||||
HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS,
|
||||
HoodieInputFormatUtils.HOODIE_COMMIT_TIME_COL_POS,
|
||||
HoodieInputFormatUtils.HOODIE_PARTITION_PATH_COL_POS)
|
||||
);
|
||||
projectedNames.addAll(Arrays.asList(
|
||||
HoodieRecord.RECORD_KEY_METADATA_FIELD,
|
||||
HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
HoodieRecord.PARTITION_PATH_METADATA_FIELD)
|
||||
);
|
||||
|
||||
HoodieColumnProjectionUtils.setReadColumns(configuration, projectedIds, projectedNames);
|
||||
}
|
||||
|
||||
/**
|
||||
* Hive will append read columns' ids to old columns' ids during getRecordReader. In some cases, e.g. SELECT COUNT(*),
|
||||
* the read columns' id is an empty string and Hive will combine it with Hoodie required projection ids and becomes
|
||||
* e.g. ",2,0,3" and will cause an error. Actually this method is a temporary solution because the real bug is from
|
||||
* Hive. Hive has fixed this bug after 3.0.0, but the version before that would still face this problem. (HIVE-22438)
|
||||
*/
|
||||
private static void cleanProjectionColumnIds(Configuration conf) {
|
||||
String columnIds = conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR);
|
||||
if (!columnIds.isEmpty() && columnIds.charAt(0) == ',') {
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, columnIds.substring(1));
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("The projection Ids: {" + columnIds + "} start with ','. First comma is removed");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public RecordReader<NullWritable, ArrayWritable> getRecordReader(final InputSplit split, final JobConf jobConf,
|
||||
final Reporter reporter) throws IOException {
|
||||
@@ -166,8 +100,8 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
|
||||
// For e:g _hoodie_record_key would be missing and merge step would throw exceptions.
|
||||
// TO fix this, hoodie columns are appended late at the time record-reader gets built instead of construction
|
||||
// time.
|
||||
cleanProjectionColumnIds(jobConf);
|
||||
addRequiredProjectionFields(jobConf);
|
||||
HoodieRealtimeInputFormatUtils.cleanProjectionColumnIds(jobConf);
|
||||
HoodieRealtimeInputFormatUtils.addRequiredProjectionFields(jobConf);
|
||||
|
||||
this.conf = jobConf;
|
||||
this.conf.set(HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP, "true");
|
||||
|
||||
@@ -33,8 +33,10 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hadoop.FileStatusWithBootstrapBaseFile;
|
||||
import org.apache.hudi.hadoop.HoodieHFileInputFormat;
|
||||
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
||||
import org.apache.hudi.hadoop.LocatedFileStatusWithBootstrapBaseFile;
|
||||
import org.apache.hudi.hadoop.realtime.HoodieHFileRealtimeInputFormat;
|
||||
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
@@ -82,6 +84,16 @@ public class HoodieInputFormatUtils {
|
||||
inputFormat.setConf(conf);
|
||||
return inputFormat;
|
||||
}
|
||||
case HFILE:
|
||||
if (realtime) {
|
||||
HoodieHFileRealtimeInputFormat inputFormat = new HoodieHFileRealtimeInputFormat();
|
||||
inputFormat.setConf(conf);
|
||||
return inputFormat;
|
||||
} else {
|
||||
HoodieHFileInputFormat inputFormat = new HoodieHFileInputFormat();
|
||||
inputFormat.setConf(conf);
|
||||
return inputFormat;
|
||||
}
|
||||
default:
|
||||
throw new HoodieIOException("Hoodie InputFormat not implemented for base file format " + baseFileFormat);
|
||||
}
|
||||
@@ -96,6 +108,8 @@ public class HoodieInputFormatUtils {
|
||||
switch (baseFileFormat) {
|
||||
case PARQUET:
|
||||
return MapredParquetOutputFormat.class.getName();
|
||||
case HFILE:
|
||||
return MapredParquetOutputFormat.class.getName();
|
||||
default:
|
||||
throw new HoodieIOException("No OutputFormat for base file format " + baseFileFormat);
|
||||
}
|
||||
@@ -105,6 +119,8 @@ public class HoodieInputFormatUtils {
|
||||
switch (baseFileFormat) {
|
||||
case PARQUET:
|
||||
return ParquetHiveSerDe.class.getName();
|
||||
case HFILE:
|
||||
return ParquetHiveSerDe.class.getName();
|
||||
default:
|
||||
throw new HoodieIOException("No SerDe for base file format " + baseFileFormat);
|
||||
}
|
||||
@@ -115,6 +131,9 @@ public class HoodieInputFormatUtils {
|
||||
if (extension.equals(HoodieFileFormat.PARQUET.getFileExtension())) {
|
||||
return getInputFormat(HoodieFileFormat.PARQUET, realtime, conf);
|
||||
}
|
||||
if (extension.equals(HoodieFileFormat.HFILE.getFileExtension())) {
|
||||
return getInputFormat(HoodieFileFormat.HFILE, realtime, conf);
|
||||
}
|
||||
throw new HoodieIOException("Hoodie InputFormat not implemented for base file of type " + extension);
|
||||
}
|
||||
|
||||
|
||||
@@ -22,6 +22,7 @@ import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
@@ -36,6 +37,7 @@ import org.apache.hudi.hadoop.realtime.RealtimeBootstrapBaseFileSplit;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.SplitLocationInfo;
|
||||
@@ -165,4 +167,56 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
|
||||
return resultMap;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Add a field to the existing fields projected.
|
||||
*/
|
||||
private static Configuration addProjectionField(Configuration conf, String fieldName, int fieldIndex) {
|
||||
String readColNames = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "");
|
||||
String readColIds = conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "");
|
||||
|
||||
String readColNamesPrefix = readColNames + ",";
|
||||
if (readColNames == null || readColNames.isEmpty()) {
|
||||
readColNamesPrefix = "";
|
||||
}
|
||||
String readColIdsPrefix = readColIds + ",";
|
||||
if (readColIds == null || readColIds.isEmpty()) {
|
||||
readColIdsPrefix = "";
|
||||
}
|
||||
|
||||
if (!readColNames.contains(fieldName)) {
|
||||
// If not already in the list - then add it
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, readColNamesPrefix + fieldName);
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, readColIdsPrefix + fieldIndex);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format("Adding extra column " + fieldName + ", to enable log merging cols (%s) ids (%s) ",
|
||||
conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
|
||||
conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)));
|
||||
}
|
||||
}
|
||||
return conf;
|
||||
}
|
||||
|
||||
public static void addRequiredProjectionFields(Configuration configuration) {
|
||||
// Need this to do merge records in HoodieRealtimeRecordReader
|
||||
addProjectionField(configuration, HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS);
|
||||
addProjectionField(configuration, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieInputFormatUtils.HOODIE_COMMIT_TIME_COL_POS);
|
||||
addProjectionField(configuration, HoodieRecord.PARTITION_PATH_METADATA_FIELD, HoodieInputFormatUtils.HOODIE_PARTITION_PATH_COL_POS);
|
||||
}
|
||||
|
||||
/**
|
||||
* Hive will append read columns' ids to old columns' ids during getRecordReader. In some cases, e.g. SELECT COUNT(*),
|
||||
* the read columns' id is an empty string and Hive will combine it with Hoodie required projection ids and becomes
|
||||
* e.g. ",2,0,3" and will cause an error. Actually this method is a temporary solution because the real bug is from
|
||||
* Hive. Hive has fixed this bug after 3.0.0, but the version before that would still face this problem. (HIVE-22438)
|
||||
*/
|
||||
public static void cleanProjectionColumnIds(Configuration conf) {
|
||||
String columnIds = conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR);
|
||||
if (!columnIds.isEmpty() && columnIds.charAt(0) == ',') {
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, columnIds.substring(1));
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("The projection Ids: {" + columnIds + "} start with ','. First comma is removed");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user