1
0

[HUDI-3993] Replacing UDF in Bulk Insert w/ RDD transformation (#5470)

This commit is contained in:
Alexey Kudinkin
2022-07-21 06:20:47 -07:00
committed by GitHub
parent c7fe3fd01d
commit a33bdd32e3
41 changed files with 1180 additions and 870 deletions

View File

@@ -24,31 +24,66 @@ import org.apache.spark.sql.catalyst.util.ArrayData;
import org.apache.spark.sql.catalyst.util.MapData;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.Decimal;
import org.apache.spark.sql.types.StringType$;
import org.apache.spark.unsafe.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;
import java.util.Arrays;
/**
* Internal Row implementation for Hoodie Row. It wraps an {@link InternalRow} and keeps meta columns locally. But the {@link InternalRow}
* does include the meta columns as well just that {@link HoodieInternalRow} will intercept queries for meta columns and serve from its
* copy rather than fetching from {@link InternalRow}.
* Hudi internal implementation of the {@link InternalRow} allowing to extend arbitrary
* {@link InternalRow} overlaying Hudi-internal meta-fields on top of it.
*
* Capable of overlaying meta-fields in both cases: whether original {@link #row} contains
* meta columns or not. This allows to handle following use-cases allowing to avoid any
* manipulation (reshuffling) of the source row, by simply creating new instance
* of {@link HoodieInternalRow} with all the meta-values provided
*
* <ul>
* <li>When meta-fields need to be prepended to the source {@link InternalRow}</li>
* <li>When meta-fields need to be updated w/in the source {@link InternalRow}
* ({@link org.apache.spark.sql.catalyst.expressions.UnsafeRow} currently does not
* allow in-place updates due to its memory layout)</li>
* </ul>
*/
public class HoodieInternalRow extends InternalRow {
private String commitTime;
private String commitSeqNumber;
private String recordKey;
private String partitionPath;
private String fileName;
private InternalRow row;
/**
* Collection of meta-fields as defined by {@link HoodieRecord#HOODIE_META_COLUMNS}
*/
private final UTF8String[] metaFields;
private final InternalRow row;
/**
* Specifies whether source {@link #row} contains meta-fields
*/
private final boolean containsMetaFields;
public HoodieInternalRow(UTF8String commitTime,
UTF8String commitSeqNumber,
UTF8String recordKey,
UTF8String partitionPath,
UTF8String fileName,
InternalRow row,
boolean containsMetaFields) {
this.metaFields = new UTF8String[] {
commitTime,
commitSeqNumber,
recordKey,
partitionPath,
fileName
};
public HoodieInternalRow(String commitTime, String commitSeqNumber, String recordKey, String partitionPath,
String fileName, InternalRow row) {
this.commitTime = commitTime;
this.commitSeqNumber = commitSeqNumber;
this.recordKey = recordKey;
this.partitionPath = partitionPath;
this.fileName = fileName;
this.row = row;
this.containsMetaFields = containsMetaFields;
}
private HoodieInternalRow(UTF8String[] metaFields,
InternalRow row,
boolean containsMetaFields) {
this.metaFields = metaFields;
this.row = row;
this.containsMetaFields = containsMetaFields;
}
@Override
@@ -57,187 +92,153 @@ public class HoodieInternalRow extends InternalRow {
}
@Override
public void setNullAt(int i) {
if (i < HoodieRecord.HOODIE_META_COLUMNS.size()) {
switch (i) {
case 0: {
this.commitTime = null;
break;
}
case 1: {
this.commitSeqNumber = null;
break;
}
case 2: {
this.recordKey = null;
break;
}
case 3: {
this.partitionPath = null;
break;
}
case 4: {
this.fileName = null;
break;
}
default: throw new IllegalArgumentException("Not expected");
}
public void setNullAt(int ordinal) {
if (ordinal < metaFields.length) {
metaFields[ordinal] = null;
} else {
row.setNullAt(i);
row.setNullAt(rebaseOrdinal(ordinal));
}
}
@Override
public void update(int i, Object value) {
if (i < HoodieRecord.HOODIE_META_COLUMNS.size()) {
switch (i) {
case 0: {
this.commitTime = value.toString();
break;
}
case 1: {
this.commitSeqNumber = value.toString();
break;
}
case 2: {
this.recordKey = value.toString();
break;
}
case 3: {
this.partitionPath = value.toString();
break;
}
case 4: {
this.fileName = value.toString();
break;
}
default: throw new IllegalArgumentException("Not expected");
public void update(int ordinal, Object value) {
if (ordinal < metaFields.length) {
if (value instanceof UTF8String) {
metaFields[ordinal] = (UTF8String) value;
} else if (value instanceof String) {
metaFields[ordinal] = UTF8String.fromString((String) value);
} else {
throw new IllegalArgumentException(
String.format("Could not update the row at (%d) with value of type (%s), either UTF8String or String are expected", ordinal, value.getClass().getSimpleName()));
}
} else {
row.update(i, value);
}
}
private String getMetaColumnVal(int ordinal) {
switch (ordinal) {
case 0: {
return commitTime;
}
case 1: {
return commitSeqNumber;
}
case 2: {
return recordKey;
}
case 3: {
return partitionPath;
}
case 4: {
return fileName;
}
default: throw new IllegalArgumentException("Not expected");
row.update(rebaseOrdinal(ordinal), value);
}
}
@Override
public boolean isNullAt(int ordinal) {
if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) {
return null == getMetaColumnVal(ordinal);
if (ordinal < metaFields.length) {
return metaFields[ordinal] == null;
}
return row.isNullAt(ordinal);
}
@Override
public boolean getBoolean(int ordinal) {
return row.getBoolean(ordinal);
}
@Override
public byte getByte(int ordinal) {
return row.getByte(ordinal);
}
@Override
public short getShort(int ordinal) {
return row.getShort(ordinal);
}
@Override
public int getInt(int ordinal) {
return row.getInt(ordinal);
}
@Override
public long getLong(int ordinal) {
return row.getLong(ordinal);
}
@Override
public float getFloat(int ordinal) {
return row.getFloat(ordinal);
}
@Override
public double getDouble(int ordinal) {
return row.getDouble(ordinal);
}
@Override
public Decimal getDecimal(int ordinal, int precision, int scale) {
return row.getDecimal(ordinal, precision, scale);
return row.isNullAt(rebaseOrdinal(ordinal));
}
@Override
public UTF8String getUTF8String(int ordinal) {
if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) {
return UTF8String.fromBytes(getMetaColumnVal(ordinal).getBytes());
return metaFields[ordinal];
}
return row.getUTF8String(ordinal);
}
@Override
public String getString(int ordinal) {
if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) {
return new String(getMetaColumnVal(ordinal).getBytes());
}
return row.getString(ordinal);
}
@Override
public byte[] getBinary(int ordinal) {
return row.getBinary(ordinal);
}
@Override
public CalendarInterval getInterval(int ordinal) {
return row.getInterval(ordinal);
}
@Override
public InternalRow getStruct(int ordinal, int numFields) {
return row.getStruct(ordinal, numFields);
}
@Override
public ArrayData getArray(int ordinal) {
return row.getArray(ordinal);
}
@Override
public MapData getMap(int ordinal) {
return row.getMap(ordinal);
return row.getUTF8String(rebaseOrdinal(ordinal));
}
@Override
public Object get(int ordinal, DataType dataType) {
if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) {
return UTF8String.fromBytes(getMetaColumnVal(ordinal).getBytes());
validateMetaFieldDataType(dataType);
return metaFields[ordinal];
}
return row.get(ordinal, dataType);
return row.get(rebaseOrdinal(ordinal), dataType);
}
@Override
public boolean getBoolean(int ordinal) {
ruleOutMetaFieldsAccess(ordinal, Boolean.class);
return row.getBoolean(rebaseOrdinal(ordinal));
}
@Override
public byte getByte(int ordinal) {
ruleOutMetaFieldsAccess(ordinal, Byte.class);
return row.getByte(rebaseOrdinal(ordinal));
}
@Override
public short getShort(int ordinal) {
ruleOutMetaFieldsAccess(ordinal, Short.class);
return row.getShort(rebaseOrdinal(ordinal));
}
@Override
public int getInt(int ordinal) {
ruleOutMetaFieldsAccess(ordinal, Integer.class);
return row.getInt(rebaseOrdinal(ordinal));
}
@Override
public long getLong(int ordinal) {
ruleOutMetaFieldsAccess(ordinal, Long.class);
return row.getLong(rebaseOrdinal(ordinal));
}
@Override
public float getFloat(int ordinal) {
ruleOutMetaFieldsAccess(ordinal, Float.class);
return row.getFloat(rebaseOrdinal(ordinal));
}
@Override
public double getDouble(int ordinal) {
ruleOutMetaFieldsAccess(ordinal, Double.class);
return row.getDouble(rebaseOrdinal(ordinal));
}
@Override
public Decimal getDecimal(int ordinal, int precision, int scale) {
ruleOutMetaFieldsAccess(ordinal, Decimal.class);
return row.getDecimal(rebaseOrdinal(ordinal), precision, scale);
}
@Override
public byte[] getBinary(int ordinal) {
ruleOutMetaFieldsAccess(ordinal, Byte[].class);
return row.getBinary(rebaseOrdinal(ordinal));
}
@Override
public CalendarInterval getInterval(int ordinal) {
ruleOutMetaFieldsAccess(ordinal, CalendarInterval.class);
return row.getInterval(rebaseOrdinal(ordinal));
}
@Override
public InternalRow getStruct(int ordinal, int numFields) {
ruleOutMetaFieldsAccess(ordinal, InternalRow.class);
return row.getStruct(rebaseOrdinal(ordinal), numFields);
}
@Override
public ArrayData getArray(int ordinal) {
ruleOutMetaFieldsAccess(ordinal, ArrayData.class);
return row.getArray(rebaseOrdinal(ordinal));
}
@Override
public MapData getMap(int ordinal) {
ruleOutMetaFieldsAccess(ordinal, MapData.class);
return row.getMap(rebaseOrdinal(ordinal));
}
@Override
public InternalRow copy() {
return new HoodieInternalRow(commitTime, commitSeqNumber, recordKey, partitionPath, fileName, row.copy());
return new HoodieInternalRow(Arrays.copyOf(metaFields, metaFields.length), row.copy(), containsMetaFields);
}
private int rebaseOrdinal(int ordinal) {
// NOTE: In cases when source row does not contain meta fields, we will have to
// rebase ordinal onto its indexes
return containsMetaFields ? ordinal : ordinal - metaFields.length;
}
private void validateMetaFieldDataType(DataType dataType) {
if (!dataType.sameType(StringType$.MODULE$)) {
throw new ClassCastException(String.format("Can not cast meta-field of type UTF8String to %s", dataType.simpleString()));
}
}
private void ruleOutMetaFieldsAccess(int ordinal, Class<?> expectedDataType) {
if (ordinal < metaFields.length) {
throw new ClassCastException(String.format("Can not cast meta-field of type UTF8String at (%d) as %s", ordinal, expectedDataType.getName()));
}
}
}

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.io.storage.row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.unsafe.types.UTF8String;
import java.io.IOException;
@@ -37,7 +38,7 @@ public interface HoodieInternalRowFileWriter {
*
* @throws IOException on any exception while writing.
*/
void writeRow(String key, InternalRow row) throws IOException;
void writeRow(UTF8String key, InternalRow row) throws IOException;
/**
* Writes an {@link InternalRow} to the HoodieInternalRowFileWriter.

View File

@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hudi.io.storage.HoodieParquetConfig;
import org.apache.hudi.io.storage.HoodieBaseParquetWriter;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.unsafe.types.UTF8String;
import java.io.IOException;
@@ -41,7 +42,7 @@ public class HoodieInternalRowParquetWriter extends HoodieBaseParquetWriter<Inte
}
@Override
public void writeRow(String key, InternalRow row) throws IOException {
public void writeRow(UTF8String key, InternalRow row) throws IOException {
super.write(row);
writeSupport.add(key);
}

View File

@@ -25,11 +25,11 @@ import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieInsertException;
import org.apache.hudi.hadoop.CachingPath;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.marker.WriteMarkersFactory;
@@ -39,10 +39,12 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.unsafe.types.UTF8String;
import java.io.IOException;
import java.io.Serializable;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
/**
* Create handle with InternalRow for datasource implementation of bulk insert.
@@ -50,38 +52,61 @@ import java.util.concurrent.atomic.AtomicLong;
public class HoodieRowCreateHandle implements Serializable {
private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(HoodieRowCreateHandle.class);
private static final AtomicLong SEQGEN = new AtomicLong(1);
private final String instantTime;
private final int taskPartitionId;
private final long taskId;
private final long taskEpochId;
private static final Logger LOG = LogManager.getLogger(HoodieRowCreateHandle.class);
private static final AtomicLong GLOBAL_SEQ_NO = new AtomicLong(1);
private static final Integer RECORD_KEY_META_FIELD_ORD =
HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.RECORD_KEY_METADATA_FIELD);
private static final Integer PARTITION_PATH_META_FIELD_ORD =
HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD);
private final HoodieTable table;
private final HoodieWriteConfig writeConfig;
protected final HoodieInternalRowFileWriter fileWriter;
private final String partitionPath;
private final Path path;
private final String fileId;
private final FileSystem fs;
protected final HoodieInternalWriteStatus writeStatus;
private final boolean populateMetaFields;
private final UTF8String fileName;
private final UTF8String commitTime;
private final Function<Long, String> seqIdGenerator;
private final HoodieTimer currTimer;
public HoodieRowCreateHandle(HoodieTable table, HoodieWriteConfig writeConfig, String partitionPath, String fileId,
String instantTime, int taskPartitionId, long taskId, long taskEpochId,
StructType structType) {
protected final HoodieInternalRowFileWriter fileWriter;
protected final HoodieInternalWriteStatus writeStatus;
public HoodieRowCreateHandle(HoodieTable table,
HoodieWriteConfig writeConfig,
String partitionPath,
String fileId,
String instantTime,
int taskPartitionId,
long taskId,
long taskEpochId,
StructType structType,
boolean populateMetaFields) {
this.partitionPath = partitionPath;
this.table = table;
this.writeConfig = writeConfig;
this.instantTime = instantTime;
this.taskPartitionId = taskPartitionId;
this.taskId = taskId;
this.taskEpochId = taskEpochId;
this.fileId = fileId;
this.currTimer = new HoodieTimer();
this.currTimer.startTimer();
this.fs = table.getMetaClient().getFs();
this.path = makeNewPath(partitionPath);
this.currTimer = new HoodieTimer(true);
FileSystem fs = table.getMetaClient().getFs();
String writeToken = getWriteToken(taskPartitionId, taskId, taskEpochId);
String fileName = FSUtils.makeBaseFileName(instantTime, writeToken, this.fileId, table.getBaseFileExtension());
this.path = makeNewPath(fs, partitionPath, fileName, writeConfig);
this.populateMetaFields = populateMetaFields;
this.fileName = UTF8String.fromString(path.getName());
this.commitTime = UTF8String.fromString(instantTime);
this.seqIdGenerator = (id) -> HoodieRecord.generateSequenceId(instantTime, taskPartitionId, id);
this.writeStatus = new HoodieInternalWriteStatus(!table.getIndex().isImplicitWithStorage(),
writeConfig.getWriteStatusFailureFraction());
writeStatus.setPartitionPath(partitionPath);
@@ -96,7 +121,7 @@ public class HoodieRowCreateHandle implements Serializable {
FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath),
table.getPartitionMetafileFormat());
partitionMetadata.trySave(taskPartitionId);
createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension()));
createMarkerFile(partitionPath, fileName, instantTime, table, writeConfig);
this.fileWriter = createNewFileWriter(path, table, writeConfig, structType);
} catch (IOException e) {
throw new HoodieInsertException("Failed to initialize file writer for path " + path, e);
@@ -108,21 +133,42 @@ public class HoodieRowCreateHandle implements Serializable {
* Writes an {@link InternalRow} to the underlying HoodieInternalRowFileWriter. Before writing, value for meta columns are computed as required
* and wrapped in {@link HoodieInternalRow}. {@link HoodieInternalRow} is what gets written to HoodieInternalRowFileWriter.
*
* @param record instance of {@link InternalRow} that needs to be written to the fileWriter.
* @param row instance of {@link InternalRow} that needs to be written to the fileWriter.
* @throws IOException
*/
public void write(InternalRow record) throws IOException {
public void write(InternalRow row) throws IOException {
try {
final String partitionPath = String.valueOf(record.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_POS));
final String seqId = HoodieRecord.generateSequenceId(instantTime, taskPartitionId, SEQGEN.getAndIncrement());
final String recordKey = String.valueOf(record.getUTF8String(HoodieRecord.RECORD_KEY_META_FIELD_POS));
HoodieInternalRow internalRow = new HoodieInternalRow(instantTime, seqId, recordKey, partitionPath, path.getName(),
record);
// NOTE: PLEASE READ THIS CAREFULLY BEFORE MODIFYING
// This code lays in the hot-path, and substantial caution should be
// exercised making changes to it to minimize amount of excessive:
// - Conversions b/w Spark internal (low-level) types and JVM native ones (like
// [[UTF8String]] and [[String]])
// - Repeated computations (for ex, converting file-path to [[UTF8String]] over and
// over again)
UTF8String recordKey = row.getUTF8String(RECORD_KEY_META_FIELD_ORD);
InternalRow updatedRow;
// In cases when no meta-fields need to be added we simply relay provided row to
// the writer as is
if (!populateMetaFields) {
updatedRow = row;
} else {
UTF8String partitionPath = row.getUTF8String(PARTITION_PATH_META_FIELD_ORD);
// This is the only meta-field that is generated dynamically, hence conversion b/w
// [[String]] and [[UTF8String]] is unavoidable
UTF8String seqId = UTF8String.fromString(seqIdGenerator.apply(GLOBAL_SEQ_NO.getAndIncrement()));
updatedRow = new HoodieInternalRow(commitTime, seqId, recordKey,
partitionPath, fileName, row, true);
}
try {
fileWriter.writeRow(recordKey, internalRow);
writeStatus.markSuccess(recordKey);
fileWriter.writeRow(recordKey, updatedRow);
// NOTE: To avoid conversion on the hot-path we only convert [[UTF8String]] into [[String]]
// in cases when successful records' writes are being tracked
writeStatus.markSuccess(writeStatus.isTrackingSuccessfulWrites() ? recordKey.toString() : null);
} catch (Throwable t) {
writeStatus.markFailure(recordKey, t);
writeStatus.markFailure(recordKey.toString(), t);
}
} catch (Throwable ge) {
writeStatus.setGlobalError(ge);
@@ -168,7 +214,7 @@ public class HoodieRowCreateHandle implements Serializable {
return path.getName();
}
private Path makeNewPath(String partitionPath) {
private static Path makeNewPath(FileSystem fs, String partitionPath, String fileName, HoodieWriteConfig writeConfig) {
Path path = FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath);
try {
if (!fs.exists(path)) {
@@ -177,9 +223,7 @@ public class HoodieRowCreateHandle implements Serializable {
} catch (IOException e) {
throw new HoodieIOException("Failed to make dir " + path, e);
}
HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig();
return new Path(path.toString(), FSUtils.makeBaseFileName(instantTime, getWriteToken(), fileId,
tableConfig.getBaseFileFormat().getFileExtension()));
return new CachingPath(path.toString(), fileName);
}
/**
@@ -187,12 +231,17 @@ public class HoodieRowCreateHandle implements Serializable {
*
* @param partitionPath Partition path
*/
private void createMarkerFile(String partitionPath, String dataFileName) {
private static void createMarkerFile(String partitionPath,
String dataFileName,
String instantTime,
HoodieTable<?, ?, ?, ?> table,
HoodieWriteConfig writeConfig) {
WriteMarkersFactory.get(writeConfig.getMarkersType(), table, instantTime)
.create(partitionPath, dataFileName, IOType.CREATE);
}
private String getWriteToken() {
// TODO extract to utils
private static String getWriteToken(int taskPartitionId, long taskId, long taskEpochId) {
return taskPartitionId + "-" + taskId + "-" + taskEpochId;
}

View File

@@ -1,64 +0,0 @@
/*
* 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.io.storage.row;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.table.HoodieTable;
import org.apache.hadoop.fs.Path;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.types.StructType;
import java.io.IOException;
/**
* RowCreateHandle to be used when meta fields are disabled.
*/
public class HoodieRowCreateHandleWithoutMetaFields extends HoodieRowCreateHandle {
public HoodieRowCreateHandleWithoutMetaFields(HoodieTable table, HoodieWriteConfig writeConfig, String partitionPath, String fileId, String instantTime,
int taskPartitionId, long taskId, long taskEpochId, StructType structType) {
super(table, writeConfig, partitionPath, fileId, instantTime, taskPartitionId, taskId, taskEpochId, structType);
}
/**
* Write the incoming InternalRow as is.
*
* @param record instance of {@link InternalRow} that needs to be written to the fileWriter.
* @throws IOException
*/
@Override
public void write(InternalRow record) throws IOException {
try {
fileWriter.writeRow(record);
writeStatus.markSuccess();
} catch (Throwable ge) {
writeStatus.setGlobalError(ge);
throw new HoodieException("Exception thrown while writing spark InternalRows to file ", ge);
}
}
protected HoodieInternalRowFileWriter createNewFileWriter(
Path path, HoodieTable hoodieTable, HoodieWriteConfig config, StructType schema)
throws IOException {
return HoodieInternalRowFileWriterFactory.getInternalRowFileWriterWithoutMetaFields(
path, hoodieTable, config, schema);
}
}

View File

@@ -25,6 +25,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.unsafe.types.UTF8String;
import java.util.HashMap;
@@ -38,10 +39,11 @@ import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_
*/
public class HoodieRowParquetWriteSupport extends ParquetWriteSupport {
private Configuration hadoopConf;
private BloomFilter bloomFilter;
private String minRecordKey;
private String maxRecordKey;
private final Configuration hadoopConf;
private final BloomFilter bloomFilter;
private UTF8String minRecordKey;
private UTF8String maxRecordKey;
public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, BloomFilter bloomFilter, HoodieWriteConfig writeConfig) {
super();
@@ -63,8 +65,8 @@ public class HoodieRowParquetWriteSupport extends ParquetWriteSupport {
if (bloomFilter != null) {
extraMetaData.put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilter.serializeToString());
if (minRecordKey != null && maxRecordKey != null) {
extraMetaData.put(HOODIE_MIN_RECORD_KEY_FOOTER, minRecordKey);
extraMetaData.put(HOODIE_MAX_RECORD_KEY_FOOTER, maxRecordKey);
extraMetaData.put(HOODIE_MIN_RECORD_KEY_FOOTER, minRecordKey.toString());
extraMetaData.put(HOODIE_MAX_RECORD_KEY_FOOTER, maxRecordKey.toString());
}
if (bloomFilter.getBloomFilterTypeCode().name().contains(HoodieDynamicBoundedBloomFilter.TYPE_CODE_PREFIX)) {
extraMetaData.put(HOODIE_BLOOM_FILTER_TYPE_CODE, bloomFilter.getBloomFilterTypeCode().name());
@@ -73,18 +75,18 @@ public class HoodieRowParquetWriteSupport extends ParquetWriteSupport {
return new WriteSupport.FinalizedWriteContext(extraMetaData);
}
public void add(String recordKey) {
this.bloomFilter.add(recordKey);
if (minRecordKey != null) {
minRecordKey = minRecordKey.compareTo(recordKey) <= 0 ? minRecordKey : recordKey;
} else {
minRecordKey = recordKey;
public void add(UTF8String recordKey) {
this.bloomFilter.add(recordKey.getBytes());
if (minRecordKey == null || minRecordKey.compareTo(recordKey) < 0) {
// NOTE: [[clone]] is performed here (rather than [[copy]]) to only copy underlying buffer in
// cases when [[UTF8String]] is pointing into a buffer storing the whole containing record,
// and simply do a pass over when it holds a (immutable) buffer holding just the string
minRecordKey = recordKey.clone();
}
if (maxRecordKey != null) {
maxRecordKey = maxRecordKey.compareTo(recordKey) >= 0 ? maxRecordKey : recordKey;
} else {
maxRecordKey = recordKey;
if (maxRecordKey == null || maxRecordKey.compareTo(recordKey) > 0) {
maxRecordKey = recordKey.clone();
}
}
}

View File

@@ -18,26 +18,24 @@
package org.apache.hudi.keygen;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.ApiMaturityLevel;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.PublicAPIMethod;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.exception.HoodieException;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.StructType;
import scala.Function1;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import scala.Function1;
/**
* Base class for the built-in key generators. Contains methods structured for
* code reuse amongst them.
@@ -66,18 +64,32 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp
@Override
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
public String getRecordKey(Row row) {
// TODO avoid conversion to avro
// since converterFn is transient this will be repeatedly initialized over and over again
if (null == converterFn) {
converterFn = AvroConversionUtils.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE);
}
return getKey(converterFn.apply(row)).getRecordKey();
}
@Override
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
public String getRecordKey(InternalRow internalRow, StructType schema) {
try {
// TODO fix
buildFieldSchemaInfoIfNeeded(schema);
return RowKeyGeneratorHelper.getRecordKeyFromInternalRow(internalRow, getRecordKeyFields(), recordKeySchemaInfo, false);
} catch (Exception e) {
throw new HoodieException("Conversion of InternalRow to Row failed with exception", e);
}
}
/**
* Fetch partition path from {@link Row}.
*
* @param row instance of {@link Row} from which partition path is requested
* @return the partition path of interest from {@link Row}.
*/
@Override
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
public String getPartitionPath(Row row) {
@@ -102,12 +114,13 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp
return RowKeyGeneratorHelper.getPartitionPathFromInternalRow(internalRow, getPartitionPathFields(),
hiveStylePartitioning, partitionPathSchemaInfo);
} catch (Exception e) {
throw new HoodieIOException("Conversion of InternalRow to Row failed with exception " + e);
throw new HoodieException("Conversion of InternalRow to Row failed with exception", e);
}
}
void buildFieldSchemaInfoIfNeeded(StructType structType) {
if (this.structType == null) {
this.structType = structType;
getRecordKeyFields()
.stream().filter(f -> !f.isEmpty())
.forEach(f -> recordKeySchemaInfo.put(f, RowKeyGeneratorHelper.getFieldSchemaInfo(structType, f, true)));
@@ -115,7 +128,6 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp
getPartitionPathFields().stream().filter(f -> !f.isEmpty())
.forEach(f -> partitionPathSchemaInfo.put(f, RowKeyGeneratorHelper.getFieldSchemaInfo(structType, f, false)));
}
this.structType = structType;
}
}

View File

@@ -64,6 +64,12 @@ public class ComplexKeyGenerator extends BuiltinKeyGenerator {
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, true);
}
@Override
public String getRecordKey(InternalRow internalRow, StructType schema) {
buildFieldSchemaInfoIfNeeded(schema);
return RowKeyGeneratorHelper.getRecordKeyFromInternalRow(internalRow, getRecordKeyFields(), recordKeySchemaInfo, true);
}
@Override
public String getPartitionPath(Row row) {
buildFieldSchemaInfoIfNeeded(row.schema());

View File

@@ -64,6 +64,12 @@ public class GlobalDeleteKeyGenerator extends BuiltinKeyGenerator {
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, true);
}
@Override
public String getRecordKey(InternalRow internalRow, StructType schema) {
buildFieldSchemaInfoIfNeeded(schema);
return RowKeyGeneratorHelper.getRecordKeyFromInternalRow(internalRow, getRecordKeyFields(), recordKeySchemaInfo, true);
}
@Override
public String getPartitionPath(Row row) {
return globalAvroDeleteKeyGenerator.getEmptyPartition();

View File

@@ -0,0 +1,59 @@
/*
* 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.keygen;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.DateType;
import org.apache.spark.sql.types.TimestampType;
import java.sql.Timestamp;
import java.time.Instant;
import java.time.LocalDate;
public class RowKeyGenUtils {
/**
* Converts provided (raw) value extracted from the {@link InternalRow} object into a deserialized,
* JVM native format (for ex, converting {@code Long} into {@link Instant},
* {@code Integer} to {@link LocalDate}, etc)
*
* This method allows to avoid costly full-row deserialization sequence. Note, that this method
* should be maintained in sync w/
*
* <ol>
* <li>{@code RowEncoder#deserializerFor}, as well as</li>
* <li>{@code HoodieAvroUtils#convertValueForAvroLogicalTypes}</li>
* </ol>
*
* @param dataType target data-type of the given value
* @param value target value to be converted
*/
public static Object convertToLogicalDataType(DataType dataType, Object value) {
if (dataType instanceof TimestampType) {
// Provided value have to be [[Long]] in this case, representing micros since epoch
return new Timestamp((Long) value / 1000);
} else if (dataType instanceof DateType) {
// Provided value have to be [[Int]] in this case
return LocalDate.ofEpochDay((Integer) value);
}
return value;
}
}

View File

@@ -39,18 +39,56 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.spark.sql.types.StructType$;
import scala.Option;
import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH_SEPARATOR;
import static org.apache.hudi.keygen.KeyGenUtils.EMPTY_RECORDKEY_PLACEHOLDER;
import static org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH;
import static org.apache.hudi.keygen.KeyGenUtils.NULL_RECORDKEY_PLACEHOLDER;
import static org.apache.hudi.keygen.RowKeyGenUtils.convertToLogicalDataType;
/**
* Helper class to fetch fields from Row.
*
* TODO cleanup
*/
@Deprecated
public class RowKeyGeneratorHelper {
public static String getRecordKeyFromInternalRow(InternalRow internalRow, List<String> recordKeyFields,
Map<String, Pair<List<Integer>, DataType>> recordKeyPositions, boolean prefixFieldName) {
AtomicBoolean keyIsNullOrEmpty = new AtomicBoolean(true);
String toReturn = recordKeyFields.stream().map(field -> {
String val = null;
List<Integer> fieldPositions = recordKeyPositions.get(field).getKey();
if (fieldPositions.size() == 1) { // simple field
Integer fieldPos = fieldPositions.get(0);
if (internalRow.isNullAt(fieldPos)) {
val = NULL_RECORDKEY_PLACEHOLDER;
} else {
DataType dataType = recordKeyPositions.get(field).getValue();
val = convertToLogicalDataType(dataType, internalRow.get(fieldPos, dataType)).toString();
if (val.isEmpty()) {
val = EMPTY_RECORDKEY_PLACEHOLDER;
} else {
keyIsNullOrEmpty.set(false);
}
}
} else { // nested fields
val = getNestedFieldVal(internalRow, recordKeyPositions.get(field)).toString();
if (!val.contains(NULL_RECORDKEY_PLACEHOLDER) && !val.contains(EMPTY_RECORDKEY_PLACEHOLDER)) {
keyIsNullOrEmpty.set(false);
}
}
return prefixFieldName ? (field + ":" + val) : val;
}).collect(Collectors.joining(","));
if (keyIsNullOrEmpty.get()) {
throw new HoodieKeyException("recordKey value: \"" + toReturn + "\" for fields: \"" + Arrays.toString(recordKeyFields.toArray()) + "\" cannot be null or empty.");
}
return toReturn;
}
/**
* Generates record key for the corresponding {@link Row}.
*
@@ -146,7 +184,7 @@ public class RowKeyGeneratorHelper {
if (fieldPos == -1 || internalRow.isNullAt(fieldPos)) {
val = HUDI_DEFAULT_PARTITION_PATH;
} else {
Object value = internalRow.get(fieldPos, dataType);
Object value = convertToLogicalDataType(dataType, internalRow.get(fieldPos, dataType));
if (value == null || value.toString().isEmpty()) {
val = HUDI_DEFAULT_PARTITION_PATH;
} else {
@@ -231,6 +269,35 @@ public class RowKeyGeneratorHelper {
return toReturn;
}
public static Object getNestedFieldVal(InternalRow internalRow, Pair<List<Integer>, DataType> positionsAndType) {
if (positionsAndType.getKey().size() == 1 && positionsAndType.getKey().get(0) == -1) {
return HUDI_DEFAULT_PARTITION_PATH;
}
int index = 0;
int totalCount = positionsAndType.getKey().size();
InternalRow valueToProcess = internalRow;
Object toReturn = null;
while (index < totalCount) {
if (valueToProcess.isNullAt(positionsAndType.getKey().get(index))) {
toReturn = NULL_RECORDKEY_PLACEHOLDER;
break;
}
if (index < totalCount - 1) {
valueToProcess = (InternalRow) valueToProcess.get(positionsAndType.getKey().get(index), StructType$.MODULE$.defaultConcreteType());
} else { // last index
if (valueToProcess.get(positionsAndType.getKey().get(index), positionsAndType.getValue()).toString().isEmpty()) {
toReturn = EMPTY_RECORDKEY_PLACEHOLDER;
break;
}
toReturn = valueToProcess.get(positionsAndType.getKey().get(index), positionsAndType.getValue());
}
index++;
}
return toReturn;
}
/**
* Generate the tree style positions for the field requested for as per the defined struct type.
*

View File

@@ -29,6 +29,8 @@ public interface SparkKeyGeneratorInterface extends KeyGeneratorInterface {
String getRecordKey(Row row);
String getRecordKey(InternalRow row, StructType schema);
String getPartitionPath(Row row);
String getPartitionPath(InternalRow internalRow, StructType structType);

View File

@@ -19,112 +19,17 @@
package org.apache.hudi.util;
import org.apache.spark.sql.types.ArrayType;
import org.apache.spark.sql.types.ByteType$;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.Decimal;
import org.apache.spark.sql.types.DecimalType;
import org.apache.spark.sql.types.DoubleType$;
import org.apache.spark.sql.types.FloatType$;
import org.apache.spark.sql.types.IntegerType$;
import org.apache.spark.sql.types.LongType$;
import org.apache.spark.sql.types.MapType;
import org.apache.spark.sql.types.ShortType$;
import org.apache.spark.sql.types.StringType$;
import org.apache.spark.sql.types.StructField;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.sql.types.VarcharType$;
import javax.annotation.Nonnull;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
public class DataTypeUtils {
private static Map<Class<?>, Set<Class<?>>> sparkPrimitiveTypesCompatibilityMap =
new HashMap<Class<?>, Set<Class<?>>>() {{
// Integral types
put(ShortType$.class,
newHashSet(ByteType$.class, ShortType$.class));
put(IntegerType$.class,
newHashSet(ByteType$.class, ShortType$.class, IntegerType$.class));
put(LongType$.class,
newHashSet(ByteType$.class, ShortType$.class, IntegerType$.class, LongType$.class));
// Float types
put(DoubleType$.class,
newHashSet(FloatType$.class, DoubleType$.class));
// String types
put(StringType$.class,
newHashSet(VarcharType$.class, StringType$.class));
}
};
/**
* Validates whether one {@link StructType} is compatible w/ the other one.
* Compatibility rules are defined like following: types A and B are considered
* compatible iff
*
* <ol>
* <li>A and B are identical</li>
* <li>All values comprising A domain are contained w/in B domain (for ex, {@code ShortType}
* in this sense is compatible w/ {@code IntegerType})</li>
* </ol>
*
* @param left operand
* @param right operand
* @return true if {@code left} instance of {@link StructType} is compatible w/ the {@code right}
*/
public static boolean areCompatible(@Nonnull DataType left, @Nonnull DataType right) {
// First, check if types are equal
if (Objects.equals(left, right)) {
return true;
}
// If not, check whether both are instances of {@code StructType} that
// should be matched structurally
if (left instanceof StructType && right instanceof StructType) {
return areCompatible((StructType) left, (StructType) right);
}
// If not, simply check if those data-types constitute compatibility
// relationship outlined above; otherwise return false
return sparkPrimitiveTypesCompatibilityMap.getOrDefault(left.getClass(), Collections.emptySet())
.contains(right.getClass());
}
private static boolean areCompatible(@Nonnull StructType left, @Nonnull StructType right) {
StructField[] oneSchemaFields = left.fields();
StructField[] anotherSchemaFields = right.fields();
if (oneSchemaFields.length != anotherSchemaFields.length) {
return false;
}
for (int i = 0; i < oneSchemaFields.length; ++i) {
StructField oneField = oneSchemaFields[i];
StructField anotherField = anotherSchemaFields[i];
// NOTE: Metadata is deliberately omitted from comparison
if (!Objects.equals(oneField.name(), anotherField.name())
|| !areCompatible(oneField.dataType(), anotherField.dataType())
|| oneField.nullable() != anotherField.nullable()) {
return false;
}
}
return true;
}
private static <T> HashSet<T> newHashSet(T... ts) {
return new HashSet<>(Arrays.asList(ts));
}
/**
* Checks whether provided {@link DataType} contains {@link DecimalType} whose scale is less than
* {@link Decimal#MAX_LONG_DIGITS()}

View File

@@ -0,0 +1,120 @@
/*
* 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.spark.sql
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.types.{StructField, StructType}
import scala.collection.mutable.ArrayBuffer
object HoodieUnsafeRowUtils {
/**
* Fetches (nested) value w/in provided [[Row]] uniquely identified by the provided nested-field path
* previously composed by [[composeNestedFieldPath]]
*/
def getNestedRowValue(row: Row, nestedFieldPath: Array[(Int, StructField)]): Any = {
var curRow = row
for (idx <- nestedFieldPath.indices) {
val (ord, f) = nestedFieldPath(idx)
if (curRow.isNullAt(ord)) {
// scalastyle:off return
if (f.nullable) return null
else throw new IllegalArgumentException(s"Found null value for the field that is declared as non-nullable: $f")
// scalastyle:on return
} else if (idx == nestedFieldPath.length - 1) {
// scalastyle:off return
return curRow.get(ord)
// scalastyle:on return
} else {
curRow = f.dataType match {
case _: StructType =>
curRow.getStruct(ord)
case dt@_ =>
throw new IllegalArgumentException(s"Invalid nested-field path: expected StructType, but was $dt")
}
}
}
}
/**
* Fetches (nested) value w/in provided [[InternalRow]] uniquely identified by the provided nested-field path
* previously composed by [[composeNestedFieldPath]]
*/
def getNestedInternalRowValue(row: InternalRow, nestedFieldPath: Array[(Int, StructField)]): Any = {
if (nestedFieldPath.length == 0) {
throw new IllegalArgumentException("Nested field-path could not be empty")
}
var curRow = row
var idx = 0
while (idx < nestedFieldPath.length) {
val (ord, f) = nestedFieldPath(idx)
if (curRow.isNullAt(ord)) {
// scalastyle:off return
if (f.nullable) return null
else throw new IllegalArgumentException(s"Found null value for the field that is declared as non-nullable: $f")
// scalastyle:on return
} else if (idx == nestedFieldPath.length - 1) {
// scalastyle:off return
return curRow.get(ord, f.dataType)
// scalastyle:on return
} else {
curRow = f.dataType match {
case st: StructType =>
curRow.getStruct(ord, st.fields.length)
case dt@_ =>
throw new IllegalArgumentException(s"Invalid nested-field path: expected StructType, but was $dt")
}
}
idx += 1
}
}
/**
* For the provided [[nestedFieldRef]] (of the form "a.b.c") and [[schema]], produces nested-field path comprised
* of (ordinal, data-type) tuples of the respective fields w/in the provided schema.
*
* This method produces nested-field path, that is subsequently used by [[getNestedInternalRowValue]], [[getNestedRowValue]]
*/
def composeNestedFieldPath(schema: StructType, nestedFieldRef: String): Array[(Int, StructField)] = {
val fieldRefParts = nestedFieldRef.split('.')
val ordSeq = ArrayBuffer[(Int, StructField)]()
var curSchema = schema
var idx = 0
while (idx < fieldRefParts.length) {
val fieldRefPart = fieldRefParts(idx)
val ord = curSchema.fieldIndex(fieldRefPart)
val field = curSchema(ord)
// Append current field's (ordinal, data-type)
ordSeq.append((ord, field))
// Update current schema, unless terminal field-ref part
if (idx < fieldRefParts.length - 1) {
curSchema = field.dataType match {
case st: StructType => st
case dt@_ =>
throw new IllegalArgumentException(s"Invalid nested field reference ${fieldRefParts.drop(idx).mkString(".")} into $dt")
}
}
idx += 1
}
ordSeq.toArray
}
}

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.client.model;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.unsafe.types.UTF8String;
import org.junit.jupiter.api.Test;
import java.util.ArrayList;
@@ -64,7 +65,13 @@ public class TestHoodieInternalRow {
Object[] values = getRandomValue(true);
InternalRow row = new GenericInternalRow(values);
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow(UTF8String.fromString("commitTime"),
UTF8String.fromString("commitSeqNo"),
UTF8String.fromString("recordKey"),
UTF8String.fromString("partitionPath"),
UTF8String.fromString("fileName"),
row,
true);
assertValues(hoodieInternalRow, "commitTime", "commitSeqNo", "recordKey", "partitionPath",
"fileName", values, nullIndices);
@@ -74,7 +81,13 @@ public class TestHoodieInternalRow {
public void testUpdate() {
Object[] values = getRandomValue(true);
InternalRow row = new GenericInternalRow(values);
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow(UTF8String.fromString("commitTime"),
UTF8String.fromString("commitSeqNo"),
UTF8String.fromString("recordKey"),
UTF8String.fromString("partitionPath"),
UTF8String.fromString("fileName"),
row,
true);
hoodieInternalRow.update(0, "commitTime_updated");
hoodieInternalRow.update(1, "commitSeqNo_updated");
@@ -106,7 +119,13 @@ public class TestHoodieInternalRow {
Object[] values = getRandomValue(true);
InternalRow row = new GenericInternalRow(values);
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow(UTF8String.fromString("commitTime"),
UTF8String.fromString("commitSeqNo"),
UTF8String.fromString("recordKey"),
UTF8String.fromString("partitionPath"),
UTF8String.fromString("fileName"),
row,
true);
hoodieInternalRow.setNullAt(i);
nullIndices.clear();
@@ -129,7 +148,13 @@ public class TestHoodieInternalRow {
Object[] values = getRandomValue(true);
InternalRow row = new GenericInternalRow(values);
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow(UTF8String.fromString("commitTime"),
UTF8String.fromString("commitSeqNo"),
UTF8String.fromString("recordKey"),
UTF8String.fromString("partitionPath"),
UTF8String.fromString("fileName"),
row,
true);
nullIndices.clear();
@@ -173,7 +198,7 @@ public class TestHoodieInternalRow {
}
private void assertValues(HoodieInternalRow hoodieInternalRow, String commitTime, String commitSeqNo, String recordKey, String partitionPath, String filename, Object[] values,
List<Integer> nullIndexes) {
List<Integer> nullIndexes) {
for (Integer index : nullIndexes) {
assertTrue(hoodieInternalRow.isNullAt(index));
}

View File

@@ -23,6 +23,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieInsertException;
import org.apache.hudi.exception.TableNotFoundException;
@@ -75,8 +76,9 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
cleanupResources();
}
@Test
public void testRowCreateHandle() throws Exception {
@ParameterizedTest
@ValueSource(booleans = { true, false })
public void testRowCreateHandle(boolean populateMetaFields) throws Exception {
// init config and table
HoodieWriteConfig cfg =
SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort).build();
@@ -93,7 +95,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
String fileId = UUID.randomUUID().toString();
String instantTime = "000";
HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime,
RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE, populateMetaFields);
int size = 10 + RANDOM.nextInt(1000);
// Generate inputs
Dataset<Row> inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, size, partitionPath, false);
@@ -109,7 +112,7 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
fileAbsPaths.add(basePath + "/" + writeStatus.getStat().getPath());
fileNames.add(handle.getFileName());
// verify output
assertOutput(writeStatus, size, fileId, partitionPath, instantTime, totalInputRows, fileNames, fileAbsPaths);
assertOutput(writeStatus, size, fileId, partitionPath, instantTime, totalInputRows, fileNames, fileAbsPaths, populateMetaFields);
}
}
@@ -130,7 +133,7 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
String instantTime = "000";
HoodieRowCreateHandle handle =
new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE, true);
int size = 10 + RANDOM.nextInt(1000);
int totalFailures = 5;
// Generate first batch of valid rows
@@ -169,7 +172,7 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
// verify rows
Dataset<Row> result = sqlContext.read().parquet(basePath + "/" + partitionPath);
// passing only first batch of inputRows since after first batch global error would have been thrown
assertRows(inputRows, result, instantTime, fileNames);
assertRows(inputRows, result, instantTime, fileNames, true);
}
@ParameterizedTest
@@ -183,7 +186,7 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
try {
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
new HoodieRowCreateHandle(table, cfg, " def", UUID.randomUUID().toString(), "001", RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
new HoodieRowCreateHandle(table, cfg, " def", UUID.randomUUID().toString(), "001", RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE, true);
fail("Should have thrown exception");
} catch (HoodieInsertException ioe) {
// expected without metadata table
@@ -209,8 +212,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
return handle.close();
}
private void assertOutput(HoodieInternalWriteStatus writeStatus, int size, String fileId, String partitionPath, String instantTime, Dataset<Row> inputRows, List<String> filenames,
List<String> fileAbsPaths) {
private void assertOutput(HoodieInternalWriteStatus writeStatus, int size, String fileId, String partitionPath,
String instantTime, Dataset<Row> inputRows, List<String> filenames, List<String> fileAbsPaths, boolean populateMetaFields) {
assertEquals(writeStatus.getPartitionPath(), partitionPath);
assertEquals(writeStatus.getTotalRecords(), size);
assertEquals(writeStatus.getFailedRowsSize(), 0);
@@ -229,15 +232,25 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
// verify rows
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0]));
assertRows(inputRows, result, instantTime, filenames);
assertRows(inputRows, result, instantTime, filenames, populateMetaFields);
}
private void assertRows(Dataset<Row> expectedRows, Dataset<Row> actualRows, String instantTime, List<String> filenames) {
private void assertRows(Dataset<Row> expectedRows, Dataset<Row> actualRows, String instantTime, List<String> filenames, boolean populateMetaFields) {
// verify 3 meta fields that are filled in within create handle
actualRows.collectAsList().forEach(entry -> {
assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime);
assertTrue(filenames.contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)).toString()));
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)));
String commitTime = entry.getString(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD));
String fileName = entry.getString(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD));
String seqId = entry.getString(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD));
if (populateMetaFields) {
assertEquals(instantTime, commitTime);
assertFalse(StringUtils.isNullOrEmpty(seqId));
assertTrue(filenames.contains(fileName));
} else {
assertEquals("", commitTime);
assertEquals("", seqId);
assertEquals("", fileName);
}
});
// after trimming 2 of the meta fields, rest of the fields should match

View File

@@ -0,0 +1,166 @@
/*
* 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.spark.sql
import org.apache.spark.sql.HoodieUnsafeRowUtils.{composeNestedFieldPath, getNestedInternalRowValue, getNestedRowValue}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.types._
import org.junit.jupiter.api.Assertions.{assertEquals, fail}
import org.junit.jupiter.api.Test
class TestHoodieUnsafeRowUtils {
@Test
def testComposeNestedFieldPath(): Unit = {
val schema = StructType(Seq(
StructField("foo", StringType),
StructField(
name = "bar",
dataType = StructType(Seq(
StructField("baz", DateType),
StructField("bor", LongType)
))
)
))
assertEquals(
Seq((1, schema(1)), (0, schema(1).dataType.asInstanceOf[StructType](0))),
composeNestedFieldPath(schema, "bar.baz").toSeq)
assertThrows(classOf[IllegalArgumentException]) { () =>
composeNestedFieldPath(schema, "foo.baz")
}
}
@Test
def testGetNestedInternalRowValue(): Unit = {
val schema = StructType(Seq(
StructField("foo", StringType, nullable = false),
StructField(
name = "bar",
dataType = StructType(Seq(
StructField("baz", DateType),
StructField("bor", LongType)
))
)
))
val row = InternalRow("str", InternalRow(123, 456L))
assertEquals(
123,
getNestedInternalRowValue(row, composeNestedFieldPath(schema, "bar.baz"))
)
assertEquals(
456L,
getNestedInternalRowValue(row, composeNestedFieldPath(schema, "bar.bor"))
)
assertEquals(
"str",
getNestedInternalRowValue(row, composeNestedFieldPath(schema, "foo"))
)
assertEquals(
row.getStruct(1, 2),
getNestedInternalRowValue(row, composeNestedFieldPath(schema, "bar"))
)
val rowProperNullable = InternalRow("str", null)
assertEquals(
null,
getNestedInternalRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar.baz"))
)
assertEquals(
null,
getNestedInternalRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar"))
)
val rowInvalidNullable = InternalRow(null, InternalRow(123, 456L))
assertThrows(classOf[IllegalArgumentException]) { () =>
getNestedInternalRowValue(rowInvalidNullable, composeNestedFieldPath(schema, "foo"))
}
}
@Test
def testGetNestedRowValue(): Unit = {
val schema = StructType(Seq(
StructField("foo", StringType, nullable = false),
StructField(
name = "bar",
dataType = StructType(Seq(
StructField("baz", DateType),
StructField("bor", LongType)
))
)
))
val row = Row("str", Row(123, 456L))
assertEquals(
123,
getNestedRowValue(row, composeNestedFieldPath(schema, "bar.baz"))
)
assertEquals(
456L,
getNestedRowValue(row, composeNestedFieldPath(schema, "bar.bor"))
)
assertEquals(
"str",
getNestedRowValue(row, composeNestedFieldPath(schema, "foo"))
)
assertEquals(
row.getStruct(1),
getNestedRowValue(row, composeNestedFieldPath(schema, "bar"))
)
val rowProperNullable = Row("str", null)
assertEquals(
null,
getNestedRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar.baz"))
)
assertEquals(
null,
getNestedRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar"))
)
val rowInvalidNullable = Row(null, Row(123, 456L))
assertThrows(classOf[IllegalArgumentException]) { () =>
getNestedRowValue(rowInvalidNullable, composeNestedFieldPath(schema, "foo"))
}
}
private def assertThrows[T <: Throwable](expectedExceptionClass: Class[T])(f: () => Unit): T = {
try {
f.apply()
} catch {
case t: Throwable if expectedExceptionClass.isAssignableFrom(t.getClass) =>
// scalastyle:off return
return t.asInstanceOf[T]
// scalastyle:on return
case ot @ _ =>
fail(s"Expected exception of class $expectedExceptionClass, but ${ot.getClass} has been thrown")
}
fail(s"Expected exception of class $expectedExceptionClass, but nothing has been thrown")
}
}