diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java index dca0c25775133..9ff5c522e4527 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java @@ -44,7 +44,7 @@ public ComplexAvroKeyGenerator(TypedProperties props) { @Override public String getRecordKey(GenericRecord record) { - return KeyGenUtils.getRecordKey(record, getRecordKeyFields(), isConsistentLogicalTimestampEnabled()); + return KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames(), isConsistentLogicalTimestampEnabled()); } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/CustomAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/CustomAvroKeyGenerator.java index 370b57b5308c4..77377de7ab8c7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/CustomAvroKeyGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/CustomAvroKeyGenerator.java @@ -103,13 +103,13 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(GenericRecord record) { validateRecordKeyFields(); - return getRecordKeyFields().size() == 1 + return getRecordKeyFieldNames().size() == 1 ? new SimpleAvroKeyGenerator(config).getRecordKey(record) : new ComplexAvroKeyGenerator(config).getRecordKey(record); } private void validateRecordKeyFields() { - if (getRecordKeyFields() == null || getRecordKeyFields().isEmpty()) { + if (getRecordKeyFieldNames() == null || getRecordKeyFieldNames().isEmpty()) { throw new HoodieKeyException("Unable to find field names for record key in cfg"); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/GlobalAvroDeleteKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/GlobalAvroDeleteKeyGenerator.java index 10a57602386fc..dc0bc3cef2f00 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/GlobalAvroDeleteKeyGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/GlobalAvroDeleteKeyGenerator.java @@ -40,7 +40,7 @@ public GlobalAvroDeleteKeyGenerator(TypedProperties config) { @Override public String getRecordKey(GenericRecord record) { - return KeyGenUtils.getRecordKey(record, getRecordKeyFields(), isConsistentLogicalTimestampEnabled()); + return KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames(), isConsistentLogicalTimestampEnabled()); } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java index f1e41296f1dd3..362ef208d4e84 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java @@ -42,6 +42,7 @@ public class KeyGenUtils { protected static final String HUDI_DEFAULT_PARTITION_PATH = PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH; public static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/"; + public static final String DEFAULT_RECORD_KEY_PARTS_SEPARATOR = ","; /** * Fetches record key from the GenericRecord. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/NonpartitionedAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/NonpartitionedAvroKeyGenerator.java index db7596993d578..5b5cedcbf8855 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/NonpartitionedAvroKeyGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/NonpartitionedAvroKeyGenerator.java @@ -56,10 +56,10 @@ public String getRecordKey(GenericRecord record) { // for backward compatibility, we need to use the right format according to the number of record key fields // 1. if there is only one record key field, the format of record key is just "" // 2. if there are multiple record key fields, the format is ":,:,..." - if (getRecordKeyFields().size() == 1) { - return KeyGenUtils.getRecordKey(record, getRecordKeyFields().get(0), isConsistentLogicalTimestampEnabled()); + if (getRecordKeyFieldNames().size() == 1) { + return KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames().get(0), isConsistentLogicalTimestampEnabled()); } - return KeyGenUtils.getRecordKey(record, getRecordKeyFields(), isConsistentLogicalTimestampEnabled()); + return KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames(), isConsistentLogicalTimestampEnabled()); } public String getEmptyPartition() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java index 943091225a3e1..c7398e94ecea0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java @@ -47,7 +47,7 @@ public SimpleAvroKeyGenerator(TypedProperties props) { @Override public String getRecordKey(GenericRecord record) { - return KeyGenUtils.getRecordKey(record, getRecordKeyFields().get(0), isConsistentLogicalTimestampEnabled()); + return KeyGenUtils.getRecordKey(record, getRecordKeyFieldNames().get(0), isConsistentLogicalTimestampEnabled()); } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java index c543fd26041a1..60ccc694f947d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java @@ -111,7 +111,7 @@ public String getPartitionPath(GenericRecord record) { try { return getPartitionPath(partitionVal); } catch (Exception e) { - throw new HoodieKeyGeneratorException("Unable to parse input partition field :" + partitionVal, e); + throw new HoodieKeyGeneratorException("Unable to parse input partition field: " + partitionVal, e); } } @@ -181,7 +181,7 @@ public String getPartitionPath(Object partitionVal) { timeMs = convertLongTimeToMillis(((LocalDate) partitionVal).toEpochDay()); } else if (partitionVal instanceof CharSequence) { if (!inputFormatter.isPresent()) { - throw new HoodieException("Missing inputformatter. Ensure " + KeyGeneratorOptions.Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP + " config is set when timestampType is DATE_STRING or MIXED!"); + throw new HoodieException("Missing input formatter. Ensure " + KeyGeneratorOptions.Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP + " config is set when timestampType is DATE_STRING or MIXED!"); } DateTime parsedDateTime = inputFormatter.get().parseDateTime(partitionVal.toString()); if (this.outputDateTimeZone == null) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java index c2f86bd6b8001..4651c0d4b4ce1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.UnsafeRow; import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.sql.types.DataType; @@ -34,7 +35,7 @@ * 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 + * Capable of overlaying meta-fields in both cases: whether original {@link #sourceRow} 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 @@ -50,22 +51,27 @@ public class HoodieInternalRow extends InternalRow { /** * Collection of meta-fields as defined by {@link HoodieRecord#HOODIE_META_COLUMNS} + * + * NOTE: {@code HoodieInternalRow} *always* overlays its own meta-fields even in case + * when source row also contains them, to make sure these fields are mutable and + * can be updated (for ex, {@link UnsafeRow} doesn't support mutations due to + * its memory layout, as it persists field offsets) */ private final UTF8String[] metaFields; - private final InternalRow row; + private final InternalRow sourceRow; /** - * Specifies whether source {@link #row} contains meta-fields + * Specifies whether source {@link #sourceRow} contains meta-fields */ - private final boolean containsMetaFields; + private final boolean sourceContainsMetaFields; public HoodieInternalRow(UTF8String commitTime, UTF8String commitSeqNumber, UTF8String recordKey, UTF8String partitionPath, UTF8String fileName, - InternalRow row, - boolean containsMetaFields) { + InternalRow sourceRow, + boolean sourceContainsMetaFields) { this.metaFields = new UTF8String[] { commitTime, commitSeqNumber, @@ -74,21 +80,21 @@ public HoodieInternalRow(UTF8String commitTime, fileName }; - this.row = row; - this.containsMetaFields = containsMetaFields; + this.sourceRow = sourceRow; + this.sourceContainsMetaFields = sourceContainsMetaFields; } private HoodieInternalRow(UTF8String[] metaFields, - InternalRow row, - boolean containsMetaFields) { + InternalRow sourceRow, + boolean sourceContainsMetaFields) { this.metaFields = metaFields; - this.row = row; - this.containsMetaFields = containsMetaFields; + this.sourceRow = sourceRow; + this.sourceContainsMetaFields = sourceContainsMetaFields; } @Override public int numFields() { - return row.numFields(); + return sourceRow.numFields(); } @Override @@ -96,7 +102,7 @@ public void setNullAt(int ordinal) { if (ordinal < metaFields.length) { metaFields[ordinal] = null; } else { - row.setNullAt(rebaseOrdinal(ordinal)); + sourceRow.setNullAt(rebaseOrdinal(ordinal)); } } @@ -112,7 +118,7 @@ public void update(int ordinal, Object value) { 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(rebaseOrdinal(ordinal), value); + sourceRow.update(rebaseOrdinal(ordinal), value); } } @@ -121,113 +127,113 @@ public boolean isNullAt(int ordinal) { if (ordinal < metaFields.length) { return metaFields[ordinal] == null; } - return row.isNullAt(rebaseOrdinal(ordinal)); + return sourceRow.isNullAt(rebaseOrdinal(ordinal)); } @Override public UTF8String getUTF8String(int ordinal) { - if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) { + if (ordinal < metaFields.length) { return metaFields[ordinal]; } - return row.getUTF8String(rebaseOrdinal(ordinal)); + return sourceRow.getUTF8String(rebaseOrdinal(ordinal)); } @Override public Object get(int ordinal, DataType dataType) { - if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) { + if (ordinal < metaFields.length) { validateMetaFieldDataType(dataType); return metaFields[ordinal]; } - return row.get(rebaseOrdinal(ordinal), dataType); + return sourceRow.get(rebaseOrdinal(ordinal), dataType); } @Override public boolean getBoolean(int ordinal) { ruleOutMetaFieldsAccess(ordinal, Boolean.class); - return row.getBoolean(rebaseOrdinal(ordinal)); + return sourceRow.getBoolean(rebaseOrdinal(ordinal)); } @Override public byte getByte(int ordinal) { ruleOutMetaFieldsAccess(ordinal, Byte.class); - return row.getByte(rebaseOrdinal(ordinal)); + return sourceRow.getByte(rebaseOrdinal(ordinal)); } @Override public short getShort(int ordinal) { ruleOutMetaFieldsAccess(ordinal, Short.class); - return row.getShort(rebaseOrdinal(ordinal)); + return sourceRow.getShort(rebaseOrdinal(ordinal)); } @Override public int getInt(int ordinal) { ruleOutMetaFieldsAccess(ordinal, Integer.class); - return row.getInt(rebaseOrdinal(ordinal)); + return sourceRow.getInt(rebaseOrdinal(ordinal)); } @Override public long getLong(int ordinal) { ruleOutMetaFieldsAccess(ordinal, Long.class); - return row.getLong(rebaseOrdinal(ordinal)); + return sourceRow.getLong(rebaseOrdinal(ordinal)); } @Override public float getFloat(int ordinal) { ruleOutMetaFieldsAccess(ordinal, Float.class); - return row.getFloat(rebaseOrdinal(ordinal)); + return sourceRow.getFloat(rebaseOrdinal(ordinal)); } @Override public double getDouble(int ordinal) { ruleOutMetaFieldsAccess(ordinal, Double.class); - return row.getDouble(rebaseOrdinal(ordinal)); + return sourceRow.getDouble(rebaseOrdinal(ordinal)); } @Override public Decimal getDecimal(int ordinal, int precision, int scale) { ruleOutMetaFieldsAccess(ordinal, Decimal.class); - return row.getDecimal(rebaseOrdinal(ordinal), precision, scale); + return sourceRow.getDecimal(rebaseOrdinal(ordinal), precision, scale); } @Override public byte[] getBinary(int ordinal) { ruleOutMetaFieldsAccess(ordinal, Byte[].class); - return row.getBinary(rebaseOrdinal(ordinal)); + return sourceRow.getBinary(rebaseOrdinal(ordinal)); } @Override public CalendarInterval getInterval(int ordinal) { ruleOutMetaFieldsAccess(ordinal, CalendarInterval.class); - return row.getInterval(rebaseOrdinal(ordinal)); + return sourceRow.getInterval(rebaseOrdinal(ordinal)); } @Override public InternalRow getStruct(int ordinal, int numFields) { ruleOutMetaFieldsAccess(ordinal, InternalRow.class); - return row.getStruct(rebaseOrdinal(ordinal), numFields); + return sourceRow.getStruct(rebaseOrdinal(ordinal), numFields); } @Override public ArrayData getArray(int ordinal) { ruleOutMetaFieldsAccess(ordinal, ArrayData.class); - return row.getArray(rebaseOrdinal(ordinal)); + return sourceRow.getArray(rebaseOrdinal(ordinal)); } @Override public MapData getMap(int ordinal) { ruleOutMetaFieldsAccess(ordinal, MapData.class); - return row.getMap(rebaseOrdinal(ordinal)); + return sourceRow.getMap(rebaseOrdinal(ordinal)); } @Override public InternalRow copy() { - return new HoodieInternalRow(Arrays.copyOf(metaFields, metaFields.length), row.copy(), containsMetaFields); + return new HoodieInternalRow(Arrays.copyOf(metaFields, metaFields.length), sourceRow.copy(), sourceContainsMetaFields); } 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; + return sourceContainsMetaFields ? ordinal : ordinal - metaFields.length; } private void validateMetaFieldDataType(DataType dataType) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java index eb408f81c12d2..e68873f92efdb 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java @@ -18,15 +18,14 @@ package org.apache.hudi.io.storage.row; +import org.apache.hadoop.fs.Path; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.storage.HoodieParquetConfig; import org.apache.hudi.table.HoodieTable; - -import org.apache.hadoop.fs.Path; import org.apache.spark.sql.types.StructType; import java.io.IOException; @@ -42,32 +41,34 @@ public class HoodieInternalRowFileWriterFactory { * Factory method to assist in instantiating an instance of {@link HoodieInternalRowFileWriter}. * @param path path of the RowFileWriter. * @param hoodieTable instance of {@link HoodieTable} in use. - * @param config instance of {@link HoodieWriteConfig} to use. + * @param writeConfig instance of {@link HoodieWriteConfig} to use. * @param schema schema of the dataset in use. * @return the instantiated {@link HoodieInternalRowFileWriter}. * @throws IOException if format is not supported or if any exception during instantiating the RowFileWriter. * */ - public static HoodieInternalRowFileWriter getInternalRowFileWriter( - Path path, HoodieTable hoodieTable, HoodieWriteConfig config, StructType schema) + public static HoodieInternalRowFileWriter getInternalRowFileWriter(Path path, + HoodieTable hoodieTable, + HoodieWriteConfig writeConfig, + StructType schema) throws IOException { final String extension = FSUtils.getFileExtension(path.getName()); if (PARQUET.getFileExtension().equals(extension)) { - return newParquetInternalRowFileWriter(path, config, schema, hoodieTable); + return newParquetInternalRowFileWriter(path, hoodieTable, writeConfig, schema, tryInstantiateBloomFilter(writeConfig)); } throw new UnsupportedOperationException(extension + " format not supported yet."); } - private static HoodieInternalRowFileWriter newParquetInternalRowFileWriter( - Path path, HoodieWriteConfig writeConfig, StructType structType, HoodieTable table) + private static HoodieInternalRowFileWriter newParquetInternalRowFileWriter(Path path, + HoodieTable table, + HoodieWriteConfig writeConfig, + StructType structType, + Option bloomFilterOpt + ) throws IOException { - BloomFilter filter = BloomFilterFactory.createBloomFilter( - writeConfig.getBloomFilterNumEntries(), - writeConfig.getBloomFilterFPP(), - writeConfig.getDynamicBloomFilterMaxNumEntries(), - writeConfig.getBloomFilterType()); HoodieRowParquetWriteSupport writeSupport = - new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, filter, writeConfig); + new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, bloomFilterOpt, writeConfig); + return new HoodieInternalRowParquetWriter( path, new HoodieParquetConfig<>( @@ -82,30 +83,18 @@ private static HoodieInternalRowFileWriter newParquetInternalRowFileWriter( )); } - public static HoodieInternalRowFileWriter getInternalRowFileWriterWithoutMetaFields( - Path path, HoodieTable hoodieTable, HoodieWriteConfig config, StructType schema) - throws IOException { - if (PARQUET.getFileExtension().equals(hoodieTable.getBaseFileExtension())) { - return newParquetInternalRowFileWriterWithoutMetaFields(path, config, schema, hoodieTable); + private static Option tryInstantiateBloomFilter(HoodieWriteConfig writeConfig) { + // NOTE: Currently Bloom Filter is only going to be populated if meta-fields are populated + if (writeConfig.populateMetaFields()) { + BloomFilter bloomFilter = BloomFilterFactory.createBloomFilter( + writeConfig.getBloomFilterNumEntries(), + writeConfig.getBloomFilterFPP(), + writeConfig.getDynamicBloomFilterMaxNumEntries(), + writeConfig.getBloomFilterType()); + + return Option.of(bloomFilter); } - throw new HoodieIOException(hoodieTable.getBaseFileExtension() + " format not supported yet in row writer path"); - } - private static HoodieInternalRowFileWriter newParquetInternalRowFileWriterWithoutMetaFields( - Path path, HoodieWriteConfig writeConfig, StructType structType, HoodieTable table) - throws IOException { - HoodieRowParquetWriteSupport writeSupport = - new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, null, writeConfig); - return new HoodieInternalRowParquetWriter( - path, new HoodieParquetConfig<>( - writeSupport, - writeConfig.getParquetCompressionCodec(), - writeConfig.getParquetBlockSize(), - writeConfig.getParquetPageSize(), - writeConfig.getParquetMaxFileSize(), - writeSupport.getHadoopConf(), - writeConfig.getParquetCompressionRatio(), - writeConfig.parquetDictionaryEnabled()) - ); + return Option.empty(); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java index 310afd4f142b2..e7c6ccd6fadb1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieInsertException; import org.apache.hudi.hadoop.CachingPath; @@ -56,11 +57,6 @@ public class HoodieRowCreateHandle implements Serializable { 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; @@ -87,14 +83,13 @@ public HoodieRowCreateHandle(HoodieTable table, int taskPartitionId, long taskId, long taskEpochId, - StructType structType, - boolean populateMetaFields) { + StructType structType) { this.partitionPath = partitionPath; this.table = table; this.writeConfig = writeConfig; this.fileId = fileId; - this.currTimer = new HoodieTimer(true); + this.currTimer = HoodieTimer.start(); FileSystem fs = table.getMetaClient().getFs(); @@ -102,7 +97,7 @@ public HoodieRowCreateHandle(HoodieTable table, String fileName = FSUtils.makeBaseFileName(instantTime, writeToken, this.fileId, table.getBaseFileExtension()); this.path = makeNewPath(fs, partitionPath, fileName, writeConfig); - this.populateMetaFields = populateMetaFields; + this.populateMetaFields = writeConfig.populateMetaFields(); this.fileName = UTF8String.fromString(path.getName()); this.commitTime = UTF8String.fromString(instantTime); this.seqIdGenerator = (id) -> HoodieRecord.generateSequenceId(instantTime, taskPartitionId, id); @@ -121,12 +116,15 @@ public HoodieRowCreateHandle(HoodieTable table, FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath), table.getPartitionMetafileFormat()); partitionMetadata.trySave(taskPartitionId); + createMarkerFile(partitionPath, fileName, instantTime, table, writeConfig); - this.fileWriter = createNewFileWriter(path, table, writeConfig, structType); + + this.fileWriter = HoodieInternalRowFileWriterFactory.getInternalRowFileWriter(path, table, writeConfig, structType); } catch (IOException e) { throw new HoodieInsertException("Failed to initialize file writer for path " + path, e); } - LOG.info("New handle created for partition :" + partitionPath + " with fileId " + fileId); + + LOG.info("New handle created for partition: " + partitionPath + " with fileId " + fileId); } /** @@ -137,47 +135,59 @@ public HoodieRowCreateHandle(HoodieTable table, * @throws IOException */ public void write(InternalRow row) throws IOException { + if (populateMetaFields) { + writeRow(row); + } else { + writeRowNoMetaFields(row); + } + } + + private void writeRow(InternalRow row) { try { // 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]]) + // - Conversions b/w Spark internal 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); - } + UTF8String recordKey = row.getUTF8String(HoodieRecord.RECORD_KEY_META_FIELD_ORD); + UTF8String partitionPath = row.getUTF8String(HoodieRecord.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())); + + InternalRow updatedRow = new HoodieInternalRow(commitTime, seqId, recordKey, + partitionPath, fileName, row, true); try { 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) { + } catch (Exception t) { writeStatus.markFailure(recordKey.toString(), t); } - } catch (Throwable ge) { - writeStatus.setGlobalError(ge); - throw ge; + } catch (Exception e) { + writeStatus.setGlobalError(e); + throw e; + } + } + + private void writeRowNoMetaFields(InternalRow row) { + try { + // TODO make sure writing w/ and w/o meta fields is consistent (currently writing w/o + // meta-fields would fail if any record will, while when writing w/ meta-fields it won't) + fileWriter.writeRow(row); + writeStatus.markSuccess(); + } catch (Exception e) { + writeStatus.setGlobalError(e); + throw new HoodieException("Exception thrown while writing spark InternalRows to file ", e); } } /** - * @returns {@code true} if this handle can take in more writes. else {@code false}. + * Returns {@code true} if this handle can take in more writes. else {@code false}. */ public boolean canWrite() { return fileWriter.canWrite(); @@ -188,7 +198,6 @@ public boolean canWrite() { * status of the writes to this handle. * * @return the {@link HoodieInternalWriteStatus} containing the stats and status of the writes to this handle. - * @throws IOException */ public HoodieInternalWriteStatus close() throws IOException { fileWriter.close(); @@ -245,10 +254,4 @@ private static String getWriteToken(int taskPartitionId, long taskId, long taskE return taskPartitionId + "-" + taskId + "-" + taskEpochId; } - protected HoodieInternalRowFileWriter createNewFileWriter( - Path path, HoodieTable hoodieTable, HoodieWriteConfig config, StructType schema) - throws IOException { - return HoodieInternalRowFileWriterFactory.getInternalRowFileWriter( - path, hoodieTable, config, schema); - } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java index 46c2a6d835191..ce26bcb4748e5 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java @@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.parquet.hadoop.api.WriteSupport; import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport; @@ -45,14 +46,13 @@ public class HoodieRowParquetWriteSupport extends ParquetWriteSupport { private UTF8String minRecordKey; private UTF8String maxRecordKey; - public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, BloomFilter bloomFilter, HoodieWriteConfig writeConfig) { - super(); + public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, Option bloomFilterOpt, HoodieWriteConfig writeConfig) { Configuration hadoopConf = new Configuration(conf); hadoopConf.set("spark.sql.parquet.writeLegacyFormat", writeConfig.parquetWriteLegacyFormatEnabled()); hadoopConf.set("spark.sql.parquet.outputTimestampType", writeConfig.parquetOutputTimestampType()); this.hadoopConf = hadoopConf; setSchema(structType, hadoopConf); - this.bloomFilter = bloomFilter; + this.bloomFilter = bloomFilterOpt.orElse(null); } public Configuration getHadoopConf() { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java index b7cdcf851acd4..afda4c0a7faad 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java @@ -19,132 +19,560 @@ 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.HoodieSparkUtils; +import org.apache.hudi.client.utils.SparkRowSerDe; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.common.util.PartitionPathEncodeUtils; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieKeyException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.sql.HoodieUnsafeRowUtils; +import org.apache.spark.sql.HoodieUnsafeRowUtils$; 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.DateType; import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.TimestampType; +import org.apache.spark.unsafe.types.UTF8String; import scala.Function1; -import java.util.HashMap; +import javax.annotation.concurrent.ThreadSafe; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.function.Supplier; + +import static org.apache.hudi.common.util.CollectionUtils.tail; +import static org.apache.hudi.common.util.ValidationUtils.checkState; +import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH_SEPARATOR; +import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_RECORD_KEY_PARTS_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; /** - * Base class for the built-in key generators. Contains methods structured for - * code reuse amongst them. + * Base class for all built-in key generators. + * + * NOTE: By default it implements all the methods of {@link SparkKeyGeneratorInterface}, which + * by default however fallback to Avro implementation. For maximum performance (to avoid + * conversion from Spark's internal data-types to Avro) you should override these methods + * in your implementation. + * + * TODO rename to AvroFallbackBaseKeyGenerator */ +@ThreadSafe public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements SparkKeyGeneratorInterface { - private static final String STRUCT_NAME = "hoodieRowTopLevelField"; - private static final String NAMESPACE = "hoodieRow"; - private Function1 converterFn = null; - private final AtomicBoolean validatePartitionFields = new AtomicBoolean(false); - protected StructType structType; + private static final Logger LOG = LogManager.getLogger(BuiltinKeyGenerator.class); + + private static final String COMPOSITE_KEY_FIELD_VALUE_INFIX = ":"; + + protected static final UTF8String HUDI_DEFAULT_PARTITION_PATH_UTF8 = UTF8String.fromString(HUDI_DEFAULT_PARTITION_PATH); + protected static final UTF8String NULL_RECORD_KEY_PLACEHOLDER_UTF8 = UTF8String.fromString(NULL_RECORDKEY_PLACEHOLDER); + protected static final UTF8String EMPTY_RECORD_KEY_PLACEHOLDER_UTF8 = UTF8String.fromString(EMPTY_RECORDKEY_PLACEHOLDER); + - protected Map, DataType>> recordKeySchemaInfo = new HashMap<>(); - protected Map, DataType>> partitionPathSchemaInfo = new HashMap<>(); + protected transient volatile SparkRowConverter rowConverter; + protected transient volatile SparkRowAccessor rowAccessor; protected BuiltinKeyGenerator(TypedProperties config) { super(config); } - /** - * Fetch record key from {@link Row}. - * - * @param row instance of {@link Row} from which record key is requested. - * @return the record key of interest from {@link Row}. - */ @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(); + tryInitRowConverter(row.schema()); + // NOTE: This implementation has considerable computational overhead and has to be overridden + // to provide for optimal performance on Spark. This implementation provided exclusively + // for compatibility reasons. + return getRecordKey(rowConverter.convertToAvro(row)); } @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); + public UTF8String getRecordKey(InternalRow internalRow, StructType schema) { + tryInitRowConverter(schema); + // NOTE: This implementation has considerable computational overhead and has to be overridden + // to provide for optimal performance on Spark. This implementation provided exclusively + // for compatibility reasons. + return UTF8String.fromString(getRecordKey(rowConverter.convertToAvro(internalRow))); + } + + @Override + public String getPartitionPath(Row row) { + tryInitRowConverter(row.schema()); + // NOTE: This implementation has considerable computational overhead and has to be overridden + // to provide for optimal performance on Spark. This implementation provided exclusively + // for compatibility reasons. + return getPartitionPath(rowConverter.convertToAvro(row)); + } + + @Override + public UTF8String getPartitionPath(InternalRow internalRow, StructType schema) { + tryInitRowConverter(schema); + // NOTE: This implementation has considerable computational overhead and has to be overridden + // to provide for optimal performance on Spark. This implementation provided exclusively + // for compatibility reasons. + GenericRecord avroRecord = rowConverter.convertToAvro(internalRow); + return UTF8String.fromString(getPartitionPath(avroRecord)); + } + + protected void tryInitRowAccessor(StructType schema) { + if (this.rowAccessor == null) { + synchronized (this) { + if (this.rowAccessor == null) { + this.rowAccessor = new SparkRowAccessor(schema); + } + } } } + /** - * 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}. + * NOTE: This method has to stay final (so that it's easier for JIT compiler to apply certain + * optimizations, like inlining) */ + protected final String combinePartitionPath(Object... partitionPathParts) { + return combinePartitionPathInternal( + JavaStringBuilder::new, + BuiltinKeyGenerator::toString, + this::tryEncodePartitionPath, + BuiltinKeyGenerator::handleNullOrEmptyPartitionPathPart, + partitionPathParts + ); + } - @Override - @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public String getPartitionPath(Row row) { - if (null == converterFn) { - converterFn = AvroConversionUtils.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE); + /** + * NOTE: This method has to stay final (so that it's easier for JIT compiler to apply certain + * optimizations, like inlining) + */ + protected final UTF8String combinePartitionPathUnsafe(Object... partitionPathParts) { + return combinePartitionPathInternal( + UTF8StringBuilder::new, + BuiltinKeyGenerator::toUTF8String, + this::tryEncodePartitionPathUTF8, + BuiltinKeyGenerator::handleNullOrEmptyPartitionPathPartUTF8, + partitionPathParts + ); + } + + /** + * NOTE: This method has to stay final (so that it's easier for JIT compiler to apply certain + * optimizations, like inlining) + */ + protected final String combineRecordKey(Object... recordKeyParts) { + return combineRecordKeyInternal( + JavaStringBuilder::new, + BuiltinKeyGenerator::toString, + BuiltinKeyGenerator::handleNullRecordKey, + recordKeyParts + ); + } + + /** + * NOTE: This method has to stay final (so that it's easier for JIT compiler to apply certain + * optimizations, like inlining) + */ + protected final UTF8String combineRecordKeyUnsafe(Object... recordKeyParts) { + return combineRecordKeyInternal( + UTF8StringBuilder::new, + BuiltinKeyGenerator::toUTF8String, + BuiltinKeyGenerator::handleNullRecordKey, + recordKeyParts + ); + } + + /** + * NOTE: This method has to stay final (so that it's easier for JIT compiler to apply certain + * optimizations, like inlining) + */ + protected final String combineCompositeRecordKey(Object... recordKeyParts) { + return combineCompositeRecordKeyInternal( + JavaStringBuilder::new, + BuiltinKeyGenerator::toString, + BuiltinKeyGenerator::handleNullOrEmptyCompositeKeyPart, + BuiltinKeyGenerator::isNullOrEmptyCompositeKeyPart, + recordKeyParts + ); + } + + /** + * NOTE: This method has to stay final (so that it's easier for JIT compiler to apply certain + * optimizations, like inlining) + */ + protected final UTF8String combineCompositeRecordKeyUnsafe(Object... recordKeyParts) { + return combineCompositeRecordKeyInternal( + UTF8StringBuilder::new, + BuiltinKeyGenerator::toUTF8String, + BuiltinKeyGenerator::handleNullOrEmptyCompositeKeyPartUTF8, + BuiltinKeyGenerator::isNullOrEmptyCompositeKeyPartUTF8, + recordKeyParts + ); + } + + private S combineRecordKeyInternal( + Supplier> builderFactory, + Function converter, + Function emptyKeyPartHandler, + Object... recordKeyParts + ) { + if (recordKeyParts.length == 1) { + return emptyKeyPartHandler.apply(converter.apply(recordKeyParts[0])); } - return getKey(converterFn.apply(row)).getPartitionPath(); + + StringBuilder sb = builderFactory.get(); + for (int i = 0; i < recordKeyParts.length; ++i) { + // NOTE: If record-key part has already been a string [[toString]] will be a no-op + sb.append(emptyKeyPartHandler.apply(converter.apply(recordKeyParts[i]))); + + if (i < recordKeyParts.length - 1) { + sb.appendJava(DEFAULT_RECORD_KEY_PARTS_SEPARATOR); + } + } + + return sb.build(); + } + + private S combineCompositeRecordKeyInternal( + Supplier> builderFactory, + Function converter, + Function emptyKeyPartHandler, + Predicate isNullOrEmptyKeyPartPredicate, + Object... recordKeyParts + ) { + boolean hasNonNullNonEmptyPart = false; + + StringBuilder sb = builderFactory.get(); + for (int i = 0; i < recordKeyParts.length; ++i) { + // NOTE: If record-key part has already been a string [[toString]] will be a no-op + S convertedKeyPart = emptyKeyPartHandler.apply(converter.apply(recordKeyParts[i])); + + sb.appendJava(recordKeyFields.get(i)); + sb.appendJava(COMPOSITE_KEY_FIELD_VALUE_INFIX); + sb.append(convertedKeyPart); + // This check is to validate that overall composite-key has at least one non-null, non-empty + // segment + hasNonNullNonEmptyPart |= !isNullOrEmptyKeyPartPredicate.test(convertedKeyPart); + + if (i < recordKeyParts.length - 1) { + sb.appendJava(DEFAULT_RECORD_KEY_PARTS_SEPARATOR); + } + } + + if (hasNonNullNonEmptyPart) { + return sb.build(); + } else { + throw new HoodieKeyException(String.format("All of the values for (%s) were either null or empty", recordKeyFields)); + } + } + + private S combinePartitionPathInternal(Supplier> builderFactory, + Function converter, + Function encoder, + Function emptyHandler, + Object... partitionPathParts) { + checkState(partitionPathParts.length == partitionPathFields.size()); + // Avoid creating [[StringBuilder]] in case there's just one partition-path part, + // and Hive-style of partitioning is not required + if (!hiveStylePartitioning && partitionPathParts.length == 1) { + return emptyHandler.apply(converter.apply(partitionPathParts[0])); + } + + StringBuilder sb = builderFactory.get(); + for (int i = 0; i < partitionPathParts.length; ++i) { + S partitionPathPartStr = encoder.apply(emptyHandler.apply(converter.apply(partitionPathParts[i]))); + + if (hiveStylePartitioning) { + sb.appendJava(partitionPathFields.get(i)) + .appendJava("=") + .append(partitionPathPartStr); + } else { + sb.append(partitionPathPartStr); + } + + if (i < partitionPathParts.length - 1) { + sb.appendJava(DEFAULT_PARTITION_PATH_SEPARATOR); + } + } + + return sb.build(); + } + + private String tryEncodePartitionPath(String partitionPathPart) { + return encodePartitionPath ? PartitionPathEncodeUtils.escapePathName(partitionPathPart) : partitionPathPart; + } + + private UTF8String tryEncodePartitionPathUTF8(UTF8String partitionPathPart) { + // NOTE: This method avoids [[UTF8String]] to [[String]] conversion (and back) unless + // partition-path encoding is enabled + return encodePartitionPath ? UTF8String.fromString(PartitionPathEncodeUtils.escapePathName(partitionPathPart.toString())) : partitionPathPart; + } + + private void tryInitRowConverter(StructType structType) { + if (rowConverter == null) { + synchronized (this) { + if (rowConverter == null) { + rowConverter = new SparkRowConverter(structType); + } + } + } + } + + protected static String requireNonNullNonEmptyKey(String key) { + if (key != null && key.length() > 0) { + return key; + } else { + throw new HoodieKeyException("Record key has to be non-empty string!"); + } + } + + protected static UTF8String requireNonNullNonEmptyKey(UTF8String key) { + if (key != null && key.numChars() > 0) { + return key; + } else { + throw new HoodieKeyException("Record key has to be non-empty string!"); + } + } + + protected static S handleNullRecordKey(S s) { + if (s == null || s.toString().isEmpty()) { + throw new HoodieKeyException("Record key has to be non-null!"); + } + + return s; + } + + private static UTF8String toUTF8String(Object o) { + if (o == null) { + return null; + } else if (o instanceof UTF8String) { + return (UTF8String) o; + } else { + // NOTE: If object is a [[String]], [[toString]] would be a no-op + return UTF8String.fromString(o.toString()); + } + } + + private static String toString(Object o) { + return o == null ? null : o.toString(); + } + + private static String handleNullOrEmptyCompositeKeyPart(Object keyPart) { + if (keyPart == null) { + return NULL_RECORDKEY_PLACEHOLDER; + } else { + // NOTE: [[toString]] is a no-op if key-part was already a [[String]] + String keyPartStr = keyPart.toString(); + return !keyPartStr.isEmpty() ? keyPartStr : EMPTY_RECORDKEY_PLACEHOLDER; + } + } + + private static UTF8String handleNullOrEmptyCompositeKeyPartUTF8(UTF8String keyPart) { + if (keyPart == null) { + return NULL_RECORD_KEY_PLACEHOLDER_UTF8; + } else if (keyPart.numChars() == 0) { + return EMPTY_RECORD_KEY_PLACEHOLDER_UTF8; + } + + return keyPart; + } + + @SuppressWarnings("StringEquality") + private static boolean isNullOrEmptyCompositeKeyPart(String keyPart) { + // NOTE: Converted key-part is compared against null/empty stub using ref-equality + // for performance reasons (it relies on the fact that we're using internalized + // constants) + return keyPart == NULL_RECORDKEY_PLACEHOLDER || keyPart == EMPTY_RECORDKEY_PLACEHOLDER; + } + + private static boolean isNullOrEmptyCompositeKeyPartUTF8(UTF8String keyPart) { + // NOTE: Converted key-part is compared against null/empty stub using ref-equality + // for performance reasons (it relies on the fact that we're using internalized + // constants) + return keyPart == NULL_RECORD_KEY_PLACEHOLDER_UTF8 || keyPart == EMPTY_RECORD_KEY_PLACEHOLDER_UTF8; + } + + private static String handleNullOrEmptyPartitionPathPart(Object partitionPathPart) { + if (partitionPathPart == null) { + return HUDI_DEFAULT_PARTITION_PATH; + } else { + // NOTE: [[toString]] is a no-op if key-part was already a [[String]] + String keyPartStr = partitionPathPart.toString(); + return keyPartStr.isEmpty() ? HUDI_DEFAULT_PARTITION_PATH : keyPartStr; + } + } + + private static UTF8String handleNullOrEmptyPartitionPathPartUTF8(UTF8String keyPart) { + if (keyPart == null || keyPart.numChars() == 0) { + return HUDI_DEFAULT_PARTITION_PATH_UTF8; + } + + return keyPart; } /** - * Fetch partition path from {@link InternalRow}. + * 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/ + * + *
    + *
  1. {@code RowEncoder#deserializerFor}, as well as
  2. + *
  3. {@code HoodieAvroUtils#convertValueForAvroLogicalTypes}
  4. + *
* - * @param internalRow {@link InternalRow} instance from which partition path needs to be fetched from. - * @param structType schema of the internalRow. - * @return the partition path. + * @param dataType target data-type of the given value + * @param value target value to be converted */ - @Override - @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public String getPartitionPath(InternalRow internalRow, StructType structType) { - try { - buildFieldSchemaInfoIfNeeded(structType); - return RowKeyGeneratorHelper.getPartitionPathFromInternalRow(internalRow, getPartitionPathFields(), - hiveStylePartitioning, partitionPathSchemaInfo); - } catch (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))); - if (getPartitionPathFields() != null) { - getPartitionPathFields().stream().filter(f -> !f.isEmpty()) - .forEach(f -> partitionPathSchemaInfo.put(f, RowKeyGeneratorHelper.getFieldSchemaInfo(structType, f, false))); + private static Object convertToLogicalDataType(DataType dataType, Object value) { + if (value == null) { + return null; + } else 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; + } + + protected static class SparkRowConverter { + private static final String STRUCT_NAME = "hoodieRowTopLevelField"; + private static final String NAMESPACE = "hoodieRow"; + + private final Function1 avroConverter; + private final SparkRowSerDe rowSerDe; + + SparkRowConverter(StructType schema) { + this.rowSerDe = HoodieSparkUtils.getDeserializer(schema); + this.avroConverter = AvroConversionUtils.createConverterToAvro(schema, STRUCT_NAME, NAMESPACE); + } + + GenericRecord convertToAvro(Row row) { + return avroConverter.apply(row); + } + + GenericRecord convertToAvro(InternalRow row) { + return avroConverter.apply(rowSerDe.deserializeRow(row)); + } + } + + protected class SparkRowAccessor { + private final HoodieUnsafeRowUtils.NestedFieldPath[] recordKeyFieldsPaths; + private final HoodieUnsafeRowUtils.NestedFieldPath[] partitionPathFieldsPaths; + + SparkRowAccessor(StructType schema) { + this.recordKeyFieldsPaths = resolveNestedFieldPaths(getRecordKeyFieldNames(), schema); + this.partitionPathFieldsPaths = resolveNestedFieldPaths(getPartitionPathFields(), schema); + } + + public Object[] getRecordKeyParts(Row row) { + return getNestedFieldValues(row, recordKeyFieldsPaths); + } + + public Object[] getRecordPartitionPathValues(Row row) { + return getNestedFieldValues(row, partitionPathFieldsPaths); + } + + public Object[] getRecordKeyParts(InternalRow row) { + return getNestedFieldValues(row, recordKeyFieldsPaths); + } + + public Object[] getRecordPartitionPathValues(InternalRow row) { + return getNestedFieldValues(row, partitionPathFieldsPaths); + } + + private Object[] getNestedFieldValues(Row row, HoodieUnsafeRowUtils.NestedFieldPath[] nestedFieldsPaths) { + Object[] nestedFieldValues = new Object[nestedFieldsPaths.length]; + for (int i = 0; i < nestedFieldsPaths.length; ++i) { + nestedFieldValues[i] = HoodieUnsafeRowUtils$.MODULE$.getNestedRowValue(row, nestedFieldsPaths[i]); + } + return nestedFieldValues; + } + + private Object[] getNestedFieldValues(InternalRow row, HoodieUnsafeRowUtils.NestedFieldPath[] nestedFieldsPaths) { + Object[] nestedFieldValues = new Object[nestedFieldsPaths.length]; + for (int i = 0; i < nestedFieldsPaths.length; ++i) { + Object rawValue = HoodieUnsafeRowUtils$.MODULE$.getNestedInternalRowValue(row, nestedFieldsPaths[i]); + DataType dataType = tail(nestedFieldsPaths[i].parts())._2.dataType(); + + nestedFieldValues[i] = convertToLogicalDataType(dataType, rawValue); + } + + return nestedFieldValues; + } + + private HoodieUnsafeRowUtils.NestedFieldPath[] resolveNestedFieldPaths(List fieldPaths, StructType schema) { + try { + return fieldPaths.stream() + .map(fieldPath -> HoodieUnsafeRowUtils$.MODULE$.composeNestedFieldPath(schema, fieldPath)) + .toArray(HoodieUnsafeRowUtils.NestedFieldPath[]::new); + } catch (Exception e) { + LOG.error(String.format("Failed to resolve nested field-paths (%s) in schema (%s)", fieldPaths, schema), e); + throw new HoodieException("Failed to resolve nested field-paths", e); } } } - protected String getPartitionPathInternal(InternalRow row, StructType structType) { - buildFieldSchemaInfoIfNeeded(structType); - validatePartitionFieldsForInternalRow(); - return RowKeyGeneratorHelper.getPartitionPathFromInternalRow(row, getPartitionPathFields(), - hiveStylePartitioning, partitionPathSchemaInfo); + /** + * This is a generic interface closing the gap and unifying the {@link java.lang.StringBuilder} with + * {@link org.apache.hudi.unsafe.UTF8StringBuilder} implementations, allowing us to avoid code-duplication by performing + * most of the key-generation in a generic and unified way + * + * @param target string type this builder is producing (could either be native {@link String} + * or alternatively {@link UTF8String} + */ + private interface StringBuilder { + default StringBuilder append(S s) { + return appendJava(s.toString()); + } + + StringBuilder appendJava(String s); + + S build(); } - protected void validatePartitionFieldsForInternalRow() { - if (!validatePartitionFields.getAndSet(true)) { - partitionPathSchemaInfo.values().forEach(entry -> { - if (entry.getKey().size() > 1) { - throw new IllegalArgumentException("Nested column for partitioning is not supported with disabling meta columns"); - } - }); + private static class JavaStringBuilder implements StringBuilder { + private final java.lang.StringBuilder sb = new java.lang.StringBuilder(); + + @Override + public StringBuilder appendJava(String s) { + sb.append(s); + return this; + } + + @Override + public String build() { + return sb.toString(); + } + } + + private static class UTF8StringBuilder implements StringBuilder { + private final org.apache.hudi.unsafe.UTF8StringBuilder sb = new org.apache.hudi.unsafe.UTF8StringBuilder(); + + @Override + public StringBuilder appendJava(String s) { + sb.append(s); + return this; + } + + @Override + public StringBuilder append(UTF8String s) { + sb.append(s); + return this; + } + + @Override + public UTF8String build() { + return sb.build(); } } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java index b09ff0755a63f..1eac7579757c7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java @@ -17,19 +17,23 @@ package org.apache.hudi.keygen; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; - -import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; import java.util.Arrays; import java.util.stream.Collectors; /** - * Complex key generator, which takes names of fields to be used for recordKey and partitionPath as configs. + * Key generator prefixing field names before corresponding record-key parts. + * + *

+ * For example, for the schema of {@code { "key": string, "value": bytes }}, and corresponding record + * {@code { "key": "foo" }}, record-key "key:foo" will be produced. */ public class ComplexKeyGenerator extends BuiltinKeyGenerator { @@ -45,7 +49,7 @@ public ComplexKeyGenerator(TypedProperties props) { .map(String::trim) .filter(s -> !s.isEmpty()) .collect(Collectors.toList()); - complexAvroKeyGenerator = new ComplexAvroKeyGenerator(props); + this.complexAvroKeyGenerator = new ComplexAvroKeyGenerator(props); } @Override @@ -60,26 +64,25 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(Row row) { - buildFieldSchemaInfoIfNeeded(row.schema()); - return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, true); + tryInitRowAccessor(row.schema()); + return combineCompositeRecordKey(rowAccessor.getRecordKeyParts(row)); } @Override - public String getRecordKey(InternalRow internalRow, StructType schema) { - buildFieldSchemaInfoIfNeeded(schema); - return RowKeyGeneratorHelper.getRecordKeyFromInternalRow(internalRow, getRecordKeyFields(), recordKeySchemaInfo, true); + public UTF8String getRecordKey(InternalRow internalRow, StructType schema) { + tryInitRowAccessor(schema); + return combineCompositeRecordKeyUnsafe(rowAccessor.getRecordKeyParts(internalRow)); } @Override public String getPartitionPath(Row row) { - buildFieldSchemaInfoIfNeeded(row.schema()); - return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(), - hiveStylePartitioning, partitionPathSchemaInfo); + tryInitRowAccessor(row.schema()); + return combinePartitionPath(rowAccessor.getRecordPartitionPathValues(row)); } @Override - public String getPartitionPath(InternalRow row, StructType structType) { - return getPartitionPathInternal(row, structType); + public UTF8String getPartitionPath(InternalRow row, StructType schema) { + tryInitRowAccessor(schema); + return combinePartitionPathUnsafe(rowAccessor.getRecordPartitionPathValues(row)); } - } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java index c43892af451c5..fcd94bb4f1550 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java @@ -29,9 +29,11 @@ import org.apache.spark.sql.Row; 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.util.Arrays; +import java.util.Collections; import java.util.stream.Collectors; /** @@ -46,16 +48,29 @@ * field in the partition path, use field1:simple 3. If you want your table to be non partitioned, simply leave it as blank. * * RecordKey is internally generated using either SimpleKeyGenerator or ComplexKeyGenerator. + * + * @deprecated */ +@Deprecated public class CustomKeyGenerator extends BuiltinKeyGenerator { private final CustomAvroKeyGenerator customAvroKeyGenerator; public CustomKeyGenerator(TypedProperties props) { - super(props); - this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()).split(",")).map(String::trim).collect(Collectors.toList()); - this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()).split(",")).map(String::trim).collect(Collectors.toList()); - customAvroKeyGenerator = new CustomAvroKeyGenerator(props); + // NOTE: We have to strip partition-path configuration, since it could only be interpreted by + // this key-gen + super(stripPartitionPathConfig(props)); + this.recordKeyFields = + Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()).split(",")) + .map(String::trim) + .collect(Collectors.toList()); + String partitionPathFields = props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()); + this.partitionPathFields = partitionPathFields == null + ? Collections.emptyList() + : Arrays.stream(partitionPathFields.split(",")).map(String::trim).collect(Collectors.toList()); + this.customAvroKeyGenerator = new CustomAvroKeyGenerator(props); + + validateRecordKeyFields(); } @Override @@ -70,9 +85,8 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(Row row) { - validateRecordKeyFields(); - return getRecordKeyFields().size() == 1 - ? new SimpleKeyGenerator(config).getRecordKey(row) + return getRecordKeyFieldNames().size() == 1 + ? new SimpleKeyGenerator(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()), null).getRecordKey(row) : new ComplexKeyGenerator(config).getRecordKey(row); } @@ -82,8 +96,8 @@ public String getPartitionPath(Row row) { } @Override - public String getPartitionPath(InternalRow row, StructType structType) { - return getPartitionPath(Option.empty(), Option.empty(), Option.of(Pair.of(row, structType))); + public UTF8String getPartitionPath(InternalRow row, StructType schema) { + return UTF8String.fromString(getPartitionPath(Option.empty(), Option.empty(), Option.of(Pair.of(row, schema)))); } private String getPartitionPath(Option record, Option row, Option> internalRowStructTypePair) { @@ -99,7 +113,7 @@ private String getPartitionPath(Option record, Option row, O return ""; } for (String field : getPartitionPathFields()) { - String[] fieldWithType = field.split(customAvroKeyGenerator.SPLIT_REGEX); + String[] fieldWithType = field.split(CustomAvroKeyGenerator.SPLIT_REGEX); if (fieldWithType.length != 2) { throw new HoodieKeyGeneratorException("Unable to find field names for partition path in proper format"); } @@ -142,9 +156,18 @@ private String getPartitionPath(Option record, Option row, O } private void validateRecordKeyFields() { - if (getRecordKeyFields() == null || getRecordKeyFields().isEmpty()) { + if (getRecordKeyFieldNames() == null || getRecordKeyFieldNames().isEmpty()) { throw new HoodieKeyException("Unable to find field names for record key in cfg"); } } + + private static TypedProperties stripPartitionPathConfig(TypedProperties props) { + TypedProperties filtered = new TypedProperties(props); + // NOTE: We have to stub it out w/ empty string, since we properties are: + // - Expected to bear this config + // - Can't be stubbed out w/ null + filtered.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), ""); + return filtered; + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java index 6cf674f18e508..7fcc16094eadc 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java @@ -25,6 +25,7 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; import java.util.ArrayList; import java.util.Arrays; @@ -40,7 +41,12 @@ public class GlobalDeleteKeyGenerator extends BuiltinKeyGenerator { public GlobalDeleteKeyGenerator(TypedProperties config) { super(config); this.recordKeyFields = Arrays.asList(config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()).split(",")); - globalAvroDeleteKeyGenerator = new GlobalAvroDeleteKeyGenerator(config); + this.globalAvroDeleteKeyGenerator = new GlobalAvroDeleteKeyGenerator(config); + } + + @Override + public List getPartitionPathFields() { + return new ArrayList<>(); } @Override @@ -53,21 +59,16 @@ public String getPartitionPath(GenericRecord record) { return globalAvroDeleteKeyGenerator.getPartitionPath(record); } - @Override - public List getPartitionPathFields() { - return new ArrayList<>(); - } - @Override public String getRecordKey(Row row) { - buildFieldSchemaInfoIfNeeded(row.schema()); - return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, true); + tryInitRowAccessor(row.schema()); + return combineCompositeRecordKey(rowAccessor.getRecordKeyParts(row)); } @Override - public String getRecordKey(InternalRow internalRow, StructType schema) { - buildFieldSchemaInfoIfNeeded(schema); - return RowKeyGeneratorHelper.getRecordKeyFromInternalRow(internalRow, getRecordKeyFields(), recordKeySchemaInfo, true); + public UTF8String getRecordKey(InternalRow internalRow, StructType schema) { + tryInitRowAccessor(schema); + return combineCompositeRecordKeyUnsafe(rowAccessor.getRecordKeyParts(internalRow)); } @Override @@ -76,8 +77,8 @@ public String getPartitionPath(Row row) { } @Override - public String getPartitionPath(InternalRow row, StructType structType) { - return globalAvroDeleteKeyGenerator.getEmptyPartition(); + public UTF8String getPartitionPath(InternalRow row, StructType schema) { + return UTF8String.EMPTY_UTF8; } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java index dc8b253b0f1be..ccd37dc9ce324 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java @@ -18,13 +18,13 @@ package org.apache.hudi.keygen; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; - -import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; import java.util.Arrays; import java.util.Collections; @@ -41,9 +41,16 @@ public class NonpartitionedKeyGenerator extends BuiltinKeyGenerator { public NonpartitionedKeyGenerator(TypedProperties props) { super(props); this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()) - .split(",")).map(String::trim).collect(Collectors.toList()); + .split(",")) + .map(String::trim) + .collect(Collectors.toList()); this.partitionPathFields = Collections.emptyList(); - nonpartitionedAvroKeyGenerator = new NonpartitionedAvroKeyGenerator(props); + this.nonpartitionedAvroKeyGenerator = new NonpartitionedAvroKeyGenerator(props); + } + + @Override + public List getPartitionPathFields() { + return nonpartitionedAvroKeyGenerator.getPartitionPathFields(); } @Override @@ -52,19 +59,20 @@ public String getRecordKey(GenericRecord record) { } @Override - public String getPartitionPath(GenericRecord record) { - return nonpartitionedAvroKeyGenerator.getPartitionPath(record); + public String getRecordKey(Row row) { + tryInitRowAccessor(row.schema()); + return combineRecordKey(rowAccessor.getRecordKeyParts(row)); } @Override - public List getPartitionPathFields() { - return nonpartitionedAvroKeyGenerator.getPartitionPathFields(); + public UTF8String getRecordKey(InternalRow internalRow, StructType schema) { + tryInitRowAccessor(schema); + return combineRecordKeyUnsafe(rowAccessor.getRecordKeyParts(internalRow)); } @Override - public String getRecordKey(Row row) { - buildFieldSchemaInfoIfNeeded(row.schema()); - return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, false); + public String getPartitionPath(GenericRecord record) { + return nonpartitionedAvroKeyGenerator.getPartitionPath(record); } @Override @@ -73,8 +81,8 @@ public String getPartitionPath(Row row) { } @Override - public String getPartitionPath(InternalRow internalRow, StructType structType) { - return nonpartitionedAvroKeyGenerator.getEmptyPartition(); + public UTF8String getPartitionPath(InternalRow row, StructType schema) { + return UTF8String.EMPTY_UTF8; } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/RowKeyGenUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/RowKeyGenUtils.java deleted file mode 100644 index 9616212378202..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/RowKeyGenUtils.java +++ /dev/null @@ -1,59 +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.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/ - * - *

    - *
  1. {@code RowEncoder#deserializerFor}, as well as
  2. - *
  3. {@code HoodieAvroUtils#convertValueForAvroLogicalTypes}
  4. - *
- * - * @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; - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java deleted file mode 100644 index c79481bd2ba5d..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java +++ /dev/null @@ -1,355 +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.keygen; - -import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.exception.HoodieKeyException; - -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.DataTypes; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -import java.sql.Timestamp; -import java.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -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 recordKeyFields, - Map, DataType>> recordKeyPositions, boolean prefixFieldName) { - AtomicBoolean keyIsNullOrEmpty = new AtomicBoolean(true); - String toReturn = recordKeyFields.stream().map(field -> { - String val = null; - List 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}. - * - * @param row instance of {@link Row} of interest - * @param recordKeyFields record key fields as a list - * @param recordKeyPositions record key positions for the corresponding record keys in {@code recordKeyFields} - * @param prefixFieldName {@code true} if field name need to be prefixed in the returned result. {@code false} otherwise. - * @return the record key thus generated - */ - public static String getRecordKeyFromRow(Row row, List recordKeyFields, Map, DataType>> recordKeyPositions, boolean prefixFieldName) { - AtomicBoolean keyIsNullOrEmpty = new AtomicBoolean(true); - String toReturn = recordKeyFields.stream().map(field -> { - String val = null; - List fieldPositions = recordKeyPositions.get(field).getKey(); - if (fieldPositions.size() == 1) { // simple field - Integer fieldPos = fieldPositions.get(0); - if (row.isNullAt(fieldPos)) { - val = NULL_RECORDKEY_PLACEHOLDER; - } else { - val = row.getAs(field).toString(); - if (val.isEmpty()) { - val = EMPTY_RECORDKEY_PLACEHOLDER; - } else { - keyIsNullOrEmpty.set(false); - } - } - } else { // nested fields - val = getNestedFieldVal(row, recordKeyPositions.get(field).getKey()).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 partition path for the corresponding {@link Row}. - * - * @param row instance of {@link Row} of interest - * @param partitionPathFields partition path fields as a list - * @param hiveStylePartitioning {@code true} if hive style partitioning is set. {@code false} otherwise - * @param partitionPathPositions partition path positions for the corresponding fields in {@code partitionPathFields} - * @return the generated partition path for the row - */ - public static String getPartitionPathFromRow(Row row, List partitionPathFields, boolean hiveStylePartitioning, Map, DataType>> partitionPathPositions) { - return IntStream.range(0, partitionPathFields.size()).mapToObj(idx -> { - String field = partitionPathFields.get(idx); - String val = null; - List fieldPositions = partitionPathPositions.get(field).getKey(); - if (fieldPositions.size() == 1) { // simple - Integer fieldPos = fieldPositions.get(0); - // for partition path, if field is not found, index will be set to -1 - if (fieldPos == -1 || row.isNullAt(fieldPos)) { - val = HUDI_DEFAULT_PARTITION_PATH; - } else { - Object data = row.get(fieldPos); - val = convertToTimestampIfInstant(data).toString(); - if (val.isEmpty()) { - val = HUDI_DEFAULT_PARTITION_PATH; - } - } - if (hiveStylePartitioning) { - val = field + "=" + val; - } - } else { // nested - Object data = getNestedFieldVal(row, partitionPathPositions.get(field).getKey()); - data = convertToTimestampIfInstant(data); - if (data.toString().contains(NULL_RECORDKEY_PLACEHOLDER) || data.toString().contains(EMPTY_RECORDKEY_PLACEHOLDER)) { - val = hiveStylePartitioning ? field + "=" + HUDI_DEFAULT_PARTITION_PATH : HUDI_DEFAULT_PARTITION_PATH; - } else { - val = hiveStylePartitioning ? field + "=" + data.toString() : data.toString(); - } - } - return val; - }).collect(Collectors.joining(DEFAULT_PARTITION_PATH_SEPARATOR)); - } - - public static String getPartitionPathFromInternalRow(InternalRow internalRow, List partitionPathFields, boolean hiveStylePartitioning, - Map, DataType>> partitionPathPositions) { - return IntStream.range(0, partitionPathFields.size()).mapToObj(idx -> { - String field = partitionPathFields.get(idx); - String val = null; - List fieldPositions = partitionPathPositions.get(field).getKey(); - DataType dataType = partitionPathPositions.get(field).getValue(); - if (fieldPositions.size() == 1) { // simple - Integer fieldPos = fieldPositions.get(0); - // for partition path, if field is not found, index will be set to -1 - if (fieldPos == -1 || internalRow.isNullAt(fieldPos)) { - val = HUDI_DEFAULT_PARTITION_PATH; - } else { - Object value = convertToLogicalDataType(dataType, internalRow.get(fieldPos, dataType)); - if (value == null || value.toString().isEmpty()) { - val = HUDI_DEFAULT_PARTITION_PATH; - } else { - val = value.toString(); - } - } - if (hiveStylePartitioning) { - val = field + "=" + val; - } - } else { // nested - throw new IllegalArgumentException("Nested partitioning is not supported with disabling meta columns."); - } - return val; - }).collect(Collectors.joining(DEFAULT_PARTITION_PATH_SEPARATOR)); - } - - public static Object getFieldValFromInternalRow(InternalRow internalRow, - Integer partitionPathPosition, - DataType partitionPathDataType) { - Object val = null; - if (internalRow.isNullAt(partitionPathPosition)) { - return HUDI_DEFAULT_PARTITION_PATH; - } else { - Object value = partitionPathDataType == DataTypes.StringType ? internalRow.getString(partitionPathPosition) : internalRow.get(partitionPathPosition, partitionPathDataType); - if (value == null || value.toString().isEmpty()) { - val = HUDI_DEFAULT_PARTITION_PATH; - } else { - val = value; - } - } - return val; - } - - - /** - * Fetch the field value located at the positions requested for. - *

- * The fetching logic recursively goes into the nested field based on the position list to get the field value. - * For example, given the row [4357686,key1,2020-03-21,pi,[val1,10]] with the following schema, which has the fourth - * field as a nested field, and positions list as [4,0], - *

- * 0 = "StructField(timestamp,LongType,false)" - * 1 = "StructField(_row_key,StringType,false)" - * 2 = "StructField(ts_ms,StringType,false)" - * 3 = "StructField(pii_col,StringType,false)" - * 4 = "StructField(nested_col,StructType(StructField(prop1,StringType,false), StructField(prop2,LongType,false)),false)" - *

- * the logic fetches the value from field nested_col.prop1. - * If any level of the nested field is null, {@link KeyGenUtils#NULL_RECORDKEY_PLACEHOLDER} is returned. - * If the field value is an empty String, {@link KeyGenUtils#EMPTY_RECORDKEY_PLACEHOLDER} is returned. - * - * @param row instance of {@link Row} of interest - * @param positions tree style positions where the leaf node need to be fetched and returned - * @return the field value as per the positions requested for. - */ - public static Object getNestedFieldVal(Row row, List positions) { - if (positions.size() == 1 && positions.get(0) == -1) { - return HUDI_DEFAULT_PARTITION_PATH; - } - int index = 0; - int totalCount = positions.size(); - Row valueToProcess = row; - Object toReturn = null; - - while (index < totalCount) { - if (valueToProcess.isNullAt(positions.get(index))) { - toReturn = NULL_RECORDKEY_PLACEHOLDER; - break; - } - - if (index < totalCount - 1) { - valueToProcess = (Row) valueToProcess.get(positions.get(index)); - } else { // last index - if (valueToProcess.getAs(positions.get(index)).toString().isEmpty()) { - toReturn = EMPTY_RECORDKEY_PLACEHOLDER; - break; - } - toReturn = valueToProcess.getAs(positions.get(index)); - } - index++; - } - return toReturn; - } - - public static Object getNestedFieldVal(InternalRow internalRow, Pair, 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. - * - * @param structType schema of interest - * @param field field of interest for which the positions are requested for - * @param isRecordKey {@code true} if the field requested for is a record key. {@code false} in case of a partition path. - * @return the positions of the field as per the struct type and the leaf field's datatype. - */ - public static Pair, DataType> getFieldSchemaInfo(StructType structType, String field, boolean isRecordKey) { - String[] slices = field.split("\\."); - List positions = new ArrayList<>(); - int index = 0; - int totalCount = slices.length; - DataType leafFieldDataType = null; - while (index < totalCount) { - String slice = slices[index]; - Option curIndexOpt = structType.getFieldIndex(slice); - if (curIndexOpt.isDefined()) { - int curIndex = (int) curIndexOpt.get(); - positions.add(curIndex); - final StructField nestedField = structType.fields()[curIndex]; - if (index < totalCount - 1) { - if (!(nestedField.dataType() instanceof StructType)) { - if (isRecordKey) { - throw new HoodieKeyException("Nested field should be of type StructType " + nestedField); - } else { - positions = Collections.singletonList(-1); - break; - } - } - structType = (StructType) nestedField.dataType(); - } else { - // leaf node. - leafFieldDataType = nestedField.dataType(); - } - } else { - if (isRecordKey) { - throw new HoodieKeyException("Can't find " + slice + " in StructType for the field " + field); - } else { - positions = Collections.singletonList(-1); - break; - } - } - index++; - } - return Pair.of(positions, leafFieldDataType); - } - - private static Object convertToTimestampIfInstant(Object data) { - if (data instanceof Instant) { - return Timestamp.from((Instant) data); - } - return data; - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java index 2f139a61eace8..dcffdf3cdb836 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java @@ -18,13 +18,13 @@ package org.apache.hudi.keygen; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; - -import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; import java.util.Collections; @@ -46,11 +46,9 @@ public SimpleKeyGenerator(TypedProperties props) { SimpleKeyGenerator(TypedProperties props, String recordKeyField, String partitionPathField) { super(props); - this.recordKeyFields = recordKeyField == null - ? Collections.emptyList() : Collections.singletonList(recordKeyField); - this.partitionPathFields = partitionPathField == null - ? Collections.emptyList() : Collections.singletonList(partitionPathField); - simpleAvroKeyGenerator = new SimpleAvroKeyGenerator(props, recordKeyField, partitionPathField); + this.recordKeyFields = recordKeyField == null ? Collections.emptyList() : Collections.singletonList(recordKeyField); + this.partitionPathFields = partitionPathField == null ? Collections.emptyList() : Collections.singletonList(partitionPathField); + this.simpleAvroKeyGenerator = new SimpleAvroKeyGenerator(props, recordKeyField, partitionPathField); } @Override @@ -65,19 +63,43 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(Row row) { - buildFieldSchemaInfoIfNeeded(row.schema()); - return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, false); + tryInitRowAccessor(row.schema()); + + Object[] recordKeys = rowAccessor.getRecordKeyParts(row); + // NOTE: [[SimpleKeyGenerator]] is restricted to allow only primitive (non-composite) + // record-key field + if (recordKeys[0] == null) { + return handleNullRecordKey(null); + } else { + return requireNonNullNonEmptyKey(recordKeys[0].toString()); + } + } + + @Override + public UTF8String getRecordKey(InternalRow internalRow, StructType schema) { + tryInitRowAccessor(schema); + + Object[] recordKeyValues = rowAccessor.getRecordKeyParts(internalRow); + // NOTE: [[SimpleKeyGenerator]] is restricted to allow only primitive (non-composite) + // record-key field + if (recordKeyValues[0] == null) { + return handleNullRecordKey(null); + } else if (recordKeyValues[0] instanceof UTF8String) { + return requireNonNullNonEmptyKey((UTF8String) recordKeyValues[0]); + } else { + return requireNonNullNonEmptyKey(UTF8String.fromString(recordKeyValues[0].toString())); + } } @Override public String getPartitionPath(Row row) { - buildFieldSchemaInfoIfNeeded(row.schema()); - return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(), - hiveStylePartitioning, partitionPathSchemaInfo); + tryInitRowAccessor(row.schema()); + return combinePartitionPath(rowAccessor.getRecordPartitionPathValues(row)); } @Override - public String getPartitionPath(InternalRow row, StructType structType) { - return getPartitionPathInternal(row, structType); + public UTF8String getPartitionPath(InternalRow row, StructType schema) { + tryInitRowAccessor(schema); + return combinePartitionPathUnsafe(rowAccessor.getRecordPartitionPathValues(row)); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SparkKeyGeneratorInterface.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SparkKeyGeneratorInterface.java index bbceaf900b29e..977ff709bb1ab 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SparkKeyGeneratorInterface.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SparkKeyGeneratorInterface.java @@ -18,20 +18,65 @@ package org.apache.hudi.keygen; +import org.apache.hudi.ApiMaturityLevel; +import org.apache.hudi.PublicAPIMethod; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; /** - * Spark key generator interface. + * Spark-specific {@link KeyGenerator} interface extension allowing implementation to + * specifically implement record-key, partition-path generation w/o the need for (expensive) + * conversion from Spark internal representation (for ex, to Avro) */ public interface SparkKeyGeneratorInterface extends KeyGeneratorInterface { + /** + * Extracts record key from Spark's {@link Row} + * + * @param row instance of {@link Row} from which record-key is extracted + * @return record's (primary) key + */ + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) String getRecordKey(Row row); - String getRecordKey(InternalRow row, StructType schema); + /** + * Extracts record key from Spark's {@link InternalRow} + * + * NOTE: Difference b/w {@link Row} and {@link InternalRow} is that {@link InternalRow} could + * internally hold just a binary representation of the data, while {@link Row} has it + * deserialized into JVM-native representation (like {@code Integer}, {@code Long}, + * {@code String}, etc) + * + * @param row instance of {@link InternalRow} from which record-key is extracted + * @param schema schema {@link InternalRow} is adhering to + * @return record-key as instance of {@link UTF8String} + */ + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + UTF8String getRecordKey(InternalRow row, StructType schema); + /** + * Extracts partition-path from {@link Row} + * + * @param row instance of {@link Row} from which partition-path is extracted + * @return record's partition-path + */ + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) String getPartitionPath(Row row); - String getPartitionPath(InternalRow internalRow, StructType structType); + /** + * Extracts partition-path from Spark's {@link InternalRow} + * + * NOTE: Difference b/w {@link Row} and {@link InternalRow} is that {@link InternalRow} could + * internally hold just a binary representation of the data, while {@link Row} has it + * deserialized into JVM-native representation (like {@code Integer}, {@code Long}, + * {@code String}, etc) + * + * @param row instance of {@link InternalRow} from which record-key is extracted + * @param schema schema {@link InternalRow} is adhering to + * @return partition-path as instance of {@link UTF8String} + */ + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + UTF8String getPartitionPath(InternalRow row, StructType schema); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java index 004753f2461ae..f090320ccbcc3 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java @@ -18,20 +18,19 @@ package org.apache.hudi.keygen; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.exception.HoodieKeyGeneratorException; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; - -import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; 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.util.Objects; -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; /** * Key generator, that relies on timestamps for partitioning field. Still picks record key by name. @@ -61,39 +60,44 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(Row row) { - buildFieldSchemaInfoIfNeeded(row.schema()); - return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, false); + tryInitRowAccessor(row.schema()); + return combineRecordKey(rowAccessor.getRecordKeyParts(row)); + } + + @Override + public UTF8String getRecordKey(InternalRow internalRow, StructType schema) { + tryInitRowAccessor(schema); + return combineRecordKeyUnsafe(rowAccessor.getRecordKeyParts(internalRow)); } @Override public String getPartitionPath(Row row) { - buildFieldSchemaInfoIfNeeded(row.schema()); - Object partitionPathFieldVal = RowKeyGeneratorHelper.getNestedFieldVal(row, partitionPathSchemaInfo.get(getPartitionPathFields().get(0)).getKey()); - return getTimestampBasedPartitionPath(partitionPathFieldVal); + tryInitRowAccessor(row.schema()); + Object[] partitionPathValues = rowAccessor.getRecordPartitionPathValues(row); + return getFormattedPartitionPath(partitionPathValues[0]); } @Override - public String getPartitionPath(InternalRow internalRow, StructType structType) { - buildFieldSchemaInfoIfNeeded(structType); - validatePartitionFieldsForInternalRow(); - Object partitionPathFieldVal = RowKeyGeneratorHelper.getFieldValFromInternalRow(internalRow, - partitionPathSchemaInfo.get(getPartitionPathFields().get(0)).getKey().get(0), - partitionPathSchemaInfo.get(getPartitionPathFields().get(0)).getValue()); - return getTimestampBasedPartitionPath(partitionPathFieldVal); + public UTF8String getPartitionPath(InternalRow row, StructType schema) { + tryInitRowAccessor(schema); + Object[] partitionPathValues = rowAccessor.getRecordPartitionPathValues(row); + return UTF8String.fromString(getFormattedPartitionPath(partitionPathValues[0])); } - private String getTimestampBasedPartitionPath(Object partitionPathFieldVal) { - Object fieldVal = null; + private String getFormattedPartitionPath(Object partitionPathPart) { + Object fieldVal; + if (partitionPathPart == null || Objects.equals(partitionPathPart, HUDI_DEFAULT_PARTITION_PATH)) { + fieldVal = timestampBasedAvroKeyGenerator.getDefaultPartitionVal(); + } else if (partitionPathPart instanceof UTF8String) { + fieldVal = partitionPathPart.toString(); + } else { + fieldVal = partitionPathPart; + } + try { - if (partitionPathFieldVal == null || partitionPathFieldVal.toString().contains(HUDI_DEFAULT_PARTITION_PATH) || partitionPathFieldVal.toString().contains(NULL_RECORDKEY_PLACEHOLDER) - || partitionPathFieldVal.toString().contains(EMPTY_RECORDKEY_PLACEHOLDER)) { - fieldVal = timestampBasedAvroKeyGenerator.getDefaultPartitionVal(); - } else { - fieldVal = partitionPathFieldVal; - } return timestampBasedAvroKeyGenerator.getPartitionPath(fieldVal); } catch (Exception e) { - throw new HoodieKeyGeneratorException("Unable to parse input partition field :" + fieldVal, e); + throw new HoodieKeyGeneratorException(String.format("Failed to properly format partition-path (%s)", fieldVal), e); } } } \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/BaseBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/BaseBootstrapMetadataHandler.java index 45a0e91335321..0bc15fa2106a5 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/BaseBootstrapMetadataHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/BaseBootstrapMetadataHandler.java @@ -18,6 +18,8 @@ package org.apache.hudi.table.action.bootstrap; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieFileStatus; import org.apache.hudi.client.bootstrap.BootstrapWriteStatus; @@ -30,14 +32,13 @@ import org.apache.hudi.io.HoodieBootstrapHandle; import org.apache.hudi.keygen.KeyGeneratorInterface; import org.apache.hudi.table.HoodieTable; - -import org.apache.avro.Schema; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.avro.AvroReadSupport; import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; public abstract class BaseBootstrapMetadataHandler implements BootstrapMetadataHandler { private static final Logger LOG = LogManager.getLogger(ParquetBootstrapMetadataHandler.class); @@ -57,8 +58,10 @@ public BootstrapWriteStatus runMetadataBootstrap(String srcPartitionPath, String table, partitionPath, FSUtils.createNewFileIdPfx(), table.getTaskContextSupplier()); try { Schema avroSchema = getAvroSchema(sourceFilePath); - Schema recordKeySchema = HoodieAvroUtils.generateProjectionSchema(avroSchema, - keyGenerator.getRecordKeyFieldNames()); + List recordKeyColumns = keyGenerator.getRecordKeyFieldNames().stream() + .map(HoodieAvroUtils::getRootLevelFieldName) + .collect(Collectors.toList()); + Schema recordKeySchema = HoodieAvroUtils.generateProjectionSchema(avroSchema, recordKeyColumns); LOG.info("Schema to be used for reading record Keys :" + recordKeySchema); AvroReadSupport.setAvroReadSchema(table.getHadoopConf(), recordKeySchema); AvroReadSupport.setRequestedProjection(table.getHadoopConf(), recordKeySchema); diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/unsafe/UTF8StringBuilder.java b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/unsafe/UTF8StringBuilder.java new file mode 100644 index 0000000000000..3d9f060515a6d --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/unsafe/UTF8StringBuilder.java @@ -0,0 +1,100 @@ +/* + * 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.unsafe; + +import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.array.ByteArrayMethods; +import org.apache.spark.unsafe.types.UTF8String; + +/** + * A helper class to write {@link UTF8String}s to an internal buffer and build the concatenated + * {@link UTF8String} at the end. + */ +public class UTF8StringBuilder { + + private static final int ARRAY_MAX = ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH; + + private byte[] buffer; + private int cursor = Platform.BYTE_ARRAY_OFFSET; + + public UTF8StringBuilder() { + // Since initial buffer size is 16 in `StringBuilder`, we set the same size here + this(16); + } + + public UTF8StringBuilder(int initialSize) { + if (initialSize < 0) { + throw new IllegalArgumentException("Size must be non-negative"); + } + if (initialSize > ARRAY_MAX) { + throw new IllegalArgumentException( + "Size " + initialSize + " exceeded maximum size of " + ARRAY_MAX); + } + this.buffer = new byte[initialSize]; + } + + // Grows the buffer by at least `neededSize` + private void grow(int neededSize) { + if (neededSize > ARRAY_MAX - totalSize()) { + throw new UnsupportedOperationException( + "Cannot grow internal buffer by size " + neededSize + " because the size after growing " + + "exceeds size limitation " + ARRAY_MAX); + } + final int length = totalSize() + neededSize; + if (buffer.length < length) { + int newLength = length < ARRAY_MAX / 2 ? length * 2 : ARRAY_MAX; + final byte[] tmp = new byte[newLength]; + Platform.copyMemory( + buffer, + Platform.BYTE_ARRAY_OFFSET, + tmp, + Platform.BYTE_ARRAY_OFFSET, + totalSize()); + buffer = tmp; + } + } + + private int totalSize() { + return cursor - Platform.BYTE_ARRAY_OFFSET; + } + + public void append(UTF8String value) { + grow(value.numBytes()); + value.writeToMemory(buffer, cursor); + cursor += value.numBytes(); + } + + public void append(String value) { + append(UTF8String.fromString(value)); + } + + public void appendBytes(Object base, long offset, int length) { + grow(length); + Platform.copyMemory( + base, + offset, + buffer, + cursor, + length); + cursor += length; + } + + public UTF8String build() { + return UTF8String.fromBytes(buffer, 0, totalSize()); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeRowUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeRowUtils.scala index 10d6a2276eafe..c105142de0f45 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeRowUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeRowUtils.scala @@ -29,16 +29,16 @@ 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 = { + def getNestedRowValue(row: Row, nestedFieldPath: NestedFieldPath): Any = { var curRow = row - for (idx <- nestedFieldPath.indices) { - val (ord, f) = nestedFieldPath(idx) + for (idx <- nestedFieldPath.parts.indices) { + val (ord, f) = nestedFieldPath.parts(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) { + } else if (idx == nestedFieldPath.parts.length - 1) { // scalastyle:off return return curRow.get(ord) // scalastyle:on return @@ -57,21 +57,21 @@ object HoodieUnsafeRowUtils { * 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) { + def getNestedInternalRowValue(row: InternalRow, nestedFieldPath: NestedFieldPath): Any = { + if (nestedFieldPath.parts.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) + while (idx < nestedFieldPath.parts.length) { + val (ord, f) = nestedFieldPath.parts(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) { + } else if (idx == nestedFieldPath.parts.length - 1) { // scalastyle:off return return curRow.get(ord, f.dataType) // scalastyle:on return @@ -93,7 +93,7 @@ object HoodieUnsafeRowUtils { * * This method produces nested-field path, that is subsequently used by [[getNestedInternalRowValue]], [[getNestedRowValue]] */ - def composeNestedFieldPath(schema: StructType, nestedFieldRef: String): Array[(Int, StructField)] = { + def composeNestedFieldPath(schema: StructType, nestedFieldRef: String): NestedFieldPath = { val fieldRefParts = nestedFieldRef.split('.') val ordSeq = ArrayBuffer[(Int, StructField)]() var curSchema = schema @@ -115,6 +115,8 @@ object HoodieUnsafeRowUtils { idx += 1 } - ordSeq.toArray + NestedFieldPath(ordSeq.toArray) } + + case class NestedFieldPath(parts: Array[(Int, StructField)]) } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java index d6c060c6bd3d3..481cda00d6e72 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.storage.HoodieParquetConfig; @@ -115,6 +116,6 @@ private HoodieRowParquetWriteSupport getWriteSupport(HoodieWriteConfig.Builder w writeConfig.getBloomFilterFPP(), writeConfig.getDynamicBloomFilterMaxNumEntries(), writeConfig.getBloomFilterType()); - return new HoodieRowParquetWriteSupport(hadoopConf, SparkDatasetTestUtils.STRUCT_TYPE, filter, writeConfig); + return new HoodieRowParquetWriteSupport(hadoopConf, SparkDatasetTestUtils.STRUCT_TYPE, Option.of(filter), writeConfig); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java index f6e65adad757f..ad73a256a6175 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java @@ -80,9 +80,11 @@ public void tearDown() throws Exception { @ValueSource(booleans = { true, false }) public void testRowCreateHandle(boolean populateMetaFields) throws Exception { // init config and table - HoodieWriteConfig cfg = - SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort).build(); - HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + HoodieWriteConfig config = SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort) + .withPopulateMetaFields(populateMetaFields) + .build(); + + HoodieTable table = HoodieSparkTable.create(config, context, metaClient); List fileNames = new ArrayList<>(); List fileAbsPaths = new ArrayList<>(); @@ -95,8 +97,8 @@ public void testRowCreateHandle(boolean populateMetaFields) throws Exception { 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, populateMetaFields); + HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, config, partitionPath, fileId, instantTime, + RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE); int size = 10 + RANDOM.nextInt(1000); // Generate inputs Dataset inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, size, partitionPath, false); @@ -133,7 +135,7 @@ public void testGlobalFailure() throws Exception { String instantTime = "000"; HoodieRowCreateHandle handle = - new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE, true); + new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE); int size = 10 + RANDOM.nextInt(1000); int totalFailures = 5; // Generate first batch of valid rows @@ -186,7 +188,7 @@ public void testInstantiationFailure(boolean enableMetadataTable) { 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, true); + new HoodieRowCreateHandle(table, cfg, " def", UUID.randomUUID().toString(), "001", RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE); fail("Should have thrown exception"); } catch (HoodieInsertException ioe) { // expected without metadata table diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java index fd8ece1e06b85..918462ac0a08f 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java @@ -179,6 +179,7 @@ public static InternalRow getInternalRowWithError(String partitionPath) { public static HoodieWriteConfig.Builder getConfigBuilder(String basePath, int timelineServicePort) { return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withPopulateMetaFields(true) .withParallelism(2, 2) .withDeleteParallelism(2) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) diff --git a/hudi-client/hudi-spark-client/src/test/scala/org/apache/hudi/keygen/TestRowGeneratorHelper.scala b/hudi-client/hudi-spark-client/src/test/scala/org/apache/hudi/keygen/TestRowGeneratorHelper.scala deleted file mode 100644 index cd55e381e2d7d..0000000000000 --- a/hudi-client/hudi-spark-client/src/test/scala/org/apache/hudi/keygen/TestRowGeneratorHelper.scala +++ /dev/null @@ -1,102 +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.keygen - -import java.sql.Timestamp -import org.apache.spark.sql.Row -import org.apache.hudi.keygen.RowKeyGeneratorHelper._ -import org.apache.spark.sql.types.{DataType, DataTypes} -import org.junit.jupiter.api.{Assertions, Test} - -import scala.collection.JavaConverters._ - -class TestRowGeneratorHelper { - - @Test - def testGetPartitionPathFromRow(): Unit = { - - /** single plain partition */ - val row1 = Row.fromSeq(Seq(1, "z3", 10.0, "20220108")) - val ptField1 = List("dt").asJava - val mapValue = org.apache.hudi.common.util.collection.Pair.of(List(new Integer(3)).asJava, DataTypes.LongType) - val ptPos1 = Map("dt" -> mapValue).asJava - - Assertions.assertEquals("20220108", - getPartitionPathFromRow(row1, ptField1, false, ptPos1)) - Assertions.assertEquals("dt=20220108", - getPartitionPathFromRow(row1, ptField1, true, ptPos1)) - - /** multiple plain partitions */ - val row2 = Row.fromSeq(Seq(1, "z3", 10.0, "2022", "01", "08")) - val ptField2 = List("year", "month", "day").asJava - val ptPos2 = Map("year" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(3)).asJava, DataTypes.StringType), - "month" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(4)).asJava, DataTypes.StringType), - "day" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(5)).asJava, DataTypes.StringType) - ).asJava - Assertions.assertEquals("2022/01/08", - getPartitionPathFromRow(row2, ptField2, false, ptPos2)) - Assertions.assertEquals("year=2022/month=01/day=08", - getPartitionPathFromRow(row2, ptField2, true, ptPos2)) - - /** multiple partitions which contains TimeStamp type or Instant type */ - val timestamp = Timestamp.valueOf("2020-01-08 10:00:00") - val instant = timestamp.toInstant - val ptField3 = List("event", "event_time").asJava - val ptPos3 = Map("event" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(3)).asJava, DataTypes.StringType), - "event_time" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(4)).asJava, DataTypes.TimestampType) - ).asJava - - // with timeStamp type - val row2_ts = Row.fromSeq(Seq(1, "z3", 10.0, "click", timestamp)) - Assertions.assertEquals("click/2020-01-08 10:00:00.0", - getPartitionPathFromRow(row2_ts, ptField3, false, ptPos3)) - Assertions.assertEquals("event=click/event_time=2020-01-08 10:00:00.0", - getPartitionPathFromRow(row2_ts, ptField3, true, ptPos3)) - - // with instant type - val row2_instant = Row.fromSeq(Seq(1, "z3", 10.0, "click", instant)) - Assertions.assertEquals("click/2020-01-08 10:00:00.0", - getPartitionPathFromRow(row2_instant, ptField3, false, ptPos3)) - Assertions.assertEquals("event=click/event_time=2020-01-08 10:00:00.0", - getPartitionPathFromRow(row2_instant, ptField3, true, ptPos3)) - - /** mixed case with plain and nested partitions */ - val nestedRow4 = Row.fromSeq(Seq(instant, "ad")) - val ptField4 = List("event_time").asJava - val ptPos4 = Map("event_time" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(3), new Integer(0)).asJava, DataTypes.TimestampType)).asJava - // with instant type - val row4 = Row.fromSeq(Seq(1, "z3", 10.0, nestedRow4, "click")) - Assertions.assertEquals("2020-01-08 10:00:00.0", - getPartitionPathFromRow(row4, ptField4, false, ptPos4)) - Assertions.assertEquals("event_time=2020-01-08 10:00:00.0", - getPartitionPathFromRow(row4, ptField4, true, ptPos4)) - - val nestedRow5 = Row.fromSeq(Seq(timestamp, "ad")) - val ptField5 = List("event", "event_time").asJava - val ptPos5 = Map( - "event_time" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(3), new Integer(0)).asJava, DataTypes.TimestampType), - "event" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(4)).asJava, DataTypes.StringType) - ).asJava - val row5 = Row.fromSeq(Seq(1, "z3", 10.0, nestedRow5, "click")) - Assertions.assertEquals("click/2020-01-08 10:00:00.0", - getPartitionPathFromRow(row5, ptField5, false, ptPos5)) - Assertions.assertEquals("event=click/event_time=2020-01-08 10:00:00.0", - getPartitionPathFromRow(row5, ptField5, true, ptPos5)) - } -} diff --git a/hudi-client/hudi-spark-client/src/test/scala/org/apache/spark/sql/TestHoodieUnsafeRowUtils.scala b/hudi-client/hudi-spark-client/src/test/scala/org/apache/spark/sql/TestHoodieUnsafeRowUtils.scala index b051a9b507b64..c23bbab99b4f0 100644 --- a/hudi-client/hudi-spark-client/src/test/scala/org/apache/spark/sql/TestHoodieUnsafeRowUtils.scala +++ b/hudi-client/hudi-spark-client/src/test/scala/org/apache/spark/sql/TestHoodieUnsafeRowUtils.scala @@ -41,7 +41,7 @@ class TestHoodieUnsafeRowUtils { assertEquals( Seq((1, schema(1)), (0, schema(1).dataType.asInstanceOf[StructType](0))), - composeNestedFieldPath(schema, "bar.baz").toSeq) + composeNestedFieldPath(schema, "bar.baz").parts.toSeq) assertThrows(classOf[IllegalArgumentException]) { () => composeNestedFieldPath(schema, "foo.baz") @@ -148,6 +148,7 @@ class TestHoodieUnsafeRowUtils { } } + // TODO rebase on ScalaAssertionSupport private def assertThrows[T <: Throwable](expectedExceptionClass: Class[T])(f: () => Unit): T = { try { f.apply() diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index fa65461bfdb0e..1daf638f2b3d1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java @@ -388,7 +388,7 @@ public static GenericRecord rewriteRecordWithMetadata(GenericRecord genericRecor copyOldValueOrSetDefault(genericRecord, newRecord, f); } // do not preserve FILENAME_METADATA_FIELD - newRecord.put(HoodieRecord.FILENAME_META_FIELD_POS, fileName); + newRecord.put(HoodieRecord.FILENAME_META_FIELD_ORD, fileName); if (!GenericData.get().validate(newSchema, newRecord)) { throw new SchemaCompatibilityException( "Unable to validate the rewritten record " + genericRecord + " against schema " + newSchema); @@ -400,7 +400,7 @@ public static GenericRecord rewriteRecordWithMetadata(GenericRecord genericRecor public static GenericRecord rewriteEvolutionRecordWithMetadata(GenericRecord genericRecord, Schema newSchema, String fileName) { GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(genericRecord, newSchema, new HashMap<>()); // do not preserve FILENAME_METADATA_FIELD - newRecord.put(HoodieRecord.FILENAME_META_FIELD_POS, fileName); + newRecord.put(HoodieRecord.FILENAME_META_FIELD_ORD, fileName); return newRecord; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java index e504b7b87dd9b..c7ef08a16264c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java @@ -57,9 +57,9 @@ public abstract class HoodieRecord implements Serializable { IntStream.range(0, HOODIE_META_COLUMNS.size()).mapToObj(idx -> Pair.of(HOODIE_META_COLUMNS.get(idx), idx)) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); - public static int RECORD_KEY_META_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(RECORD_KEY_METADATA_FIELD); - public static int PARTITION_PATH_META_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(PARTITION_PATH_METADATA_FIELD); - public static int FILENAME_META_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(FILENAME_METADATA_FIELD); + public static int RECORD_KEY_META_FIELD_ORD = HOODIE_META_COLUMNS_NAME_TO_POS.get(RECORD_KEY_METADATA_FIELD); + public static int PARTITION_PATH_META_FIELD_ORD = HOODIE_META_COLUMNS_NAME_TO_POS.get(PARTITION_PATH_METADATA_FIELD); + public static int FILENAME_META_FIELD_ORD = HOODIE_META_COLUMNS_NAME_TO_POS.get(FILENAME_METADATA_FIELD); /** * Identifies the record across the table. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index 1f52912d1cc00..6b64ec4897b7c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -594,11 +594,11 @@ public String getKeyGeneratorClassName() { } public String getHiveStylePartitioningEnable() { - return getString(HIVE_STYLE_PARTITIONING_ENABLE); + return getStringOrDefault(HIVE_STYLE_PARTITIONING_ENABLE); } public String getUrlEncodePartitioning() { - return getString(URL_ENCODE_PARTITIONING); + return getStringOrDefault(URL_ENCODE_PARTITIONING); } public Boolean shouldDropPartitionColumns() { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 9945eb0650feb..529b0e8c99edb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -402,13 +402,13 @@ public void validateTableProperties(Properties properties) { throw new HoodieException(HoodieTableConfig.POPULATE_META_FIELDS.key() + " already disabled for the table. Can't be re-enabled back"); } - // meta fields can be disabled only with SimpleKeyGenerator, NonPartitioned and ComplexKeyGen. + // Meta fields can be disabled only when either {@code SimpleKeyGenerator}, {@code ComplexKeyGenerator}, {@code NonpartitionedKeyGenerator} is used if (!getTableConfig().populateMetaFields()) { String keyGenClass = properties.getProperty(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(), "org.apache.hudi.keygen.SimpleKeyGenerator"); - if (!keyGenClass.equals("org.apache.hudi.keygen.SimpleKeyGenerator") && !keyGenClass.equals("org.apache.hudi.keygen.NonpartitionedKeyGenerator") + if (!keyGenClass.equals("org.apache.hudi.keygen.SimpleKeyGenerator") + && !keyGenClass.equals("org.apache.hudi.keygen.NonpartitionedKeyGenerator") && !keyGenClass.equals("org.apache.hudi.keygen.ComplexKeyGenerator")) { - throw new HoodieException("Only simple, non partitioned and complex key generator is supported when meta fields are disabled. KeyGenerator used : " - + properties.getProperty(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key())); + throw new HoodieException("Only simple, non-partitioned or complex key generator are supported when meta-fields are disabled. Used: " + keyGenClass); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java index bb1ef72beae3d..8036995fab567 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java @@ -40,6 +40,8 @@ import java.util.stream.Stream; import java.util.stream.StreamSupport; +import static org.apache.hudi.common.util.ValidationUtils.checkArgument; + public class CollectionUtils { public static final Properties EMPTY_PROPERTIES = new Properties(); @@ -52,6 +54,14 @@ public static boolean nonEmpty(Collection c) { return !isNullOrEmpty(c); } + /** + * Returns last element of the array of {@code T} + */ + public static T tail(T[] ts) { + checkArgument(ts.length > 0); + return ts[ts.length - 1]; + } + /** * Collects provided {@link Iterator} to a {@link Stream} */ @@ -143,7 +153,7 @@ public static List diff(List one, List another) { } public static Stream> batchesAsStream(List list, int batchSize) { - ValidationUtils.checkArgument(batchSize > 0, "batch size must be positive."); + checkArgument(batchSize > 0, "batch size must be positive."); int total = list.size(); if (total <= 0) { return Stream.empty(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieTimer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieTimer.java index ce14f6c91c711..a0a8ca0867e93 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieTimer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieTimer.java @@ -32,11 +32,15 @@ public class HoodieTimer { // Ordered stack of TimeInfo's to make sure stopping the timer returns the correct elapsed time private final Deque timeInfoDeque = new ArrayDeque<>(); + /** + * @deprecated please use either {@link HoodieTimer#start} or {@link HoodieTimer#create} APIs + */ + @Deprecated public HoodieTimer() { this(false); } - public HoodieTimer(boolean shouldStart) { + private HoodieTimer(boolean shouldStart) { if (shouldStart) { startTimer(); } @@ -79,4 +83,12 @@ public long endTimer() { } return timeInfoDeque.pop().stop(); } + + public static HoodieTimer start() { + return new HoodieTimer(true); + } + + public static HoodieTimer create() { + return new HoodieTimer(false); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java index c779a3269a12a..ddd28fc4ea10a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java @@ -168,7 +168,7 @@ public ClosableIterator getHoodieKeyIterator(Configuration configurat conf.addResource(FSUtils.getFs(filePath.toString(), conf).getConf()); Schema readSchema = keyGeneratorOpt.map(keyGenerator -> { List fields = new ArrayList<>(); - fields.addAll(keyGenerator.getRecordKeyFields()); + fields.addAll(keyGenerator.getRecordKeyFieldNames()); fields.addAll(keyGenerator.getPartitionPathFields()); return HoodieAvroUtils.getSchemaForFields(readAvroSchema(conf, filePath), fields); }) diff --git a/hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java b/hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java index de4e0c3ccbb88..a09101dedfbeb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java +++ b/hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java @@ -24,7 +24,6 @@ import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import java.util.List; -import java.util.stream.Collectors; public abstract class BaseKeyGenerator extends KeyGenerator { @@ -59,23 +58,14 @@ protected BaseKeyGenerator(TypedProperties config) { */ @Override public final HoodieKey getKey(GenericRecord record) { - if (getRecordKeyFields() == null || getPartitionPathFields() == null) { + if (getRecordKeyFieldNames() == null || getPartitionPathFields() == null) { throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg"); } return new HoodieKey(getRecordKey(record), getPartitionPath(record)); } @Override - public final List getRecordKeyFieldNames() { - // For nested columns, pick top level column name - // TODO materialize - return getRecordKeyFields().stream().map(k -> { - int idx = k.indexOf('.'); - return idx > 0 ? k.substring(0, idx) : k; - }).collect(Collectors.toList()); - } - - public List getRecordKeyFields() { + public List getRecordKeyFieldNames() { return recordKeyFields; } diff --git a/hudi-common/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-common/src/main/java/org/apache/hudi/keygen/KeyGenerator.java index 8c3f794ee6fa8..691b1f4d5560c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/keygen/KeyGenerator.java +++ b/hudi-common/src/main/java/org/apache/hudi/keygen/KeyGenerator.java @@ -34,7 +34,7 @@ @PublicAPIClass(maturity = ApiMaturityLevel.STABLE) public abstract class KeyGenerator implements KeyGeneratorInterface { - protected TypedProperties config; + protected final TypedProperties config; protected KeyGenerator(TypedProperties config) { this.config = config; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java index 7bef8477125c2..d5b769190c2a5 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java @@ -259,7 +259,7 @@ public String getPartitionPath(GenericRecord record) { } @Override - public List getRecordKeyFields() { + public List getRecordKeyFieldNames() { return Arrays.asList(new String[]{recordKeyField}); } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java index 20e12e9030854..e643d9f9d0f97 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java @@ -83,7 +83,7 @@ public DeltaGenerator(DFSDeltaConfig deltaOutputConfig, JavaSparkContext jsc, Sp this.jsc = jsc; this.sparkSession = sparkSession; this.schemaStr = schemaStr; - this.recordRowKeyFieldNames = keyGenerator.getRecordKeyFields(); + this.recordRowKeyFieldNames = keyGenerator.getRecordKeyFieldNames(); this.partitionPathFieldNames = keyGenerator.getPartitionPathFields(); } diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java index db87f5dce0087..8115d50a78c12 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java @@ -235,9 +235,8 @@ private TestExecStartResultCallback executeCommandInDocker(String containerName, } int exitCode = dockerClient.inspectExecCmd(createCmdResponse.getId()).exec().getExitCode(); LOG.info("Exit code for command : " + exitCode); - if (exitCode != 0) { - LOG.error("\n\n ###### Stdout #######\n" + callback.getStdout().toString()); - } + + LOG.error("\n\n ###### Stdout #######\n" + callback.getStdout().toString()); LOG.error("\n\n ###### Stderr #######\n" + callback.getStderr().toString()); if (checkIfSucceed) { diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java index cc37de2f2926a..d62c9a768c4d8 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java @@ -18,6 +18,8 @@ package org.apache.hudi.connect.utils; +import com.google.protobuf.ByteString; +import org.apache.hadoop.conf.Configuration; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -37,9 +39,6 @@ import org.apache.hudi.keygen.CustomKeyGenerator; import org.apache.hudi.keygen.KeyGenerator; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; - -import com.google.protobuf.ByteString; -import org.apache.hadoop.conf.Configuration; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.DescribeTopicsResult; import org.apache.kafka.clients.admin.TopicDescription; @@ -49,14 +48,14 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.nio.file.Files; import java.nio.file.FileVisitOption; +import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; -import java.util.Arrays; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java index 516c6c5fc7972..f7918cf3fd9f1 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.PartitionPathEncodeUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.storage.row.HoodieRowCreateHandle; @@ -44,6 +43,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Properties; import java.util.UUID; @@ -74,7 +74,7 @@ public class BulkInsertDataInternalWriterHelper { * NOTE: This is stored as Catalyst's internal {@link UTF8String} to avoid * conversion (deserialization) b/w {@link UTF8String} and {@link String} */ - private String lastKnownPartitionPath = null; + private UTF8String lastKnownPartitionPath = null; private HoodieRowCreateHandle handle; private int numFilesWritten = 0; @@ -133,11 +133,13 @@ private Option getKeyGenerator(Properties properties) { public void write(InternalRow row) throws IOException { try { - String partitionPath = extractPartitionPath(row); - if (lastKnownPartitionPath == null || !lastKnownPartitionPath.equals(partitionPath) || !handle.canWrite()) { + UTF8String partitionPath = extractPartitionPath(row); + if (lastKnownPartitionPath == null || !Objects.equals(lastKnownPartitionPath, partitionPath) || !handle.canWrite()) { LOG.info("Creating new file for partition path " + partitionPath); - handle = getRowCreateHandle(partitionPath); - lastKnownPartitionPath = partitionPath; + handle = getRowCreateHandle(partitionPath.toString()); + // NOTE: It's crucial to make a copy here, since [[UTF8String]] could be pointing into + // a mutable underlying buffer + lastKnownPartitionPath = partitionPath.clone(); } handle.write(row); @@ -162,31 +164,19 @@ public void close() throws IOException { handle = null; } - private String extractPartitionPath(InternalRow row) { - String partitionPath; + private UTF8String extractPartitionPath(InternalRow row) { if (populateMetaFields) { // In case meta-fields are materialized w/in the table itself, we can just simply extract // partition path from there // // NOTE: Helper keeps track of [[lastKnownPartitionPath]] as [[UTF8String]] to avoid // conversion from Catalyst internal representation into a [[String]] - partitionPath = row.getString(HoodieRecord.PARTITION_PATH_META_FIELD_POS); + return row.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_ORD); } else if (keyGeneratorOpt.isPresent()) { - // TODO(HUDI-4039) this should be handled by the SimpleKeyGenerator itself - if (simpleKeyGen) { - String partitionPathValue = row.get(simplePartitionFieldIndex, simplePartitionFieldDataType).toString(); - partitionPath = partitionPathValue != null ? partitionPathValue : PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH; - if (writeConfig.isHiveStylePartitioningEnabled()) { - partitionPath = (keyGeneratorOpt.get()).getPartitionPathFields().get(0) + "=" + partitionPath; - } - } else { - // only BuiltIn key generators are supported if meta fields are disabled. - partitionPath = keyGeneratorOpt.get().getPartitionPath(row, structType); - } + return keyGeneratorOpt.get().getPartitionPath(row, structType); } else { - partitionPath = ""; + return UTF8String.EMPTY_UTF8; } - return partitionPath; } private HoodieRowCreateHandle getRowCreateHandle(String partitionPath) throws IOException { @@ -209,7 +199,7 @@ private HoodieRowCreateHandle getRowCreateHandle(String partitionPath) throws IO private HoodieRowCreateHandle createHandle(String partitionPath) { return new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(), - instantTime, taskPartitionId, taskId, taskEpochId, structType, populateMetaFields); + instantTime, taskPartitionId, taskId, taskEpochId, structType); } private String getNextFileId() { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala index 168cc6b2653a0..dad7c1765020c 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala @@ -23,7 +23,7 @@ import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.util.ReflectionUtils import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.index.SparkHoodieIndexFactory -import org.apache.hudi.keygen.BuiltinKeyGenerator +import org.apache.hudi.keygen.{BuiltinKeyGenerator, SparkKeyGeneratorInterface} import org.apache.hudi.table.BulkInsertPartitioner import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -63,13 +63,12 @@ object HoodieDatasetBulkInsertHelper extends Logging { df.queryExecution.toRdd.mapPartitions { iter => val keyGenerator = ReflectionUtils.loadClass(keyGeneratorClassName, new TypedProperties(config.getProps)) - .asInstanceOf[BuiltinKeyGenerator] + .asInstanceOf[SparkKeyGeneratorInterface] iter.map { row => val (recordKey, partitionPath) = if (populateMetaFields) { - (UTF8String.fromString(keyGenerator.getRecordKey(row, schema)), - UTF8String.fromString(keyGenerator.getPartitionPath(row, schema))) + (keyGenerator.getRecordKey(row, schema), keyGenerator.getPartitionPath(row, schema)) } else { (UTF8String.EMPTY_UTF8, UTF8String.EMPTY_UTF8) } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java index 6719c2a3d6d23..caed61249a1d3 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java @@ -28,6 +28,7 @@ import org.apache.hudi.testutils.KeyGeneratorTestUtilities; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.unsafe.types.UTF8String; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -83,7 +84,6 @@ public void testNullRecordKeyFields() { public void testWrongRecordKeyField() { ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getWrongRecordKeyFieldProps()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldSchemaInfoIfNeeded(KeyGeneratorTestUtilities.structType)); } @Test @@ -98,7 +98,7 @@ public void testHappyFlow() { Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686/ts_ms=2020-03-21"); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686/ts_ms=2020-03-21"); + Assertions.assertEquals(UTF8String.fromString("timestamp=4357686/ts_ms=2020-03-21"), keyGenerator.getPartitionPath(internalRow, row.schema())); } @Test @@ -107,7 +107,7 @@ public void testSingleValueKeyGenerator() { properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key"); properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp"); ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(properties); - assertEquals(compositeKeyGenerator.getRecordKeyFields().size(), 1); + assertEquals(compositeKeyGenerator.getRecordKeyFieldNames().size(), 1); assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 1); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); GenericRecord record = dataGenerator.generateGenericRecords(1).get(0); @@ -119,9 +119,9 @@ public void testSingleValueKeyGenerator() { Row row = KeyGeneratorTestUtilities.getRow(record, HoodieTestDataGenerator.AVRO_SCHEMA, AvroConversionUtils.convertAvroSchemaToStructType(HoodieTestDataGenerator.AVRO_SCHEMA)); - Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(row), partitionPath); + Assertions.assertEquals(partitionPath, compositeKeyGenerator.getPartitionPath(row)); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(internalRow, row.schema()), partitionPath); + Assertions.assertEquals(UTF8String.fromString(partitionPath), compositeKeyGenerator.getPartitionPath(internalRow, row.schema())); } @Test @@ -130,7 +130,7 @@ public void testMultipleValueKeyGenerator() { properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key,timestamp"); properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "rider,driver"); ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(properties); - assertEquals(compositeKeyGenerator.getRecordKeyFields().size(), 2); + assertEquals(compositeKeyGenerator.getRecordKeyFieldNames().size(), 2); assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 2); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); GenericRecord record = dataGenerator.generateGenericRecords(1).get(0); @@ -144,10 +144,10 @@ public void testMultipleValueKeyGenerator() { Row row = KeyGeneratorTestUtilities.getRow(record, HoodieTestDataGenerator.AVRO_SCHEMA, AvroConversionUtils.convertAvroSchemaToStructType(HoodieTestDataGenerator.AVRO_SCHEMA)); - Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(row), partitionPath); + Assertions.assertEquals(partitionPath, compositeKeyGenerator.getPartitionPath(row)); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(internalRow, row.schema()), partitionPath); + Assertions.assertEquals(UTF8String.fromString(partitionPath), compositeKeyGenerator.getPartitionPath(internalRow, row.schema())); } @Test @@ -156,7 +156,7 @@ public void testMultipleValueKeyGeneratorNonPartitioned() { properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key,timestamp"); properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), ""); ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(properties); - assertEquals(compositeKeyGenerator.getRecordKeyFields().size(), 2); + assertEquals(compositeKeyGenerator.getRecordKeyFieldNames().size(), 2); assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 0); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); GenericRecord record = dataGenerator.generateGenericRecords(1).get(0); @@ -170,9 +170,9 @@ public void testMultipleValueKeyGeneratorNonPartitioned() { Row row = KeyGeneratorTestUtilities.getRow(record, HoodieTestDataGenerator.AVRO_SCHEMA, AvroConversionUtils.convertAvroSchemaToStructType(HoodieTestDataGenerator.AVRO_SCHEMA)); - Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(row), partitionPath); + Assertions.assertEquals(partitionPath, compositeKeyGenerator.getPartitionPath(row)); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(internalRow, row.schema()), partitionPath); + Assertions.assertEquals(UTF8String.fromString(partitionPath), compositeKeyGenerator.getPartitionPath(internalRow, row.schema())); } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java index 26a2b439abfb2..311356a0f710f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java @@ -28,6 +28,7 @@ import org.apache.hudi.testutils.KeyGeneratorTestUtilities; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.unsafe.types.UTF8String; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -145,13 +146,13 @@ public void testSimpleKeyGenerator(TypedProperties props) throws IOException { (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); GenericRecord record = getRecord(); HoodieKey key = keyGenerator.getKey(record); - Assertions.assertEquals(key.getRecordKey(), "key1"); - Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686"); + Assertions.assertEquals("key1", key.getRecordKey()); + Assertions.assertEquals("timestamp=4357686", key.getPartitionPath()); Row row = KeyGeneratorTestUtilities.getRow(record); - Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1"); - Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686"); + Assertions.assertEquals("key1", keyGenerator.getRecordKey(row)); + Assertions.assertEquals("timestamp=4357686", keyGenerator.getPartitionPath(row)); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686"); + Assertions.assertEquals(UTF8String.fromString("timestamp=4357686"), keyGenerator.getPartitionPath(internalRow, row.schema())); } @Test @@ -170,13 +171,13 @@ public void testTimestampBasedKeyGenerator(TypedProperties props) throws IOExcep GenericRecord record = getRecord(); HoodieKey key = keyGenerator.getKey(record); - Assertions.assertEquals(key.getRecordKey(), "key1"); - Assertions.assertEquals(key.getPartitionPath(), "ts_ms=20200321"); + Assertions.assertEquals("key1", key.getRecordKey()); + Assertions.assertEquals("ts_ms=20200321", key.getPartitionPath()); Row row = KeyGeneratorTestUtilities.getRow(record); - Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1"); - Assertions.assertEquals(keyGenerator.getPartitionPath(row), "ts_ms=20200321"); + Assertions.assertEquals("key1", keyGenerator.getRecordKey(row)); + Assertions.assertEquals("ts_ms=20200321", keyGenerator.getPartitionPath(row)); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "ts_ms=20200321"); + Assertions.assertEquals(UTF8String.fromString("ts_ms=20200321"), keyGenerator.getPartitionPath(internalRow, row.schema())); } @Test @@ -202,7 +203,7 @@ public void testNonPartitionedKeyGenerator(TypedProperties props) throws IOExcep Assertions.assertTrue(keyGenerator.getPartitionPath(row).isEmpty()); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertTrue(keyGenerator.getPartitionPath(internalRow, row.schema()).isEmpty()); + Assertions.assertEquals(0, keyGenerator.getPartitionPath(internalRow, row.schema()).numBytes()); } @Test @@ -345,15 +346,15 @@ public void testComplexRecordKeyWithSimplePartitionPath(TypedProperties props) t GenericRecord record = getRecord(); HoodieKey key = keyGenerator.getKey(record); - Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi"); - Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686"); + Assertions.assertEquals("_row_key:key1,pii_col:pi", key.getRecordKey()); + Assertions.assertEquals("timestamp=4357686", key.getPartitionPath()); Row row = KeyGeneratorTestUtilities.getRow(record); - Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi"); - Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686"); + Assertions.assertEquals("_row_key:key1,pii_col:pi", keyGenerator.getRecordKey(row)); + Assertions.assertEquals("timestamp=4357686", keyGenerator.getPartitionPath(row)); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686"); + Assertions.assertEquals(UTF8String.fromString("timestamp=4357686"), keyGenerator.getPartitionPath(internalRow, row.schema())); } @Test @@ -372,14 +373,14 @@ public void testComplexRecordKeysWithComplexPartitionPath(TypedProperties props) GenericRecord record = getRecord(); HoodieKey key = keyGenerator.getKey(record); - Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi"); - Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686/ts_ms=20200321"); + Assertions.assertEquals("_row_key:key1,pii_col:pi", key.getRecordKey()); + Assertions.assertEquals("timestamp=4357686/ts_ms=20200321", key.getPartitionPath()); Row row = KeyGeneratorTestUtilities.getRow(record); - Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi"); - Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686/ts_ms=20200321"); + Assertions.assertEquals("_row_key:key1,pii_col:pi", keyGenerator.getRecordKey(row)); + Assertions.assertEquals("timestamp=4357686/ts_ms=20200321", keyGenerator.getPartitionPath(row)); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686/ts_ms=20200321"); + Assertions.assertEquals(UTF8String.fromString("timestamp=4357686/ts_ms=20200321"), keyGenerator.getPartitionPath(internalRow, row.schema())); } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteRecordGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteRecordGenerator.java index f6c4c8a8b58cd..1b25ce650544a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteRecordGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteRecordGenerator.java @@ -68,7 +68,6 @@ public void testNullRecordKeyFields() { public void testWrongRecordKeyField() { GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getWrongRecordKeyFieldProps()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldSchemaInfoIfNeeded(KeyGeneratorTestUtilities.structType)); } @Test @@ -78,7 +77,6 @@ public void testHappyFlow() { HoodieKey key = keyGenerator.getKey(record); Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi"); Assertions.assertEquals(key.getPartitionPath(), ""); - keyGenerator.buildFieldSchemaInfoIfNeeded(KeyGeneratorTestUtilities.structType); Row row = KeyGeneratorTestUtilities.getRow(record); Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi"); Assertions.assertEquals(keyGenerator.getPartitionPath(row), ""); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java index 75d9b7da74bc8..fd299f179c36c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java @@ -94,7 +94,6 @@ public void testNullPartitionPathFields() { public void testWrongRecordKeyField() { NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(getWrongRecordKeyFieldProps()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldSchemaInfoIfNeeded(KeyGeneratorTestUtilities.structType)); } @Test @@ -103,7 +102,7 @@ public void testSingleValueKeyGeneratorNonPartitioned() { properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "timestamp"); properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), ""); NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties); - assertEquals(keyGenerator.getRecordKeyFields().size(), 1); + assertEquals(keyGenerator.getRecordKeyFieldNames().size(), 1); assertEquals(keyGenerator.getPartitionPathFields().size(), 0); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); @@ -120,7 +119,7 @@ public void testMultipleValueKeyGeneratorNonPartitioned1() { properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "timestamp,driver"); properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), ""); NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties); - assertEquals(keyGenerator.getRecordKeyFields().size(), 2); + assertEquals(keyGenerator.getRecordKeyFieldNames().size(), 2); assertEquals(keyGenerator.getPartitionPathFields().size(), 0); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); GenericRecord record = dataGenerator.generateGenericRecords(1).get(0); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java index 17cff3505ebef..32f372a2d7f99 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java @@ -21,11 +21,13 @@ import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieKeyException; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.testutils.KeyGeneratorTestUtilities; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.unsafe.types.UTF8String; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -34,6 +36,7 @@ import java.util.stream.Stream; import static org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH; +import static org.junit.jupiter.api.Assertions.assertThrows; public class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities { private TypedProperties getCommonProps() { @@ -88,35 +91,37 @@ private TypedProperties getPropsWithNestedPartitionPathField() { @Test public void testNullPartitionPathFields() { - Assertions.assertThrows(IllegalArgumentException.class, () -> new SimpleKeyGenerator(getPropertiesWithoutPartitionPathProp())); + assertThrows(IllegalArgumentException.class, () -> new SimpleKeyGenerator(getPropertiesWithoutPartitionPathProp())); } @Test public void testNullRecordKeyFields() { - Assertions.assertThrows(IllegalArgumentException.class, () -> new SimpleKeyGenerator(getPropertiesWithoutRecordKeyProp())); + assertThrows(IllegalArgumentException.class, () -> new SimpleKeyGenerator(getPropertiesWithoutRecordKeyProp())); } @Test public void testWrongRecordKeyField() { SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongRecordKeyFieldProps()); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldSchemaInfoIfNeeded(KeyGeneratorTestUtilities.structType)); + assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); } @Test public void testWrongPartitionPathField() { SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongPartitionPathFieldProps()); GenericRecord record = getRecord(); - Assertions.assertEquals(keyGenerator.getPartitionPath(record), KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH); - Assertions.assertEquals(keyGenerator.getPartitionPath(KeyGeneratorTestUtilities.getRow(record)), - KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH); + // TODO this should throw as well + //assertThrows(HoodieException.class, () -> { + // keyGenerator.getPartitionPath(record); + //}); + assertThrows(HoodieException.class, () -> { + keyGenerator.getPartitionPath(KeyGeneratorTestUtilities.getRow(record)); + }); } @Test public void testComplexRecordKeyField() { SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getComplexRecordKeyProp()); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldSchemaInfoIfNeeded(KeyGeneratorTestUtilities.structType)); + assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); } @Test @@ -124,15 +129,15 @@ public void testHappyFlow() { SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getProps()); GenericRecord record = getRecord(); HoodieKey key = keyGenerator.getKey(getRecord()); - Assertions.assertEquals(key.getRecordKey(), "key1"); - Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686"); + Assertions.assertEquals("key1", key.getRecordKey()); + Assertions.assertEquals("timestamp=4357686", key.getPartitionPath()); Row row = KeyGeneratorTestUtilities.getRow(record); - Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1"); - Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686"); + Assertions.assertEquals("key1", keyGenerator.getRecordKey(row)); + Assertions.assertEquals("timestamp=4357686", keyGenerator.getPartitionPath(row)); InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row); - Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686"); + Assertions.assertEquals(UTF8String.fromString("timestamp=4357686"), keyGenerator.getPartitionPath(internalRow, row.schema())); } private static Stream nestedColTestRecords() { diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java index 1fc4b9f1ef694..8cfd7b04507cf 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -138,7 +139,7 @@ public void testTimestampBasedKeyGenerator() throws IOException { baseRow = genericRecordToRow(baseRecord); assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow)); internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow); - assertEquals("2020-01-06 12", keyGen.getPartitionPath(internalRow, baseRow.schema())); + assertEquals(UTF8String.fromString("2020-01-06 12"), keyGen.getPartitionPath(internalRow, baseRow.schema())); // timezone is GMT+8:00, createTime is BigDecimal BigDecimal decimal = new BigDecimal("1578283932000.0001"); @@ -186,7 +187,7 @@ public void testTimestampBasedKeyGenerator() throws IOException { baseRow = genericRecordToRow(baseRecord); assertEquals("1970-01-01 08", keyGen.getPartitionPath(baseRow)); internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow); - assertEquals("1970-01-01 08", keyGen.getPartitionPath(internalRow, baseRow.schema())); + assertEquals(UTF8String.fromString("1970-01-01 08"), keyGen.getPartitionPath(internalRow, baseRow.schema())); // timestamp is DATE_STRING, timezone is GMT, createTime is null baseRecord.put("createTimeString", null); @@ -198,7 +199,7 @@ public void testTimestampBasedKeyGenerator() throws IOException { baseRow = genericRecordToRow(baseRecord); assertEquals("1970-01-01 12:00:00", keyGen.getPartitionPath(baseRow)); internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow); - assertEquals("1970-01-01 12:00:00", keyGen.getPartitionPath(internalRow, baseRow.schema())); + assertEquals(UTF8String.fromString("1970-01-01 12:00:00"), keyGen.getPartitionPath(internalRow, baseRow.schema())); } @Test @@ -216,7 +217,7 @@ public void testScalar() throws IOException { baseRow = genericRecordToRow(baseRecord); assertEquals("2024-10-04 12", keyGen.getPartitionPath(baseRow)); internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow); - assertEquals("2024-10-04 12", keyGen.getPartitionPath(internalRow, baseRow.schema())); + assertEquals(UTF8String.fromString("2024-10-04 12"), keyGen.getPartitionPath(internalRow, baseRow.schema())); // timezone is GMT, createTime is null baseRecord.put("createTime", null); @@ -229,7 +230,7 @@ public void testScalar() throws IOException { baseRow = genericRecordToRow(baseRecord); assertEquals("1970-01-02 12", keyGen.getPartitionPath(baseRow)); internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow); - assertEquals("1970-01-02 12", keyGen.getPartitionPath(internalRow, baseRow.schema())); + assertEquals(UTF8String.fromString("1970-01-02 12"), keyGen.getPartitionPath(internalRow, baseRow.schema())); // timezone is GMT. number of days store integer in mysql baseRecord.put("createTime", 18736L); @@ -260,7 +261,7 @@ public void testScalarWithLogicalType() throws IOException { baseRow = genericRecordToRow(baseRecord); assertEquals("2021/12/03", keyGen.getPartitionPath(baseRow)); internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow); - assertEquals("2021/12/03", keyGen.getPartitionPath(internalRow, baseRow.schema())); + assertEquals(UTF8String.fromString("2021/12/03"), keyGen.getPartitionPath(internalRow, baseRow.schema())); // timezone is GMT, createTime is null baseRecord.put("createTime", null); @@ -274,7 +275,7 @@ public void testScalarWithLogicalType() throws IOException { baseRow = genericRecordToRow(baseRecord); assertEquals("1970/01/01", keyGen.getPartitionPath(baseRow)); internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow); - assertEquals("1970/01/01", keyGen.getPartitionPath(internalRow, baseRow.schema())); + assertEquals(UTF8String.fromString("1970/01/01"), keyGen.getPartitionPath(internalRow, baseRow.schema())); } @Test diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/KeyGeneratorTestUtilities.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/KeyGeneratorTestUtilities.java index c2256f40c6b98..e1f8f9f6105ec 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/KeyGeneratorTestUtilities.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/KeyGeneratorTestUtilities.java @@ -43,7 +43,7 @@ public class KeyGeneratorTestUtilities { public static final String NESTED_COL_SCHEMA = "{\"type\":\"record\", \"name\":\"nested_col\",\"fields\": [" - + "{\"name\": \"prop1\",\"type\": \"string\"},{\"name\": \"prop2\", \"type\": \"long\"}]}"; + + "{\"name\": \"prop1\",\"type\": [\"null\", \"string\"]},{\"name\": \"prop2\", \"type\": \"long\"}]}"; public static final String EXAMPLE_SCHEMA = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ " + "{\"name\": \"timestamp\",\"type\": \"long\"},{\"name\": \"_row_key\", \"type\": \"string\"}," + "{\"name\": \"ts_ms\", \"type\": \"string\"}," diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/ScalaAssertionSupport.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/ScalaAssertionSupport.scala new file mode 100644 index 0000000000000..c26adb4bc019a --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/ScalaAssertionSupport.scala @@ -0,0 +1,39 @@ +/* + * 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 + +import org.junit.jupiter.api.Assertions.fail + +class ScalaAssertionSupport { + + def assertThrows[T <: Throwable, R](expectedExceptionClass: Class[T])(f: => R): T = { + try { + f + } 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") + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala index 968b4039f87c1..50f7f587937fd 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala @@ -31,19 +31,20 @@ import org.apache.hudi.testutils.KeyGeneratorTestUtilities import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types.StructType +import org.apache.spark.unsafe.types.UTF8String import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{BeforeEach, Test} -import org.scalatest.Assertions.fail /** * Tests on the default key generator, payload classes. */ -class TestDataSourceDefaults { +class TestDataSourceDefaults extends ScalaAssertionSupport { val schema = SchemaTestUtil.getComplexEvolvedSchema val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) var baseRecord: GenericRecord = _ var baseRow: Row = _ + var internalRow: InternalRow = _ val testStructName = "testStructName" val testNamespace = "testNamespace" @@ -51,6 +52,7 @@ class TestDataSourceDefaults { baseRecord = SchemaTestUtil .generateAvroRecordFromJson(schema, 1, "001", "f1") baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) + internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) } private def getKeyConfig(recordKeyFieldName: String, partitionPathField: String, hiveStylePartitioning: String): TypedProperties = { @@ -61,174 +63,136 @@ class TestDataSourceDefaults { props } - @Test def testSimpleKeyGenerator() = { + @Test def testSimpleKeyGenerator(): Unit = { - // top level, valid fields - var keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")) - val hk1 = keyGen.getKey(baseRecord) - assertEquals("field1", hk1.getRecordKey) - assertEquals("name1", hk1.getPartitionPath) + { + // Top level, valid fields + val keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")) - assertEquals("field1", keyGen.getRecordKey(baseRow)) - assertEquals("name1", keyGen.getPartitionPath(baseRow)) + val expectedKey = new HoodieKey("field1", "name1") + assertEquals(expectedKey, keyGen.getKey(baseRecord)) - // partition path field not specified - try { - val props = new TypedProperties() - props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1") - new SimpleKeyGenerator(props).getKey(baseRecord) - fail("Should have errored out") - } catch { - case e: IllegalArgumentException => - // do nothing + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) } - // partition path field not specified using Row - try { + { + // Partition path field not specified val props = new TypedProperties() props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1") - val keyGen = new SimpleKeyGenerator(props) - keyGen.getRecordKey(baseRow) - fail("Should have errored out") - } catch { - case e: IllegalArgumentException => - // do nothing + + assertThrows(classOf[IllegalArgumentException]) { + new SimpleKeyGenerator(props) + } } - // recordkey field not specified - try { + { + // Record's key field not specified val props = new TypedProperties() props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), "partitionField") - new SimpleKeyGenerator(props).getKey(baseRecord) - fail("Should have errored out") - } catch { - case e: IllegalArgumentException => - // do nothing - } - // recordkey field not specified using Row - try { - val props = new TypedProperties() - props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionField") - val keyGen = new SimpleKeyGenerator(props) - keyGen.getPartitionPath(baseRow) - fail("Should have errored out") - } catch { - case e: IllegalArgumentException => - // do nothing + assertThrows(classOf[IllegalArgumentException]) { + new SimpleKeyGenerator(props) + } } - // nested field as record key and partition path - val hk2 = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.isAdmin", "false")) - .getKey(baseRecord) - assertEquals("UserId1@001", hk2.getRecordKey) - assertEquals("false", hk2.getPartitionPath) - - // Nested record key not found - try { - new SimpleKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false")) - .getKey(baseRecord) - fail("Should have errored out") - } catch { - case e: HoodieException => - // do nothing + { + // nested field as record key and partition path + val keyGen = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.isAdmin", "false")) + + assertEquals(new HoodieKey("UserId1@001", "false"), keyGen.getKey(baseRecord)) } - // if partition path can't be found, return default partition path - val hk3 = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false")) - .getKey(baseRecord) - assertEquals("default", hk3.getPartitionPath) - - // if partition path can't be found, return default partition path using row - keyGen = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false")) - val hk3_row = keyGen.getPartitionPath(baseRow) - assertEquals("default", hk3_row) - - // if enable hive style partitioning - val hk4 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "true")).getKey(baseRecord) - assertEquals("name=name1", hk4.getPartitionPath) - - // if enable hive style partitioning using row - keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "true")) - val hk4_row = keyGen.getPartitionPath(baseRow) - assertEquals("name=name1", hk4_row) - - // if partition is null, return default partition path - baseRecord.put("name", "") - val hk5 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")) - .getKey(baseRecord) - assertEquals("default", hk5.getPartitionPath) - - // if partition is null, return default partition path using Row - keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")) - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - val hk5_row = keyGen.getPartitionPath(baseRow) - assertEquals("default", hk5_row) + { + // Nested record key not found + val keyGen = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false")) - // if partition is empty, return default partition path - baseRecord.put("name", null) - val hk6 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")) - .getKey(baseRecord) - assertEquals("default", hk6.getPartitionPath) + assertThrows(classOf[HoodieException]) { + keyGen.getKey(baseRecord) + } + } - // if partition is empty, return default partition path using Row - keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")) - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - val hk6_row = keyGen.getPartitionPath(baseRow) - assertEquals("default", hk6_row) + { + // Fail in case partition path can't be found in schema + val keyGen = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false")) + + // TODO this should throw + //assertThrows(classOf[HoodieException]) { + // keyGen.getKey(baseRecord) + //} + assertThrows(classOf[HoodieException]) { + keyGen.getPartitionPath(baseRow) + } + assertThrows(classOf[HoodieException]) { + keyGen.getPartitionPath(internalRow, structType) + } + } - // if record key is empty, throw error - try { - baseRecord.put("field1", "") - val props = new TypedProperties() - props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1") - props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "name") - new SimpleKeyGenerator(props).getKey(baseRecord) - fail("Should have errored out") - } catch { - case e: HoodieKeyException => - // do nothing + { + val keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "true")) + + assertEquals("name=name1", keyGen.getKey(baseRecord).getPartitionPath) + assertEquals("name=name1", keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString("name=name1"), keyGen.getPartitionPath(internalRow, structType)) } - // if record key is empty, throw error. Using Row - try { - val props = new TypedProperties() - props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1") - props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "name") - keyGen = new SimpleKeyGenerator(props) + { + // If partition is null/empty, return default partition path + val keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")) + + baseRecord.put("name", "") baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - keyGen.getRecordKey(baseRow) - fail("Should have errored out") - } catch { - case e: HoodieKeyException => - // do nothing - } + internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) - // if record key is null, throw error - try { - baseRecord.put("field1", null) - val props = new TypedProperties() - props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1") - props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "name") - new SimpleKeyGenerator(props).getKey(baseRecord) - fail("Should have errored out") - } catch { - case e: HoodieKeyException => - // do nothing + assertEquals("default", keyGen.getKey(baseRecord).getPartitionPath) + assertEquals("default", keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString("default"), keyGen.getPartitionPath(internalRow, structType)) + + baseRecord.put("name", null) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) + internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + + assertEquals("default", keyGen.getKey(baseRecord).getPartitionPath) + assertEquals("default", keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString("default"), keyGen.getPartitionPath(internalRow, structType)) } - // if record key is null, throw error. Using Row - try { + { + // If record key is null/empty, throw error val props = new TypedProperties() props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1") props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "name") - keyGen = new SimpleKeyGenerator(props) + val keyGen = new SimpleKeyGenerator(props) + + baseRecord.put("field1", "") baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - keyGen.getRecordKey(baseRow) - fail("Should have errored out") - } catch { - case e: HoodieKeyException => - // do nothing + internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + + assertThrows(classOf[HoodieKeyException]) { + keyGen.getKey(baseRecord) + } + assertThrows(classOf[HoodieKeyException]) { + keyGen.getRecordKey(baseRow) + } + assertThrows(classOf[HoodieKeyException]) { + keyGen.getRecordKey(internalRow, structType) + } + + baseRecord.put("field1", null) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) + internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + + assertThrows(classOf[HoodieKeyException]) { + keyGen.getKey(baseRecord) + } + assertThrows(classOf[HoodieKeyException]) { + keyGen.getRecordKey(baseRow) + } + assertThrows(classOf[HoodieKeyException]) { + keyGen.getRecordKey(internalRow, structType) + } } } @@ -256,7 +220,7 @@ class TestDataSourceDefaults { getKey(genericRecord).getRecordKey } - override def getRecordKey(row: InternalRow, schema: StructType): String = null + override def getRecordKey(row: InternalRow, schema: StructType): UTF8String = null override def getPartitionPath(row: Row): String = { if (null == converterFn) converterFn = AvroConversionUtils.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE) @@ -264,301 +228,312 @@ class TestDataSourceDefaults { getKey(genericRecord).getPartitionPath } - override def getPartitionPath(internalRow: InternalRow, structType: StructType): String = null + override def getPartitionPath(internalRow: InternalRow, structType: StructType): UTF8String = null } - @Test def testComplexKeyGenerator() = { - // top level, valid fields - var keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) - val hk1 = keyGen.getKey(baseRecord) - assertEquals("field1:field1,name:name1", hk1.getRecordKey) - assertEquals("field1/name1", hk1.getPartitionPath) + @Test def testComplexKeyGenerator(): Unit = { - // top level, valid fields with Row - assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow)) - assertEquals("field1/name1", keyGen.getPartitionPath(baseRow)) + { + // Top level, valid fields + val keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) + val expectedKey = new HoodieKey("field1:field1,name:name1", "field1/name1") - // partition path field not specified - try { - val props = new TypedProperties() - props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1") - new ComplexKeyGenerator(props).getKey(baseRecord) - fail("Should have errored out") - } catch { - case e: IllegalArgumentException => - // do nothing + assertEquals(expectedKey, keyGen.getKey(baseRecord)) + + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) } - // partition path field not specified using Row - try { + // Partition path field not specified + assertThrows(classOf[IllegalArgumentException]) { val props = new TypedProperties() props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1") val keyGen = new ComplexKeyGenerator(props) - keyGen.getRecordKey(baseRow) - fail("Should have errored out") - } catch { - case e: IllegalArgumentException => - // do nothing - } - // recordkey field not specified - try { - val props = new TypedProperties() - props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionField") - new ComplexKeyGenerator(props).getKey(baseRecord) - fail("Should have errored out") - } catch { - case e: IllegalArgumentException => - // do nothing + keyGen.getKey(baseRecord) + keyGen.getRecordKey(baseRow) + keyGen.getRecordKey(internalRow, structType) } - // recordkey field not specified - try { + // Record's key field not specified + assertThrows(classOf[IllegalArgumentException]) { val props = new TypedProperties() props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionField") val keyGen = new ComplexKeyGenerator(props) + + keyGen.getKey(baseRecord) keyGen.getPartitionPath(baseRow) - fail("Should have errored out") - } catch { - case e: IllegalArgumentException => - // do nothing + keyGen.getPartitionPath(internalRow, structType) } - // nested field as record key and partition path - keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId,testNestedRecord.isAdmin", "testNestedRecord.userId,testNestedRecord.isAdmin", "false")) - val hk2 = keyGen.getKey(baseRecord) - assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", hk2.getRecordKey) - assertEquals("UserId1@001/false", hk2.getPartitionPath) - - // nested field as record key and partition path - assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", keyGen.getRecordKey(baseRow)) - assertEquals("UserId1@001/false", keyGen.getPartitionPath(baseRow)) - - // Nested record key not found - try { - new ComplexKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false")) - .getKey(baseRecord) - fail("Should have errored out") - } catch { - case e: HoodieException => - // do nothing + { + // Nested field as record key and partition path + val keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId,testNestedRecord.isAdmin", "testNestedRecord.userId,testNestedRecord.isAdmin", "false")) + + val expectedKey = new HoodieKey("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", "UserId1@001/false") + + assertEquals(expectedKey, keyGen.getKey(baseRecord)) + + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) } - // Nested record key not found - try { + { + // Nested record key not found val keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false")) - keyGen.getRecordKey(baseRow) - fail("Should have errored out") - } catch { - case e: HoodieException => - // do nothing + + assertThrows(classOf[HoodieException]) { + keyGen.getKey(baseRecord) + } + assertThrows(classOf[HoodieException]) { + keyGen.getRecordKey(baseRow) + } + assertThrows(classOf[HoodieException]) { + keyGen.getRecordKey(internalRow, structType) + } } - // if partition path can't be found, return default partition path - keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false")) - val hk3 = keyGen.getKey(baseRecord) - assertEquals("default", hk3.getPartitionPath) + { + // If partition path can't be found, return default partition path + val keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false")) + + // TODO this should throw + //assertThrows(classOf[HoodieException]) { + // keyGen.getKey(baseRecord) + //} + assertThrows(classOf[HoodieException]) { + keyGen.getPartitionPath(baseRow) + } + assertThrows(classOf[HoodieException]) { + keyGen.getPartitionPath(internalRow, structType) + } + } - assertEquals("default", keyGen.getPartitionPath(baseRow)) + { + // If enable hive style partitioning + val keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "true")) - // if enable hive style partitioning - keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "true")) - val hk4 = keyGen.getKey(baseRecord) - assertEquals("field1:field1,name:name1", hk4.getRecordKey) - assertEquals("field1=field1/name=name1", hk4.getPartitionPath) + val expectedKey = new HoodieKey("field1:field1,name:name1", "field1=field1/name=name1") - assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow)) - assertEquals("field1=field1/name=name1", keyGen.getPartitionPath(baseRow)) + assertEquals(expectedKey, keyGen.getKey(baseRecord)) - // if one part of the record key is empty, replace with "__empty__" - baseRecord.put("name", "") - keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) - val hk5 = keyGen.getKey(baseRecord) - assertEquals("field1:field1,name:__empty__", hk5.getRecordKey) - assertEquals("field1/default", hk5.getPartitionPath) + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) + } - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow)) - assertEquals("field1/default", keyGen.getPartitionPath(baseRow)) + { + // If one part of the record key is empty, replace with "__empty__" + val keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) - // if one part of the record key is null, replace with "__null__" - baseRecord.put("name", null) - keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) - val hk6 = keyGen.getKey(baseRecord) - assertEquals("field1:field1,name:__null__", hk6.getRecordKey) - assertEquals("field1/default", hk6.getPartitionPath) + baseRecord.put("name", "") + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) + internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow)) - assertEquals("field1/default", keyGen.getPartitionPath(baseRow)) + val expectedKey = new HoodieKey("field1:field1,name:__empty__", "field1/default") - // if all parts of the composite record key are null/empty, throw error - try { - baseRecord.put("name", "") - baseRecord.put("field1", null) + assertEquals(expectedKey, keyGen.getKey(baseRecord)) + + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) + } + + { + // If one part of the record key is null, replace with "__null__" + val keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) + + baseRecord.put("name", null) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) + internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + + val expectedKey = new HoodieKey("field1:field1,name:__null__", "field1/default") + + assertEquals(expectedKey, keyGen.getKey(baseRecord)) + + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) + } + + { + // If all parts of the composite record key are null/empty, throw error val props = new TypedProperties() props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1,name") props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "field1,name") - new ComplexKeyGenerator(props).getKey(baseRecord) - fail("Should have errored out") - } catch { - case e: HoodieKeyException => - // do nothing - } + val keyGen = new ComplexKeyGenerator(props) - // if all parts of the composite record key are null/empty, throw error - try { baseRecord.put("name", "") baseRecord.put("field1", null) - val props = new TypedProperties() - props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1,name") - props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "field1,name") - keyGen = new ComplexKeyGenerator(props) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - keyGen.getRecordKey(baseRow) - fail("Should have errored out") - } catch { - case e: HoodieKeyException => - // do nothing + internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + + assertThrows(classOf[HoodieKeyException]) { + keyGen.getKey(baseRecord) + } + assertThrows(classOf[HoodieKeyException]) { + keyGen.getRecordKey(baseRow) + } + assertThrows(classOf[HoodieKeyException]) { + keyGen.getRecordKey(internalRow, structType) + } } - // reset name and field1 values. - baseRecord.put("name", "name1") - baseRecord.put("field1", "field1") - keyGen = new ComplexKeyGenerator(getKeyConfig("field1, name", "field1, name", "false")) - val hk7 = keyGen.getKey(baseRecord) - assertEquals("field1:field1,name:name1", hk7.getRecordKey) - assertEquals("field1/name1", hk7.getPartitionPath) + { + // Reset name and field1 values. + val keyGen = new ComplexKeyGenerator(getKeyConfig("field1, name", "field1, name", "false")) - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow)) - assertEquals("field1/name1", keyGen.getPartitionPath(baseRow)) + baseRecord.put("name", "name1") + baseRecord.put("field1", "field1") - keyGen = new ComplexKeyGenerator(getKeyConfig("field1,", "field1,", "false")) - val hk8 = keyGen.getKey(baseRecord) - assertEquals("field1:field1", hk8.getRecordKey) - assertEquals("field1", hk8.getPartitionPath) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) + internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) - assertEquals("field1:field1", keyGen.getRecordKey(baseRow)) - assertEquals("field1", keyGen.getPartitionPath(baseRow)) - } + val expectedKey = new HoodieKey("field1:field1,name:name1", "field1/name1") - @Test def testGlobalDeleteKeyGenerator() = { - // top level, partition value included but not actually used - var keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) - val hk1 = keyGen.getKey(baseRecord) - assertEquals("field1:field1,name:name1", hk1.getRecordKey) - assertEquals("", hk1.getPartitionPath) + assertEquals(expectedKey, keyGen.getKey(baseRecord)) - assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow)) - assertEquals("", keyGen.getPartitionPath(baseRow)) + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) + } - // top level, partition value not included - val props = new TypedProperties() - props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1,name") - keyGen = new GlobalDeleteKeyGenerator(props) - val hk2 = keyGen.getKey(baseRecord) - assertEquals("field1:field1,name:name1", hk2.getRecordKey) - assertEquals("", hk2.getPartitionPath) - - assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow)) - assertEquals("", keyGen.getPartitionPath(baseRow)) - - // if one part of the record key is empty, replace with "__empty__" - baseRecord.put("name", "") - keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) - val hk3 = keyGen.getKey(baseRecord) - assertEquals("field1:field1,name:__empty__", hk3.getRecordKey) - assertEquals("", hk3.getPartitionPath) + { + val keyGen = new ComplexKeyGenerator(getKeyConfig("field1,", "field1,", "false")) - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow)) - assertEquals("", keyGen.getPartitionPath(baseRow)) + val expectedKey = new HoodieKey("field1:field1", "field1") - // if one part of the record key is null, replace with "__null__" - baseRecord.put("name", null) - keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) - val hk4 = keyGen.getKey(baseRecord) - assertEquals("field1:field1,name:__null__", hk4.getRecordKey) - assertEquals("", hk4.getPartitionPath) + assertEquals(expectedKey, keyGen.getKey(baseRecord)) - baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow)) - assertEquals("", keyGen.getPartitionPath(baseRow)) + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) + } + } - // recordkey field not specified - try { - val props = new TypedProperties() - props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionField") - new GlobalDeleteKeyGenerator(props).getKey(baseRecord) - fail("Should have errored out") - } catch { - case e: IllegalArgumentException => - // do nothing + @Test def testGlobalDeleteKeyGenerator(): Unit = { + { + // Top level, partition value included but not actually used + val keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) + + val expectedKey = new HoodieKey("field1:field1,name:name1", "") + + assertEquals(expectedKey, keyGen.getKey(baseRecord)) + + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) } - // recordkey field not specified - try { + { + // top level, partition value not included val props = new TypedProperties() - props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionField") + props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1,name") + val keyGen = new GlobalDeleteKeyGenerator(props) - keyGen.getRecordKey(baseRow) - fail("Should have errored out") - } catch { - case e: IllegalArgumentException => - // do nothing + + val expectedKey = new HoodieKey("field1:field1,name:name1", "") + + assertEquals(expectedKey, keyGen.getKey(baseRecord)) + + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) + } + + { + // If one part of the record key is empty, replace with "__empty__" + baseRecord.put("name", "") + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) + internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + + val keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) + + val expectedKey = new HoodieKey("field1:field1,name:__empty__", "") + + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) + } + + { + // If one part of the record key is null, replace with "__null__" + baseRecord.put("name", null) + baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) + internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + + val keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")) + + val expectedKey = new HoodieKey("field1:field1,name:__null__", "") + + assertEquals(expectedKey.getRecordKey, keyGen.getRecordKey(baseRow)) + assertEquals(expectedKey.getPartitionPath, keyGen.getPartitionPath(baseRow)) + assertEquals(UTF8String.fromString(expectedKey.getRecordKey), keyGen.getRecordKey(internalRow, structType)) + assertEquals(UTF8String.fromString(expectedKey.getPartitionPath), keyGen.getPartitionPath(internalRow, structType)) } - // Nested record key not found - try { - new GlobalDeleteKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false")) - .getKey(baseRecord) - fail("Should have errored out") - } catch { - case e: HoodieException => - // do nothing + { + // Record's key field not specified + val props = new TypedProperties() + props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionField") + + assertThrows(classOf[IllegalArgumentException]) { + new GlobalDeleteKeyGenerator(props) + } } - // Nested record key not found - try { + { + // Nested record key not found val keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false")) - keyGen.getRecordKey(baseRow) - fail("Should have errored out") - } catch { - case e: HoodieException => - // do nothing + + assertThrows(classOf[HoodieException]) { + keyGen.getKey(baseRecord) + } + assertThrows(classOf[HoodieException]) { + keyGen.getRecordKey(baseRow) + } + assertThrows(classOf[HoodieException]) { + keyGen.getRecordKey(internalRow, structType) + } } - // if all parts of the composite record key are null/empty, throw error - try { - baseRecord.put("name", "") - baseRecord.put("field1", null) + { + // If all parts of the composite record key are null/empty, throw error val props = new TypedProperties() props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1,name") - new GlobalDeleteKeyGenerator(props).getKey(baseRecord) - fail("Should have errored out") - } catch { - case e: HoodieKeyException => - // do nothing - } + val keyGen = new GlobalDeleteKeyGenerator(props) - // if all parts of the composite record key are null/empty, throw error - try { baseRecord.put("name", "") baseRecord.put("field1", null) baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType) - val props = new TypedProperties() - props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1,name") - val keyGen = new GlobalDeleteKeyGenerator(props) - keyGen.getRecordKey(baseRow) - fail("Should have errored out") - } catch { - case e: HoodieKeyException => - // do nothing + internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow) + + assertThrows(classOf[HoodieKeyException]) { + keyGen.getKey(baseRecord) + } + assertThrows(classOf[HoodieKeyException]) { + keyGen.getRecordKey(baseRow) + } + assertThrows(classOf[HoodieKeyException]) { + keyGen.getRecordKey(internalRow, structType) + } } } - @Test def testOverwriteWithLatestAvroPayload() = { + @Test def testOverwriteWithLatestAvroPayload(): Unit = { val overWritePayload1 = new OverwriteWithLatestAvroPayload(baseRecord, 1) val laterRecord = SchemaTestUtil .generateAvroRecordFromJson(schema, 2, "001", "f1") @@ -575,7 +550,7 @@ class TestDataSourceDefaults { assertEquals("field2", combinedGR21.get("field1").toString) } - @Test def testOverwriteWithLatestAvroPayloadCombineAndGetUpdateValue() = { + @Test def testOverwriteWithLatestAvroPayloadCombineAndGetUpdateValue(): Unit = { val baseOrderingVal: Object = baseRecord.get("favoriteIntNumber") val fieldSchema: Schema = baseRecord.getSchema().getField("favoriteIntNumber").schema() val props = new TypedProperties() @@ -594,7 +569,7 @@ class TestDataSourceDefaults { assertEquals("field2", precombinedGR.get("field1").toString) } - @Test def testDefaultHoodieRecordPayloadCombineAndGetUpdateValue() = { + @Test def testDefaultHoodieRecordPayloadCombineAndGetUpdateValue(): Unit = { val fieldSchema: Schema = baseRecord.getSchema().getField("favoriteIntNumber").schema() val props = HoodiePayloadConfig.newBuilder() .withPayloadOrderingField("favoriteIntNumber").build().getProps; @@ -630,7 +605,7 @@ class TestDataSourceDefaults { assertEquals(laterOrderingVal, laterWithEarlierGR.get("favoriteIntNumber")) } - @Test def testEmptyHoodieRecordPayload() = { + @Test def testEmptyHoodieRecordPayload(): Unit = { val emptyPayload1 = new EmptyHoodieRecordPayload(baseRecord, 1) val laterRecord = SchemaTestUtil .generateAvroRecordFromJson(schema, 2, "001", "f1")