1
0

[HUDI-4039] Make sure all builtin KeyGenerators properly implement Spark specific APIs (#5523)

This set of changes makes sure that all builtin KeyGenerators properly implement Spark-specific APIs in a performant way (minimizing key-generators overhead)
This commit is contained in:
Alexey Kudinkin
2022-07-22 08:35:07 -07:00
committed by GitHub
parent d5c904e10e
commit eea4a692c0
52 changed files with 1507 additions and 1363 deletions

View File

@@ -23,7 +23,6 @@ import org.apache.hudi.client.HoodieInternalWriteStatus;
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.ArrayList;
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 @@ public class BulkInsertDataInternalWriterHelper {
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 class BulkInsertDataInternalWriterHelper {
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 @@ public class BulkInsertDataInternalWriterHelper {
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() {

View File

@@ -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)
}

View File

@@ -28,6 +28,7 @@ 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;
@@ -83,7 +84,6 @@ public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
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 class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
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 class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
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 class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
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 class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
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 class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
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 class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
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 class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
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()));
}
}

View File

@@ -28,6 +28,7 @@ import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
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 class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
(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 class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
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 class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
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 class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
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 class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
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()));
}
}

View File

@@ -68,7 +68,6 @@ public class TestGlobalDeleteRecordGenerator extends KeyGeneratorTestUtilities {
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 class TestGlobalDeleteRecordGenerator extends KeyGeneratorTestUtilities {
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), "");

View File

@@ -94,7 +94,6 @@ public class TestNonpartitionedKeyGenerator extends KeyGeneratorTestUtilities {
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 class TestNonpartitionedKeyGenerator extends KeyGeneratorTestUtilities {
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 class TestNonpartitionedKeyGenerator extends KeyGeneratorTestUtilities {
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);

View File

@@ -21,11 +21,13 @@ package org.apache.hudi.keygen;
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 org.junit.jupiter.params.provider.MethodSource;
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 @@ public class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities {
@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 class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities {
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<GenericRecord> nestedColTestRecords() {

View File

@@ -34,6 +34,7 @@ import org.apache.spark.sql.Row;
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 class TestTimestampBasedKeyGenerator {
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 class TestTimestampBasedKeyGenerator {
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 class TestTimestampBasedKeyGenerator {
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 class TestTimestampBasedKeyGenerator {
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 class TestTimestampBasedKeyGenerator {
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 class TestTimestampBasedKeyGenerator {
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 class TestTimestampBasedKeyGenerator {
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

View File

@@ -43,7 +43,7 @@ import java.util.stream.Collectors;
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\"},"

View File

@@ -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")
}
}

View File

@@ -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
assertThrows(classOf[IllegalArgumentException]) {
new SimpleKeyGenerator(props)
}
}
// recordkey field not specified using Row
try {
{
// 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))
}
{
// Nested record key not found
val keyGen = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
assertThrows(classOf[HoodieException]) {
keyGen.getKey(baseRecord)
}
}
{
// 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)
}
}
{
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 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)
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow)
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/empty, throw error
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionField")
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1")
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "name")
val keyGen = new SimpleKeyGenerator(props)
keyGen.getPartitionPath(baseRow)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
}
// 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
}
// 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)
// 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)
// 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)
// 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
}
// 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)
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)
}
// 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
}
// if record key is null, 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)
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)
}
}
}
@@ -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.getKey(baseRecord)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
keyGen.getRecordKey(internalRow, structType)
}
// 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
}
// 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
val keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId,testNestedRecord.isAdmin", "testNestedRecord.userId,testNestedRecord.isAdmin", "false"))
// 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))
val expectedKey = new HoodieKey("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", "UserId1@001/false")
// 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
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"))
assertEquals("default", keyGen.getPartitionPath(baseRow))
// 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)
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
assertEquals("field1=field1/name=name1", keyGen.getPartitionPath(baseRow))
// 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)
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 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)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow))
assertEquals("field1/default", keyGen.getPartitionPath(baseRow))
// 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")
new ComplexKeyGenerator(props).getKey(baseRecord)
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
// TODO this should throw
//assertThrows(classOf[HoodieException]) {
// keyGen.getKey(baseRecord)
//}
assertThrows(classOf[HoodieException]) {
keyGen.getPartitionPath(baseRow)
}
assertThrows(classOf[HoodieException]) {
keyGen.getPartitionPath(internalRow, structType)
}
}
// if all parts of the composite record key are null/empty, throw error
try {
{
// If enable hive style partitioning
val keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "true"))
val expectedKey = new HoodieKey("field1:field1,name:name1", "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__"
val keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
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)
val expectedKey = new HoodieKey("field1:field1,name:__empty__", "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))
}
// 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)
{
// If one part of the record key is null, replace with "__null__"
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", null)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow)
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,", "field1,", "false"))
val hk8 = keyGen.getKey(baseRecord)
assertEquals("field1:field1", hk8.getRecordKey)
assertEquals("field1", hk8.getPartitionPath)
val expectedKey = new HoodieKey("field1:field1,name:__null__", "field1/default")
assertEquals("field1:field1", keyGen.getRecordKey(baseRow))
assertEquals("field1", keyGen.getPartitionPath(baseRow))
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")
val keyGen = new ComplexKeyGenerator(props)
baseRecord.put("name", "")
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)
}
}
{
// Reset name and field1 values.
val keyGen = new ComplexKeyGenerator(getKeyConfig("field1, name", "field1, name", "false"))
baseRecord.put("name", "name1")
baseRecord.put("field1", "field1")
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow)
val expectedKey = new HoodieKey("field1:field1,name:name1", "field1/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))
}
{
val keyGen = new ComplexKeyGenerator(getKeyConfig("field1,", "field1,", "false"))
val expectedKey = new HoodieKey("field1:field1", "field1")
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))
}
}
@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)
@Test def testGlobalDeleteKeyGenerator(): Unit = {
{
// Top level, partition value included but not actually used
val keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
assertEquals("", keyGen.getPartitionPath(baseRow))
val expectedKey = new HoodieKey("field1:field1,name:name1", "")
// 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(expectedKey, keyGen.getKey(baseRecord))
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)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow))
assertEquals("", keyGen.getPartitionPath(baseRow))
// 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)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow))
assertEquals("", keyGen.getPartitionPath(baseRow))
// 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
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))
}
// 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
{
// 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))
}
// Nested record key not found
try {
{
// 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))
}
{
// 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
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)
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
}
// 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)
{
// 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")
val keyGen = new GlobalDeleteKeyGenerator(props)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
baseRecord.put("name", "")
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)
}
}
}
@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")