[HUDI-242] Support for RFC-12/Bootstrapping of external datasets to hudi (#1876)
- [HUDI-418] Bootstrap Index Implementation using HFile with unit-test - [HUDI-421] FileSystem View Changes to support Bootstrap with unit-tests - [HUDI-424] Implement Query Side Integration for querying tables containing bootstrap file slices - [HUDI-423] Implement upsert functionality for handling updates to these bootstrap file slices - [HUDI-421] Bootstrap Write Client with tests - [HUDI-425] Added HoodieDeltaStreamer support - [HUDI-899] Add a knob to change partition-path style while performing metadata bootstrap - [HUDI-900] Metadata Bootstrap Key Generator needs to handle complex keys correctly - [HUDI-424] Simplify Record reader implementation - [HUDI-423] Implement upsert functionality for handling updates to these bootstrap file slices - [HUDI-420] Hoodie Demo working with hive and sparkSQL. Also, Hoodie CLI working with bootstrap tables Co-authored-by: Mehrotra <uditme@amazon.com> Co-authored-by: Vinoth Chandar <vinoth@apache.org> Co-authored-by: Balaji Varadarajan <varadarb@uber.com>
This commit is contained in:
@@ -39,8 +39,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
}
|
||||
|
||||
@Override
|
||||
protected org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim
|
||||
createInputFormatShim() {
|
||||
protected org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim createInputFormatShim() {
|
||||
return new HoodieCombineFileInputFormatShim<>();
|
||||
}
|
||||
|
||||
|
||||
@@ -0,0 +1,71 @@
|
||||
/*
|
||||
* 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.mapred.FileSplit;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
|
||||
|
||||
/**
|
||||
* Sub-type of File Split which encapsulates both skeleton and bootstrap base file splits.
|
||||
*/
|
||||
public class BootstrapBaseFileSplit extends FileSplit {
|
||||
|
||||
private FileSplit bootstrapFileSplit;
|
||||
|
||||
public BootstrapBaseFileSplit() {
|
||||
super();
|
||||
}
|
||||
|
||||
public BootstrapBaseFileSplit(FileSplit baseSplit, FileSplit bootstrapFileSplit)
|
||||
throws IOException {
|
||||
super(baseSplit.getPath(), baseSplit.getStart(), baseSplit.getLength(), baseSplit.getLocations());
|
||||
this.bootstrapFileSplit = bootstrapFileSplit;
|
||||
}
|
||||
|
||||
public FileSplit getBootstrapFileSplit() {
|
||||
return bootstrapFileSplit;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
super.write(out);
|
||||
bootstrapFileSplit.write(out);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
super.readFields(in);
|
||||
bootstrapFileSplit = new WrapperFileSplit();
|
||||
bootstrapFileSplit.readFields(in);
|
||||
}
|
||||
|
||||
/**
|
||||
* Wrapper for FileSplit just to expose default constructor to the outer class.
|
||||
*/
|
||||
public static class WrapperFileSplit extends FileSplit {
|
||||
|
||||
public WrapperFileSplit() {
|
||||
super();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Stitches 2 record reader returned rows and presents a concatenated view to clients.
|
||||
*/
|
||||
public class BootstrapColumnStichingRecordReader implements RecordReader<NullWritable, ArrayWritable> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(BootstrapColumnStichingRecordReader.class);
|
||||
|
||||
private final RecordReader<NullWritable, ArrayWritable> leftColsRecordReader;
|
||||
private final RecordReader<NullWritable, ArrayWritable> rightColsRecordReader;
|
||||
private final int numLeftColumns;
|
||||
private final ArrayWritable values;
|
||||
private final boolean validate;
|
||||
|
||||
public BootstrapColumnStichingRecordReader(RecordReader<NullWritable, ArrayWritable> left,
|
||||
int numLeftColumns, RecordReader<NullWritable, ArrayWritable> right, int numRightColumns, boolean validate) {
|
||||
this.leftColsRecordReader = left;
|
||||
this.rightColsRecordReader = right;
|
||||
this.validate = validate;
|
||||
this.numLeftColumns = numLeftColumns;
|
||||
|
||||
ArrayWritable rightW = rightColsRecordReader.createValue();
|
||||
int numColumns = numLeftColumns + numRightColumns;
|
||||
if (rightW.getValueClass() != null) {
|
||||
values = new ArrayWritable(rightW.getValueClass(), new Writable[numColumns]);
|
||||
} else {
|
||||
String[] vals = IntStream.range(0, numColumns).mapToObj(idx -> "").collect(Collectors.toList())
|
||||
.toArray(new String[0]);
|
||||
values = new ArrayWritable(vals);
|
||||
}
|
||||
LOG.info("Total ArrayWritable Length :" + values.get().length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean next(NullWritable key, ArrayWritable value) throws IOException {
|
||||
ArrayWritable left = leftColsRecordReader.createValue();
|
||||
ArrayWritable right = rightColsRecordReader.createValue();
|
||||
|
||||
boolean hasMoreOnLeft = leftColsRecordReader.next(leftColsRecordReader.createKey(), left);
|
||||
boolean hasMoreOnRight = rightColsRecordReader.next(rightColsRecordReader.createKey(), right);
|
||||
if (validate) {
|
||||
ValidationUtils.checkArgument(hasMoreOnLeft == hasMoreOnRight,
|
||||
String.format("hasMoreOnLeft:%s, hasMoreOnRight: %s", hasMoreOnLeft, hasMoreOnRight));
|
||||
}
|
||||
for (int i = 0; i < numLeftColumns; i++) {
|
||||
value.get()[i] = left.get()[i];
|
||||
}
|
||||
for (int j = numLeftColumns; j < right.get().length; j++) {
|
||||
value.get()[j] = right.get()[j];
|
||||
}
|
||||
return hasMoreOnLeft && hasMoreOnRight;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NullWritable createKey() {
|
||||
return leftColsRecordReader.createKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayWritable createValue() {
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPos() throws IOException {
|
||||
return leftColsRecordReader.getPos();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
leftColsRecordReader.close();
|
||||
rightColsRecordReader.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getProgress() throws IOException {
|
||||
return leftColsRecordReader.getProgress();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,42 @@
|
||||
/*
|
||||
* 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.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Sub-Type of File Status tracking both skeleton and bootstrap base file's status.
|
||||
*/
|
||||
public class FileStatusWithBootstrapBaseFile extends FileStatus {
|
||||
|
||||
private final FileStatus bootstrapBaseFileStatus;
|
||||
|
||||
public FileStatusWithBootstrapBaseFile(FileStatus fileStatus, FileStatus bootstrapBaseFileStatus) throws IOException {
|
||||
super(fileStatus);
|
||||
this.bootstrapBaseFileStatus = bootstrapBaseFileStatus;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getPath() {
|
||||
return new PathWithBootstrapFileStatus(super.getPath(), bootstrapBaseFileStatus);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,300 @@
|
||||
/*
|
||||
* 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.hive.serde2.typeinfo.TypeInfoUtils;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hive.ql.io.IOConstants;
|
||||
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
/**
|
||||
* Utility funcitons copied from Hive ColumnProjectionUtils.java.
|
||||
* Needed to copy as we see NoSuchMethod errors when directly using these APIs with/without Spark.
|
||||
* Some of these methods are not available across hive versions.
|
||||
*/
|
||||
public class HoodieColumnProjectionUtils {
|
||||
public static final Logger LOG = LoggerFactory.getLogger(ColumnProjectionUtils.class);
|
||||
|
||||
public static final String READ_COLUMN_IDS_CONF_STR = "hive.io.file.readcolumn.ids";
|
||||
/**
|
||||
* the nested column path is the string from the root to the leaf
|
||||
* e.g.
|
||||
* c:struct_of (a:string,b:string).
|
||||
* the column a's path is c.a and b's path is c.b
|
||||
*/
|
||||
public static final String READ_NESTED_COLUMN_PATH_CONF_STR =
|
||||
"hive.io.file.readNestedColumn.paths";
|
||||
public static final String READ_ALL_COLUMNS = "hive.io.file.read.all.columns";
|
||||
public static final String READ_COLUMN_NAMES_CONF_STR = "hive.io.file.readcolumn.names";
|
||||
private static final String READ_COLUMN_IDS_CONF_STR_DEFAULT = "";
|
||||
private static final String READ_COLUMN_NAMES_CONF_STR_DEFAULT = "";
|
||||
private static final String READ_NESTED_COLUMN_PATH_CONF_STR_DEFAULT = "";
|
||||
private static final boolean READ_ALL_COLUMNS_DEFAULT = true;
|
||||
|
||||
private static final String COMMA = ",";
|
||||
|
||||
/** Special Column Names added during Parquet Projection. **/
|
||||
public static final String PARQUET_BLOCK_OFFSET_COL_NAME = "BLOCK__OFFSET__INSIDE__FILE";
|
||||
public static final String PARQUET_INPUT_FILE_NAME = "INPUT__FILE__NAME";
|
||||
public static final String PARQUET_ROW_ID = "ROW__ID";
|
||||
|
||||
public static final List<String> PARQUET_SPECIAL_COLUMN_NAMES = CollectionUtils
|
||||
.createImmutableList(PARQUET_BLOCK_OFFSET_COL_NAME, PARQUET_INPUT_FILE_NAME,
|
||||
PARQUET_ROW_ID);
|
||||
|
||||
/**
|
||||
* Sets the <em>READ_ALL_COLUMNS</em> flag and removes any previously
|
||||
* set column ids.
|
||||
*/
|
||||
public static void setReadAllColumns(Configuration conf) {
|
||||
conf.setBoolean(READ_ALL_COLUMNS, true);
|
||||
setReadColumnIDConf(conf, READ_COLUMN_IDS_CONF_STR_DEFAULT);
|
||||
setReadColumnNamesConf(conf, READ_COLUMN_NAMES_CONF_STR_DEFAULT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the <em>READ_ALL_COLUMNS</em> columns flag.
|
||||
*/
|
||||
public static boolean isReadAllColumns(Configuration conf) {
|
||||
return conf.getBoolean(READ_ALL_COLUMNS, READ_ALL_COLUMNS_DEFAULT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the <em>READ_ALL_COLUMNS</em> flag to false and overwrites column ids
|
||||
* with the provided list.
|
||||
*/
|
||||
public static void setReadColumns(Configuration conf, List<Integer> ids, List<String> names) {
|
||||
setReadColumnIDConf(conf, READ_COLUMN_IDS_CONF_STR_DEFAULT);
|
||||
setReadColumnNamesConf(conf, READ_COLUMN_NAMES_CONF_STR_DEFAULT);
|
||||
appendReadColumns(conf, ids);
|
||||
appendReadColumnNames(conf, names);
|
||||
}
|
||||
|
||||
/**
|
||||
* Appends read columns' ids (start from zero). Once a column
|
||||
* is included in the list, a underlying record reader of a columnar file format
|
||||
* (e.g. RCFile and ORC) can know what columns are needed.
|
||||
*/
|
||||
public static void appendReadColumns(Configuration conf, List<Integer> ids) {
|
||||
String id = toReadColumnIDString(ids);
|
||||
String old = conf.get(READ_COLUMN_IDS_CONF_STR, null);
|
||||
String newConfStr = id;
|
||||
if (old != null && !old.isEmpty()) {
|
||||
newConfStr = newConfStr + StringUtils.COMMA_STR + old;
|
||||
}
|
||||
setReadColumnIDConf(conf, newConfStr);
|
||||
// Set READ_ALL_COLUMNS to false
|
||||
conf.setBoolean(READ_ALL_COLUMNS, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Appends read nested column's paths. Once a read nested column path
|
||||
* is included in the list, a underlying record reader of a columnar file format
|
||||
* (e.g. Parquet and ORC) can know what columns are needed.
|
||||
*/
|
||||
public static void appendNestedColumnPaths(
|
||||
Configuration conf,
|
||||
List<String> paths) {
|
||||
if (paths == null || paths.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
String pathsStr = StringUtils.join(StringUtils.COMMA_STR,
|
||||
paths.toArray(new String[paths.size()]));
|
||||
String old = conf.get(READ_NESTED_COLUMN_PATH_CONF_STR, null);
|
||||
String newConfStr = pathsStr;
|
||||
if (old != null && !old.isEmpty()) {
|
||||
newConfStr = newConfStr + StringUtils.COMMA_STR + old;
|
||||
}
|
||||
setReadNestedColumnPathConf(conf, newConfStr);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* This method appends read column information to configuration to use for PPD. It is
|
||||
* currently called with information from TSOP. Names come from TSOP input RowSchema, and
|
||||
* IDs are the indexes inside the schema (which PPD assumes correspond to indexes inside the
|
||||
* files to PPD in; something that would be invalid in many cases of schema evolution).
|
||||
* @param conf Config to set values to.
|
||||
* @param ids Column ids.
|
||||
* @param names Column names.
|
||||
*/
|
||||
public static void appendReadColumns(
|
||||
Configuration conf, List<Integer> ids, List<String> names, List<String> groupPaths) {
|
||||
if (ids.size() != names.size()) {
|
||||
LOG.warn("Read column counts do not match: "
|
||||
+ ids.size() + " ids, " + names.size() + " names");
|
||||
}
|
||||
appendReadColumns(conf, ids);
|
||||
appendReadColumnNames(conf, names);
|
||||
appendNestedColumnPaths(conf, groupPaths);
|
||||
}
|
||||
|
||||
public static void appendReadColumns(
|
||||
StringBuilder readColumnsBuffer, StringBuilder readColumnNamesBuffer, List<Integer> ids,
|
||||
List<String> names) {
|
||||
String preppedIdStr = ids.stream().map(x -> String.valueOf(x)).collect(Collectors.joining(COMMA));
|
||||
String preppedNamesStr = names.stream().collect(Collectors.joining(COMMA));
|
||||
if (readColumnsBuffer.length() > 0) {
|
||||
readColumnsBuffer.append(COMMA);
|
||||
}
|
||||
readColumnsBuffer.append(preppedIdStr);
|
||||
if (readColumnNamesBuffer.length() > 0) {
|
||||
readColumnNamesBuffer.append(COMMA);
|
||||
}
|
||||
readColumnNamesBuffer.append(preppedNamesStr);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an array of column ids(start from zero) which is set in the given
|
||||
* parameter <tt>conf</tt>.
|
||||
*/
|
||||
public static List<Integer> getReadColumnIDs(Configuration conf) {
|
||||
String skips = conf.get(READ_COLUMN_IDS_CONF_STR, READ_COLUMN_IDS_CONF_STR_DEFAULT);
|
||||
String[] list = StringUtils.split(skips);
|
||||
List<Integer> result = new ArrayList<Integer>(list.length);
|
||||
for (String element : list) {
|
||||
// it may contain duplicates, remove duplicates
|
||||
Integer toAdd = Integer.parseInt(element);
|
||||
if (!result.contains(toAdd)) {
|
||||
result.add(toAdd);
|
||||
}
|
||||
// NOTE: some code uses this list to correlate with column names, and yet these lists may
|
||||
// contain duplicates, which this call will remove and the other won't. As far as I can
|
||||
// tell, no code will actually use these two methods together; all is good if the code
|
||||
// gets the ID list without relying on this method. Or maybe it just works by magic.
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public static Set<String> getNestedColumnPaths(Configuration conf) {
|
||||
String skips =
|
||||
conf.get(READ_NESTED_COLUMN_PATH_CONF_STR, READ_NESTED_COLUMN_PATH_CONF_STR_DEFAULT);
|
||||
return new HashSet<>(Arrays.asList(StringUtils.split(skips)));
|
||||
}
|
||||
|
||||
public static String[] getReadColumnNames(Configuration conf) {
|
||||
String colNames = conf.get(READ_COLUMN_NAMES_CONF_STR, READ_COLUMN_NAMES_CONF_STR_DEFAULT);
|
||||
if (colNames != null && !colNames.isEmpty()) {
|
||||
return colNames.split(",");
|
||||
}
|
||||
return new String[] {};
|
||||
}
|
||||
|
||||
public static List<String> getIOColumns(Configuration conf) {
|
||||
String colNames = conf.get(IOConstants.COLUMNS, "");
|
||||
if (colNames != null && !colNames.isEmpty()) {
|
||||
return Arrays.asList(colNames.split(","));
|
||||
}
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
public static List<String> getIOColumnTypes(Configuration conf) {
|
||||
String colTypes = conf.get(IOConstants.COLUMNS_TYPES, "");
|
||||
if (colTypes != null && !colTypes.isEmpty()) {
|
||||
return TypeInfoUtils.getTypeInfosFromTypeString(colTypes).stream()
|
||||
.map(t -> t.getTypeName()).collect(Collectors.toList());
|
||||
}
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
public static List<Pair<String,String>> getIOColumnNameAndTypes(Configuration conf) {
|
||||
List<String> names = getIOColumns(conf);
|
||||
List<String> types = getIOColumnTypes(conf);
|
||||
ValidationUtils.checkArgument(names.size() == types.size());
|
||||
return IntStream.range(0, names.size()).mapToObj(idx -> Pair.of(names.get(idx), types.get(idx)))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public static void setIOColumnNameAndTypes(Configuration conf, List<Pair<String,String>> colNamesAndTypes) {
|
||||
String colNames = colNamesAndTypes.stream().map(e -> e.getKey()).collect(Collectors.joining(","));
|
||||
String colTypes = colNamesAndTypes.stream().map(e -> e.getValue()).collect(Collectors.joining(","));
|
||||
conf.set(IOConstants.COLUMNS, colNames);
|
||||
conf.set(IOConstants.COLUMNS_TYPES, colTypes);
|
||||
}
|
||||
|
||||
private static void setReadColumnIDConf(Configuration conf, String id) {
|
||||
if (id.trim().isEmpty()) {
|
||||
conf.set(READ_COLUMN_IDS_CONF_STR, READ_COLUMN_IDS_CONF_STR_DEFAULT);
|
||||
} else {
|
||||
conf.set(READ_COLUMN_IDS_CONF_STR, id);
|
||||
}
|
||||
}
|
||||
|
||||
private static void setReadColumnNamesConf(Configuration conf, String id) {
|
||||
if (id.trim().isEmpty()) {
|
||||
conf.set(READ_COLUMN_NAMES_CONF_STR, READ_COLUMN_NAMES_CONF_STR_DEFAULT);
|
||||
} else {
|
||||
conf.set(READ_COLUMN_NAMES_CONF_STR, id);
|
||||
}
|
||||
}
|
||||
|
||||
private static void setReadNestedColumnPathConf(
|
||||
Configuration conf,
|
||||
String nestedColumnPaths) {
|
||||
nestedColumnPaths = nestedColumnPaths.toLowerCase();
|
||||
if (nestedColumnPaths.trim().isEmpty()) {
|
||||
conf.set(READ_NESTED_COLUMN_PATH_CONF_STR, READ_NESTED_COLUMN_PATH_CONF_STR_DEFAULT);
|
||||
} else {
|
||||
conf.set(READ_NESTED_COLUMN_PATH_CONF_STR, nestedColumnPaths);
|
||||
}
|
||||
}
|
||||
|
||||
private static void appendReadColumnNames(Configuration conf, List<String> cols) {
|
||||
String old = conf.get(READ_COLUMN_NAMES_CONF_STR, "");
|
||||
StringBuilder result = new StringBuilder(old);
|
||||
boolean first = old.isEmpty();
|
||||
for (String col: cols) {
|
||||
if (first) {
|
||||
first = false;
|
||||
} else {
|
||||
result.append(',');
|
||||
}
|
||||
result.append(col);
|
||||
}
|
||||
conf.set(READ_COLUMN_NAMES_CONF_STR, result.toString());
|
||||
}
|
||||
|
||||
private static String toReadColumnIDString(List<Integer> ids) {
|
||||
String id = "";
|
||||
for (int i = 0; i < ids.size(); i++) {
|
||||
if (i == 0) {
|
||||
id = id + ids.get(i);
|
||||
} else {
|
||||
id = id + StringUtils.COMMA_STR + ids.get(i);
|
||||
}
|
||||
}
|
||||
return id;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -19,21 +19,29 @@
|
||||
package org.apache.hudi.hadoop;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
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.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
|
||||
|
||||
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.hive.ql.io.parquet.MapredParquetInputFormat;
|
||||
import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
|
||||
import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
|
||||
import org.apache.hadoop.hive.ql.plan.TableScanDesc;
|
||||
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;
|
||||
@@ -47,6 +55,8 @@ import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
/**
|
||||
* HoodieInputFormat which understands the Hoodie File Structure and filters files based on the Hoodie Mode. If paths
|
||||
@@ -115,6 +125,8 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
|
||||
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
|
||||
@@ -153,7 +165,7 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
|
||||
|
||||
@Override
|
||||
public RecordReader<NullWritable, ArrayWritable> getRecordReader(final InputSplit split, final JobConf job,
|
||||
final Reporter reporter) throws IOException {
|
||||
final Reporter reporter) throws IOException {
|
||||
// TODO enable automatic predicate pushdown after fixing issues
|
||||
// FileSplit fileSplit = (FileSplit) split;
|
||||
// HoodieTableMetadata metadata = getTableMetadata(fileSplit.getPath().getParent());
|
||||
@@ -166,7 +178,91 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
|
||||
// ParquetInputFormat.setFilterPredicate(job, predicate);
|
||||
// clearOutExistingPredicate(job);
|
||||
// }
|
||||
if (split instanceof BootstrapBaseFileSplit) {
|
||||
BootstrapBaseFileSplit eSplit = (BootstrapBaseFileSplit)split;
|
||||
String[] rawColNames = HoodieColumnProjectionUtils.getReadColumnNames(job);
|
||||
List<Integer> rawColIds = HoodieColumnProjectionUtils.getReadColumnIDs(job);
|
||||
List<Pair<Integer, String>> projectedColsWithIndex =
|
||||
IntStream.range(0, rawColIds.size()).mapToObj(idx -> Pair.of(rawColIds.get(idx), rawColNames[idx]))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
List<Pair<Integer, String>> hoodieColsProjected = projectedColsWithIndex.stream()
|
||||
.filter(idxWithName -> HoodieRecord.HOODIE_META_COLUMNS.contains(idxWithName.getValue()))
|
||||
.collect(Collectors.toList());
|
||||
List<Pair<Integer, String>> externalColsProjected = projectedColsWithIndex.stream()
|
||||
.filter(idxWithName -> !HoodieRecord.HOODIE_META_COLUMNS.contains(idxWithName.getValue())
|
||||
&& !VirtualColumn.VIRTUAL_COLUMN_NAMES.contains(idxWithName.getValue()))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// This always matches hive table description
|
||||
List<Pair<String, String>> colNameWithTypes = HoodieColumnProjectionUtils.getIOColumnNameAndTypes(job);
|
||||
List<Pair<String, String>> colNamesWithTypesForExternal = colNameWithTypes.stream()
|
||||
.filter(p -> !HoodieRecord.HOODIE_META_COLUMNS.contains(p.getKey())).collect(Collectors.toList());
|
||||
LOG.info("colNameWithTypes =" + colNameWithTypes + ", Num Entries =" + colNameWithTypes.size());
|
||||
if (hoodieColsProjected.isEmpty()) {
|
||||
return super.getRecordReader(eSplit.getBootstrapFileSplit(), job, reporter);
|
||||
} else if (externalColsProjected.isEmpty()) {
|
||||
return super.getRecordReader(split, job, reporter);
|
||||
} else {
|
||||
FileSplit rightSplit = eSplit.getBootstrapFileSplit();
|
||||
// Hive PPD works at row-group level and only enabled when hive.optimize.index.filter=true;
|
||||
// The above config is disabled by default. But when enabled, would cause misalignment between
|
||||
// skeleton and bootstrap file. We will disable them specifically when query needs bootstrap and skeleton
|
||||
// file to be stitched.
|
||||
// This disables row-group filtering
|
||||
JobConf jobConfCopy = new JobConf(job);
|
||||
jobConfCopy.unset(TableScanDesc.FILTER_EXPR_CONF_STR);
|
||||
jobConfCopy.unset(ConvertAstToSearchArg.SARG_PUSHDOWN);
|
||||
|
||||
LOG.info("Generating column stitching reader for " + eSplit.getPath() + " and " + rightSplit.getPath());
|
||||
return new BootstrapColumnStichingRecordReader(super.getRecordReader(eSplit, jobConfCopy, reporter),
|
||||
HoodieRecord.HOODIE_META_COLUMNS.size(),
|
||||
super.getRecordReader(rightSplit, jobConfCopy, reporter),
|
||||
colNamesWithTypesForExternal.size(),
|
||||
true);
|
||||
}
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("EMPLOYING DEFAULT RECORD READER - " + split);
|
||||
}
|
||||
return super.getRecordReader(split, job, reporter);
|
||||
}
|
||||
|
||||
}
|
||||
@Override
|
||||
protected boolean isSplitable(FileSystem fs, Path filename) {
|
||||
return !(filename instanceof PathWithBootstrapFileStatus);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FileSplit makeSplit(Path file, long start, long length,
|
||||
String[] hosts) {
|
||||
FileSplit split = new FileSplit(file, start, length, hosts);
|
||||
|
||||
if (file instanceof PathWithBootstrapFileStatus) {
|
||||
return makeExternalFileSplit((PathWithBootstrapFileStatus)file, split);
|
||||
}
|
||||
return split;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FileSplit makeSplit(Path file, long start, long length,
|
||||
String[] hosts, String[] inMemoryHosts) {
|
||||
FileSplit split = new FileSplit(file, start, length, hosts, inMemoryHosts);
|
||||
if (file instanceof PathWithBootstrapFileStatus) {
|
||||
return makeExternalFileSplit((PathWithBootstrapFileStatus)file, split);
|
||||
}
|
||||
return split;
|
||||
}
|
||||
|
||||
private BootstrapBaseFileSplit makeExternalFileSplit(PathWithBootstrapFileStatus file, FileSplit split) {
|
||||
try {
|
||||
LOG.info("Making external data split for " + file);
|
||||
FileStatus externalFileStatus = file.getBootstrapFileStatus();
|
||||
FileSplit externalFileSplit = makeSplit(externalFileStatus.getPath(), 0, externalFileStatus.getLen(),
|
||||
new String[0], new String[0]);
|
||||
return new BootstrapBaseFileSplit(split, externalFileSplit);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,67 @@
|
||||
/*
|
||||
* 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.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
|
||||
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
|
||||
|
||||
public class InputSplitUtils {
|
||||
|
||||
public static void writeString(String str, DataOutput out) throws IOException {
|
||||
byte[] bytes = str.getBytes(StandardCharsets.UTF_8);
|
||||
out.writeInt(bytes.length);
|
||||
out.write(bytes);
|
||||
}
|
||||
|
||||
public static String readString(DataInput in) throws IOException {
|
||||
byte[] bytes = new byte[in.readInt()];
|
||||
in.readFully(bytes);
|
||||
return new String(bytes, StandardCharsets.UTF_8);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return correct base-file schema based on split.
|
||||
*
|
||||
* @param split File Split
|
||||
* @param conf Configuration
|
||||
* @return
|
||||
*/
|
||||
public static Schema getBaseFileSchema(FileSplit split, Configuration conf) {
|
||||
try {
|
||||
if (split instanceof BootstrapBaseFileSplit) {
|
||||
HoodieFileReader storageReader = HoodieFileReaderFactory.getFileReader(conf,
|
||||
((BootstrapBaseFileSplit)(split)).getBootstrapFileSplit().getPath());
|
||||
return HoodieAvroUtils.addMetadataFields(storageReader.getSchema());
|
||||
}
|
||||
return HoodieRealtimeRecordReaderUtils.readSchema(conf, split.getPath());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to read footer for parquet " + split.getPath(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Sub-Type of File Status tracking both skeleton and bootstrap base file's status.
|
||||
*/
|
||||
public class LocatedFileStatusWithBootstrapBaseFile extends LocatedFileStatus {
|
||||
|
||||
private final FileStatus bootstrapFileStatus;
|
||||
|
||||
public LocatedFileStatusWithBootstrapBaseFile(LocatedFileStatus fileStatus, FileStatus bootstrapFileStatus)
|
||||
throws IOException {
|
||||
super(fileStatus, fileStatus.getBlockLocations());
|
||||
this.bootstrapFileStatus = bootstrapFileStatus;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getPath() {
|
||||
return new PathWithBootstrapFileStatus(super.getPath(), bootstrapFileStatus);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
/**
|
||||
* Hacky Workaround !!!
|
||||
* With the base input format implementations in Hadoop/Hive,
|
||||
* we need to encode additional information in Path to track matching external file.
|
||||
* Hence, this weird looking class which tracks an external file status
|
||||
* in Path.
|
||||
*/
|
||||
public class PathWithBootstrapFileStatus extends Path {
|
||||
|
||||
private final FileStatus bootstrapFileStatus;
|
||||
|
||||
public PathWithBootstrapFileStatus(Path path, FileStatus bootstrapFileStatus) {
|
||||
super(path.getParent(), path.getName());
|
||||
this.bootstrapFileStatus = bootstrapFileStatus;
|
||||
}
|
||||
|
||||
public FileStatus getBootstrapFileStatus() {
|
||||
return bootstrapFileStatus;
|
||||
}
|
||||
}
|
||||
@@ -22,6 +22,7 @@ import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.log.LogReaderUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hadoop.InputSplitUtils;
|
||||
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
|
||||
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
|
||||
|
||||
@@ -29,6 +30,7 @@ import org.apache.avro.Schema;
|
||||
import org.apache.avro.Schema.Field;
|
||||
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
|
||||
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -46,7 +48,7 @@ import java.util.stream.Collectors;
|
||||
public abstract class AbstractRealtimeRecordReader {
|
||||
private static final Logger LOG = LogManager.getLogger(AbstractRealtimeRecordReader.class);
|
||||
|
||||
protected final HoodieRealtimeFileSplit split;
|
||||
protected final RealtimeSplit split;
|
||||
protected final JobConf jobConf;
|
||||
protected final boolean usesCustomPayload;
|
||||
// Schema handles
|
||||
@@ -54,7 +56,7 @@ public abstract class AbstractRealtimeRecordReader {
|
||||
private Schema writerSchema;
|
||||
private Schema hiveSchema;
|
||||
|
||||
public AbstractRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job) {
|
||||
public AbstractRealtimeRecordReader(RealtimeSplit split, JobConf job) {
|
||||
this.split = split;
|
||||
this.jobConf = job;
|
||||
LOG.info("cfg ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR));
|
||||
@@ -81,14 +83,13 @@ public abstract class AbstractRealtimeRecordReader {
|
||||
* job conf.
|
||||
*/
|
||||
private void init() throws IOException {
|
||||
Schema schemaFromLogFile =
|
||||
LogReaderUtils.readLatestSchemaFromLogFiles(split.getBasePath(), split.getDeltaLogPaths(), jobConf);
|
||||
Schema schemaFromLogFile = LogReaderUtils.readLatestSchemaFromLogFiles(split.getBasePath(), split.getDeltaLogPaths(), jobConf);
|
||||
if (schemaFromLogFile == null) {
|
||||
writerSchema = HoodieRealtimeRecordReaderUtils.readSchema(jobConf, split.getPath());
|
||||
LOG.debug("Writer Schema From Parquet => " + writerSchema.getFields());
|
||||
writerSchema = InputSplitUtils.getBaseFileSchema((FileSplit)split, jobConf);
|
||||
LOG.info("Writer Schema From Parquet => " + writerSchema.getFields());
|
||||
} else {
|
||||
writerSchema = schemaFromLogFile;
|
||||
LOG.debug("Writer Schema From Log => " + writerSchema.getFields());
|
||||
LOG.info("Writer Schema From Log => " + writerSchema.toString(true));
|
||||
}
|
||||
// Add partitioning fields to writer schema for resulting row to contain null values for these fields
|
||||
String partitionFields = jobConf.get(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "");
|
||||
|
||||
@@ -21,6 +21,7 @@ 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;
|
||||
@@ -42,7 +43,9 @@ 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;
|
||||
|
||||
/**
|
||||
@@ -110,10 +113,20 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
|
||||
}
|
||||
|
||||
private 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);
|
||||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -134,7 +147,7 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
|
||||
|
||||
@Override
|
||||
public RecordReader<NullWritable, ArrayWritable> getRecordReader(final InputSplit split, final JobConf jobConf,
|
||||
final Reporter reporter) throws IOException {
|
||||
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
|
||||
@@ -164,11 +177,12 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
|
||||
|
||||
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,
|
||||
// sanity check
|
||||
ValidationUtils.checkArgument(split instanceof RealtimeSplit,
|
||||
"HoodieRealtimeRecordReader can only work on RealtimeSplit and not with " + split);
|
||||
|
||||
return new HoodieRealtimeRecordReader((RealtimeSplit) split, jobConf,
|
||||
super.getRecordReader(split, jobConf, reporter));
|
||||
}
|
||||
|
||||
@@ -176,4 +190,4 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
|
||||
public Configuration getConf() {
|
||||
return conf;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -23,14 +23,12 @@ import org.apache.hadoop.mapred.FileSplit;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Filesplit that wraps the base split and a list of log files to merge deltas from.
|
||||
*/
|
||||
public class HoodieRealtimeFileSplit extends FileSplit {
|
||||
public class HoodieRealtimeFileSplit extends FileSplit implements RealtimeSplit {
|
||||
|
||||
private List<String> deltaLogPaths;
|
||||
|
||||
@@ -62,39 +60,28 @@ public class HoodieRealtimeFileSplit extends FileSplit {
|
||||
return basePath;
|
||||
}
|
||||
|
||||
private static void writeString(String str, DataOutput out) throws IOException {
|
||||
byte[] bytes = str.getBytes(StandardCharsets.UTF_8);
|
||||
out.writeInt(bytes.length);
|
||||
out.write(bytes);
|
||||
public void setDeltaLogPaths(List<String> deltaLogPaths) {
|
||||
this.deltaLogPaths = deltaLogPaths;
|
||||
}
|
||||
|
||||
private static String readString(DataInput in) throws IOException {
|
||||
byte[] bytes = new byte[in.readInt()];
|
||||
in.readFully(bytes);
|
||||
return new String(bytes, StandardCharsets.UTF_8);
|
||||
public void setMaxCommitTime(String maxCommitTime) {
|
||||
this.maxCommitTime = maxCommitTime;
|
||||
}
|
||||
|
||||
public void setBasePath(String basePath) {
|
||||
this.basePath = basePath;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
super.write(out);
|
||||
writeString(basePath, out);
|
||||
writeString(maxCommitTime, out);
|
||||
out.writeInt(deltaLogPaths.size());
|
||||
for (String logFilePath : deltaLogPaths) {
|
||||
writeString(logFilePath, out);
|
||||
}
|
||||
writeToOutput(out);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
super.readFields(in);
|
||||
basePath = readString(in);
|
||||
maxCommitTime = readString(in);
|
||||
int totalLogFiles = in.readInt();
|
||||
deltaLogPaths = new ArrayList<>(totalLogFiles);
|
||||
for (int i = 0; i < totalLogFiles; i++) {
|
||||
deltaLogPaths.add(readString(in));
|
||||
}
|
||||
readFromInput(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -42,7 +42,7 @@ public class HoodieRealtimeRecordReader implements RecordReader<NullWritable, Ar
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieRealtimeRecordReader.class);
|
||||
private final RecordReader<NullWritable, ArrayWritable> reader;
|
||||
|
||||
public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job,
|
||||
public HoodieRealtimeRecordReader(RealtimeSplit split, JobConf job,
|
||||
RecordReader<NullWritable, ArrayWritable> realReader) {
|
||||
this.reader = constructRecordReader(split, job, realReader);
|
||||
}
|
||||
@@ -59,7 +59,7 @@ public class HoodieRealtimeRecordReader implements RecordReader<NullWritable, Ar
|
||||
* @param realReader Parquet Record Reader
|
||||
* @return Realtime Reader
|
||||
*/
|
||||
private static RecordReader<NullWritable, ArrayWritable> constructRecordReader(HoodieRealtimeFileSplit split,
|
||||
private static RecordReader<NullWritable, ArrayWritable> constructRecordReader(RealtimeSplit split,
|
||||
JobConf jobConf, RecordReader<NullWritable, ArrayWritable> realReader) {
|
||||
try {
|
||||
if (canSkipMerging(jobConf)) {
|
||||
|
||||
@@ -0,0 +1,94 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.hadoop.BootstrapBaseFileSplit;
|
||||
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Realtime File Split with external base file.
|
||||
*/
|
||||
public class RealtimeBootstrapBaseFileSplit extends BootstrapBaseFileSplit implements RealtimeSplit {
|
||||
|
||||
private List<String> deltaLogPaths;
|
||||
|
||||
private String maxInstantTime;
|
||||
|
||||
private String basePath;
|
||||
|
||||
public RealtimeBootstrapBaseFileSplit() {
|
||||
super();
|
||||
}
|
||||
|
||||
public RealtimeBootstrapBaseFileSplit(FileSplit baseSplit, String basePath, List<String> deltaLogPaths,
|
||||
String maxInstantTime, FileSplit externalFileSplit) throws IOException {
|
||||
super(baseSplit, externalFileSplit);
|
||||
this.maxInstantTime = maxInstantTime;
|
||||
this.deltaLogPaths = deltaLogPaths;
|
||||
this.basePath = basePath;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
super.write(out);
|
||||
writeToOutput(out);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
super.readFields(in);
|
||||
readFromInput(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getDeltaLogPaths() {
|
||||
return deltaLogPaths;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getMaxCommitTime() {
|
||||
return maxInstantTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getBasePath() {
|
||||
return basePath;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setDeltaLogPaths(List<String> deltaLogPaths) {
|
||||
this.deltaLogPaths = deltaLogPaths;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setMaxCommitTime(String maxInstantTime) {
|
||||
this.maxInstantTime = maxInstantTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBasePath(String basePath) {
|
||||
this.basePath = basePath;
|
||||
}
|
||||
}
|
||||
@@ -48,7 +48,7 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader
|
||||
protected final RecordReader<NullWritable, ArrayWritable> parquetReader;
|
||||
private final Map<String, HoodieRecord<? extends HoodieRecordPayload>> deltaRecordMap;
|
||||
|
||||
public RealtimeCompactedRecordReader(HoodieRealtimeFileSplit split, JobConf job,
|
||||
public RealtimeCompactedRecordReader(RealtimeSplit split, JobConf job,
|
||||
RecordReader<NullWritable, ArrayWritable> realReader) throws IOException {
|
||||
super(split, job);
|
||||
this.parquetReader = realReader;
|
||||
@@ -120,7 +120,10 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader
|
||||
}
|
||||
Writable[] originalValue = arrayWritable.get();
|
||||
try {
|
||||
System.arraycopy(replaceValue, 0, originalValue, 0, originalValue.length);
|
||||
// Sometime originalValue.length > replaceValue.length.
|
||||
// This can happen when hive query is looking for pseudo parquet columns like BLOCK_OFFSET_INSIDE_FILE
|
||||
System.arraycopy(replaceValue, 0, originalValue, 0,
|
||||
Math.min(originalValue.length, replaceValue.length));
|
||||
arrayWritable.set(originalValue);
|
||||
} catch (RuntimeException re) {
|
||||
LOG.error("Got exception when doing array copy", re);
|
||||
|
||||
@@ -0,0 +1,107 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.hadoop.InputSplitUtils;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.InputSplitWithLocationInfo;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Realtime Input Split Interface.
|
||||
*/
|
||||
public interface RealtimeSplit extends InputSplitWithLocationInfo {
|
||||
|
||||
/**
|
||||
* Return Log File Paths.
|
||||
* @return
|
||||
*/
|
||||
List<String> getDeltaLogPaths();
|
||||
|
||||
/**
|
||||
* Return Max Instant Time.
|
||||
* @return
|
||||
*/
|
||||
String getMaxCommitTime();
|
||||
|
||||
/**
|
||||
* Return Base Path of the dataset.
|
||||
* @return
|
||||
*/
|
||||
String getBasePath();
|
||||
|
||||
/**
|
||||
* Update Log File Paths.
|
||||
* @param deltaLogPaths
|
||||
*/
|
||||
void setDeltaLogPaths(List<String> deltaLogPaths);
|
||||
|
||||
/**
|
||||
* Update Maximum valid instant time.
|
||||
* @param maxCommitTime
|
||||
*/
|
||||
void setMaxCommitTime(String maxCommitTime);
|
||||
|
||||
/**
|
||||
* Set Base Path.
|
||||
* @param basePath
|
||||
*/
|
||||
void setBasePath(String basePath);
|
||||
|
||||
default void writeToOutput(DataOutput out) throws IOException {
|
||||
InputSplitUtils.writeString(getBasePath(), out);
|
||||
InputSplitUtils.writeString(getMaxCommitTime(), out);
|
||||
out.writeInt(getDeltaLogPaths().size());
|
||||
for (String logFilePath : getDeltaLogPaths()) {
|
||||
InputSplitUtils.writeString(logFilePath, out);
|
||||
}
|
||||
}
|
||||
|
||||
default void readFromInput(DataInput in) throws IOException {
|
||||
setBasePath(InputSplitUtils.readString(in));
|
||||
setMaxCommitTime(InputSplitUtils.readString(in));
|
||||
int totalLogFiles = in.readInt();
|
||||
List<String> deltaLogPaths = new ArrayList<>(totalLogFiles);
|
||||
for (int i = 0; i < totalLogFiles; i++) {
|
||||
deltaLogPaths.add(InputSplitUtils.readString(in));
|
||||
}
|
||||
setDeltaLogPaths(deltaLogPaths);
|
||||
}
|
||||
|
||||
/**
|
||||
* The file containing this split's data.
|
||||
*/
|
||||
public Path getPath();
|
||||
|
||||
/**
|
||||
* The position of the first byte in the file to process.
|
||||
*/
|
||||
public long getStart();
|
||||
|
||||
/**
|
||||
* The number of bytes in the file to process.
|
||||
*/
|
||||
public long getLength();
|
||||
}
|
||||
@@ -66,7 +66,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
|
||||
* @param job Job Configuration
|
||||
* @param realReader Parquet Reader
|
||||
*/
|
||||
public RealtimeUnmergedRecordReader(HoodieRealtimeFileSplit split, JobConf job,
|
||||
public RealtimeUnmergedRecordReader(RealtimeSplit split, JobConf job,
|
||||
RecordReader<NullWritable, ArrayWritable> realReader) {
|
||||
super(split, job);
|
||||
this.parquetReader = new SafeParquetRecordReaderWrapper(realReader);
|
||||
|
||||
@@ -32,11 +32,14 @@ import org.apache.hudi.common.table.view.TableFileSystemView;
|
||||
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.HoodieParquetInputFormat;
|
||||
import org.apache.hudi.hadoop.LocatedFileStatusWithBootstrapBaseFile;
|
||||
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat;
|
||||
import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
|
||||
@@ -108,7 +111,7 @@ public class HoodieInputFormatUtils {
|
||||
}
|
||||
|
||||
public static FileInputFormat getInputFormat(String path, boolean realtime, Configuration conf) {
|
||||
final String extension = FSUtils.getFileExtension(path.toString());
|
||||
final String extension = FSUtils.getFileExtension(path);
|
||||
if (extension.equals(HoodieFileFormat.PARQUET.getFileExtension())) {
|
||||
return getInputFormat(HoodieFileFormat.PARQUET, realtime, conf);
|
||||
}
|
||||
@@ -281,6 +284,19 @@ public class HoodieInputFormatUtils {
|
||||
return new HoodieTableMetaClient(fs.getConf(), baseDir.toString());
|
||||
}
|
||||
|
||||
public static FileStatus getFileStatus(HoodieBaseFile baseFile) throws IOException {
|
||||
if (baseFile.getBootstrapBaseFile().isPresent()) {
|
||||
if (baseFile.getFileStatus() instanceof LocatedFileStatus) {
|
||||
return new LocatedFileStatusWithBootstrapBaseFile((LocatedFileStatus)baseFile.getFileStatus(),
|
||||
baseFile.getBootstrapBaseFile().get().getFileStatus());
|
||||
} else {
|
||||
return new FileStatusWithBootstrapBaseFile(baseFile.getFileStatus(),
|
||||
baseFile.getBootstrapBaseFile().get().getFileStatus());
|
||||
}
|
||||
}
|
||||
return baseFile.getFileStatus();
|
||||
}
|
||||
|
||||
/**
|
||||
* Filter a list of FileStatus based on commitsToCheck for incremental view.
|
||||
* @param job
|
||||
@@ -291,7 +307,7 @@ public class HoodieInputFormatUtils {
|
||||
* @return
|
||||
*/
|
||||
public static List<FileStatus> filterIncrementalFileStatus(Job job, HoodieTableMetaClient tableMetaClient,
|
||||
HoodieTimeline timeline, FileStatus[] fileStatuses, List<HoodieInstant> commitsToCheck) {
|
||||
HoodieTimeline timeline, FileStatus[] fileStatuses, List<HoodieInstant> commitsToCheck) throws IOException {
|
||||
TableFileSystemView.BaseFileOnlyView roView = new HoodieTableFileSystemView(tableMetaClient, timeline, fileStatuses);
|
||||
List<String> commitsList = commitsToCheck.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
List<HoodieBaseFile> filteredFiles = roView.getLatestBaseFilesInRange(commitsList).collect(Collectors.toList());
|
||||
@@ -299,7 +315,7 @@ public class HoodieInputFormatUtils {
|
||||
for (HoodieBaseFile filteredFile : filteredFiles) {
|
||||
LOG.debug("Processing incremental hoodie file - " + filteredFile.getPath());
|
||||
filteredFile = refreshFileStatus(job.getConfiguration(), filteredFile);
|
||||
returns.add(filteredFile.getFileStatus());
|
||||
returns.add(getFileStatus(filteredFile));
|
||||
}
|
||||
LOG.info("Total paths to process after hoodie incremental filter " + filteredFiles.size());
|
||||
return returns;
|
||||
@@ -350,7 +366,7 @@ public class HoodieInputFormatUtils {
|
||||
* @return
|
||||
*/
|
||||
public static List<FileStatus> filterFileStatusForSnapshotMode(
|
||||
JobConf job, HoodieTableMetaClient metadata, List<FileStatus> fileStatuses) {
|
||||
JobConf job, HoodieTableMetaClient metadata, List<FileStatus> fileStatuses) throws IOException {
|
||||
FileStatus[] statuses = fileStatuses.toArray(new FileStatus[0]);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metadata);
|
||||
@@ -367,7 +383,7 @@ public class HoodieInputFormatUtils {
|
||||
LOG.debug("Processing latest hoodie file - " + filteredFile.getPath());
|
||||
}
|
||||
filteredFile = refreshFileStatus(job, filteredFile);
|
||||
returns.add(filteredFile.getFileStatus());
|
||||
returns.add(getFileStatus(filteredFile));
|
||||
}
|
||||
return returns;
|
||||
}
|
||||
@@ -386,7 +402,7 @@ public class HoodieInputFormatUtils {
|
||||
if (dataFile.getFileSize() == 0) {
|
||||
FileSystem fs = dataPath.getFileSystem(conf);
|
||||
LOG.info("Refreshing file status " + dataFile.getPath());
|
||||
return new HoodieBaseFile(fs.getFileStatus(dataPath));
|
||||
return new HoodieBaseFile(fs.getFileStatus(dataPath), dataFile.getBootstrapBaseFile().orElse(null));
|
||||
}
|
||||
return dataFile;
|
||||
} catch (IOException e) {
|
||||
|
||||
@@ -29,18 +29,22 @@ import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hadoop.BootstrapBaseFileSplit;
|
||||
import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit;
|
||||
import org.apache.hudi.hadoop.realtime.RealtimeBootstrapBaseFileSplit;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.SplitLocationInfo;
|
||||
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.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -59,7 +63,7 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
|
||||
|
||||
// for all unique split parents, obtain all delta files based on delta commit timeline,
|
||||
// grouped on file id
|
||||
List<HoodieRealtimeFileSplit> rtSplits = new ArrayList<>();
|
||||
List<InputSplit> rtSplits = new ArrayList<>();
|
||||
partitionsToParquetSplits.keySet().forEach(partitionPath -> {
|
||||
// for each partition path obtain the data & log file groupings, then map back to inputsplits
|
||||
HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath);
|
||||
@@ -78,19 +82,29 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
|
||||
// subgroup splits again by file id & match with log files.
|
||||
Map<String, List<FileSplit>> groupedInputSplits = partitionsToParquetSplits.get(partitionPath).stream()
|
||||
.collect(Collectors.groupingBy(split -> FSUtils.getFileId(split.getPath().getName())));
|
||||
// Get the maxCommit from the last delta or compaction or commit - when bootstrapped from COW table
|
||||
String maxCommitTime = metaClient.getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION,
|
||||
HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
latestFileSlices.forEach(fileSlice -> {
|
||||
List<FileSplit> dataFileSplits = groupedInputSplits.get(fileSlice.getFileId());
|
||||
dataFileSplits.forEach(split -> {
|
||||
try {
|
||||
List<String> logFilePaths = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
|
||||
.map(logFile -> logFile.getPath().toString()).collect(Collectors.toList());
|
||||
// Get the maxCommit from the last delta or compaction or commit - when
|
||||
// bootstrapped from COW table
|
||||
String maxCommitTime = metaClient
|
||||
.getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION,
|
||||
HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
rtSplits.add(new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFilePaths, maxCommitTime));
|
||||
if (split instanceof BootstrapBaseFileSplit) {
|
||||
BootstrapBaseFileSplit eSplit = (BootstrapBaseFileSplit) split;
|
||||
String[] hosts = split.getLocationInfo() != null ? Arrays.stream(split.getLocationInfo())
|
||||
.filter(x -> !x.isInMemory()).toArray(String[]::new) : new String[0];
|
||||
String[] inMemoryHosts = split.getLocationInfo() != null ? Arrays.stream(split.getLocationInfo())
|
||||
.filter(SplitLocationInfo::isInMemory).toArray(String[]::new) : new String[0];
|
||||
FileSplit baseSplit = new FileSplit(eSplit.getPath(), eSplit.getStart(), eSplit.getLength(),
|
||||
hosts, inMemoryHosts);
|
||||
rtSplits.add(new RealtimeBootstrapBaseFileSplit(baseSplit,metaClient.getBasePath(),
|
||||
logFilePaths, maxCommitTime, eSplit.getBootstrapFileSplit()));
|
||||
} else {
|
||||
rtSplits.add(new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFilePaths, maxCommitTime));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Error creating hoodie real time split ", e);
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user