1
0

[HUDI-1296] Support Metadata Table in Spark Datasource (#4789)

* Bootstrapping initial support for Metadata Table in Spark Datasource

- Consolidated Avro/Row conversion utilities to center around Spark's AvroDeserializer ; removed duplication
- Bootstrapped HoodieBaseRelation
- Updated HoodieMergeOnReadRDD to be able to handle Metadata Table
- Modified MOR relations to be able to read different Base File formats (Parquet, HFile)
This commit is contained in:
Alexey Kudinkin
2022-02-24 13:23:13 -08:00
committed by GitHub
parent 521338b4d9
commit 85e8a5c4de
56 changed files with 1634 additions and 1010 deletions

View File

@@ -443,13 +443,18 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie
val partitionColumns = tableConfig.getPartitionFieldProp.split(",").map(_.toLowerCase)
val partitionSchema = StructType(tableSchema.filter(f => partitionColumns.contains(f.name)))
// NOTE: Here we fallback to "" to make sure that null value is not overridden with
// default value ("ts")
// TODO(HUDI-3456) clean up
val preCombineField = hoodieCatalogTable.preCombineKey.getOrElse("")
// Enable the hive sync by default if spark have enable the hive metastore.
val enableHive = isEnableHive(sparkSession)
withSparkConf(sparkSession, hoodieCatalogTable.catalogProperties) {
Map(
"path" -> path,
RECORDKEY_FIELD.key -> tableConfig.getRecordKeyFieldProp,
PRECOMBINE_FIELD.key -> hoodieCatalogTable.preCombineKey.getOrElse(""),
PRECOMBINE_FIELD.key -> preCombineField,
TBL_NAME.key -> hoodieCatalogTable.tableName,
PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp,
PAYLOAD_CLASS_NAME.key -> classOf[ExpressionPayload].getCanonicalName,
@@ -470,6 +475,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie
HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200",
SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL
)
.filter { case (_, v) => v != null }
}
}
}

View File

@@ -0,0 +1,178 @@
/*
* 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.avro.generic.GenericRecord;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
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.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import static junit.framework.TestCase.assertEquals;
public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps(boolean getComplexRecordKey) {
TypedProperties properties = new TypedProperties();
if (getComplexRecordKey) {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key, pii_col");
} else {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
}
properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true");
return properties;
}
private TypedProperties getPropertiesWithoutPartitionPathProp() {
return getCommonProps(false);
}
private TypedProperties getPropertiesWithoutRecordKeyProp() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
return properties;
}
private TypedProperties getWrongRecordKeyFieldProps() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_wrong_key");
return properties;
}
private TypedProperties getProps() {
TypedProperties properties = getCommonProps(true);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp,ts_ms");
return properties;
}
@Test
public void testNullPartitionPathFields() {
Assertions.assertThrows(IllegalArgumentException.class, () -> new ComplexKeyGenerator(getPropertiesWithoutPartitionPathProp()));
}
@Test
public void testNullRecordKeyFields() {
Assertions.assertThrows(IllegalArgumentException.class, () -> new ComplexKeyGenerator(getPropertiesWithoutRecordKeyProp()));
}
@Test
public void testWrongRecordKeyField() {
ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getWrongRecordKeyFieldProps());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testHappyFlow() {
ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getProps());
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=2020-03-21");
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=2020-03-21");
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686/ts_ms=2020-03-21");
}
@Test
public void testSingleValueKeyGenerator() {
TypedProperties properties = new TypedProperties();
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.getPartitionPathFields().size(), 1);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
GenericRecord record = dataGenerator.generateGenericRecords(1).get(0);
String rowKey = record.get("_row_key").toString();
String partitionPath = record.get("timestamp").toString();
HoodieKey hoodieKey = compositeKeyGenerator.getKey(record);
assertEquals("_row_key:" + rowKey, hoodieKey.getRecordKey());
assertEquals(partitionPath, hoodieKey.getPartitionPath());
Row row = KeyGeneratorTestUtilities.getRow(record, HoodieTestDataGenerator.AVRO_SCHEMA,
AvroConversionUtils.convertAvroSchemaToStructType(HoodieTestDataGenerator.AVRO_SCHEMA));
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(row), partitionPath);
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(internalRow, row.schema()), partitionPath);
}
@Test
public void testMultipleValueKeyGenerator() {
TypedProperties properties = new TypedProperties();
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.getPartitionPathFields().size(), 2);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
GenericRecord record = dataGenerator.generateGenericRecords(1).get(0);
String rowKey =
"_row_key" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("_row_key").toString() + ","
+ "timestamp" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("timestamp").toString();
String partitionPath = record.get("rider").toString() + "/" + record.get("driver").toString();
HoodieKey hoodieKey = compositeKeyGenerator.getKey(record);
assertEquals(rowKey, hoodieKey.getRecordKey());
assertEquals(partitionPath, hoodieKey.getPartitionPath());
Row row = KeyGeneratorTestUtilities.getRow(record, HoodieTestDataGenerator.AVRO_SCHEMA,
AvroConversionUtils.convertAvroSchemaToStructType(HoodieTestDataGenerator.AVRO_SCHEMA));
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(row), partitionPath);
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(internalRow, row.schema()), partitionPath);
}
@Test
public void testMultipleValueKeyGeneratorNonPartitioned() {
TypedProperties properties = new TypedProperties();
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.getPartitionPathFields().size(), 0);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
GenericRecord record = dataGenerator.generateGenericRecords(1).get(0);
String rowKey =
"_row_key" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("_row_key").toString() + ","
+ "timestamp" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("timestamp").toString();
String partitionPath = "";
HoodieKey hoodieKey = compositeKeyGenerator.getKey(record);
assertEquals(rowKey, hoodieKey.getRecordKey());
assertEquals(partitionPath, hoodieKey.getPartitionPath());
Row row = KeyGeneratorTestUtilities.getRow(record, HoodieTestDataGenerator.AVRO_SCHEMA,
AvroConversionUtils.convertAvroSchemaToStructType(HoodieTestDataGenerator.AVRO_SCHEMA));
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(row), partitionPath);
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(internalRow, row.schema()), partitionPath);
}
}

View File

@@ -0,0 +1,385 @@
/*
* 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.avro.generic.GenericRecord;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.keygen.constant.KeyGeneratorType;
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.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.io.PrintWriter;
import java.io.StringWriter;
public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
/**
* Method to create props used for common cases.
*
* @param getComplexRecordKey Use complex record key or not
* @param useKeyGeneratorClassName Use KeyGenerator class name initialize KeyGenerator or not.
* true use {@code HoodieWriteConfig.KEYGENERATOR_CLASS_PROP},
* false use {@code HoodieWriteConfig.KEYGENERATOR_TYPE_PROP}
* @return TypedProperties used to initialize KeyGenerator.
*/
private TypedProperties getCommonProps(boolean getComplexRecordKey, boolean useKeyGeneratorClassName) {
TypedProperties properties = new TypedProperties();
if (getComplexRecordKey) {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key, pii_col");
} else {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
}
if (useKeyGeneratorClassName) {
properties.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), CustomKeyGenerator.class.getName());
} else {
properties.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), KeyGeneratorType.CUSTOM.name());
}
properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true");
return properties;
}
private TypedProperties getPropertiesForSimpleKeyGen(boolean useKeyGeneratorClassName) {
TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp:simple");
return properties;
}
private TypedProperties getImproperPartitionFieldFormatProp(boolean useKeyGeneratorClassName) {
TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
return properties;
}
private TypedProperties getInvalidPartitionKeyTypeProps(boolean useKeyGeneratorClassName) {
TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp:dummy");
return properties;
}
private TypedProperties getComplexRecordKeyWithSimplePartitionProps(boolean useKeyGeneratorClassName) {
TypedProperties properties = getCommonProps(true, useKeyGeneratorClassName);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp:simple");
return properties;
}
private TypedProperties getComplexRecordKeyAndPartitionPathProps(boolean useKeyGeneratorClassName) {
TypedProperties properties = getCommonProps(true, useKeyGeneratorClassName);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp:simple,ts_ms:timestamp");
populateNecessaryPropsForTimestampBasedKeyGen(properties);
return properties;
}
private TypedProperties getPropsWithoutRecordKeyFieldProps(boolean useKeyGeneratorClassName) {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp:simple");
if (useKeyGeneratorClassName) {
properties.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), CustomKeyGenerator.class.getName());
} else {
properties.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), KeyGeneratorType.CUSTOM.name());
}
return properties;
}
private void populateNecessaryPropsForTimestampBasedKeyGen(TypedProperties properties) {
properties.put("hoodie.deltastreamer.keygen.timebased.timestamp.type", "DATE_STRING");
properties.put("hoodie.deltastreamer.keygen.timebased.input.dateformat", "yyyy-MM-dd");
properties.put("hoodie.deltastreamer.keygen.timebased.output.dateformat", "yyyyMMdd");
}
private TypedProperties getPropertiesForTimestampBasedKeyGen(boolean useKeyGeneratorClassName) {
TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "ts_ms:timestamp");
populateNecessaryPropsForTimestampBasedKeyGen(properties);
return properties;
}
private TypedProperties getPropertiesForNonPartitionedKeyGen(boolean useKeyGeneratorClassName) {
TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "");
return properties;
}
private String stackTraceToString(Throwable e) {
StringWriter sw = new StringWriter();
PrintWriter pw = new PrintWriter(sw);
e.printStackTrace(pw);
return sw.toString();
}
@Test
public void testSimpleKeyGeneratorWithKeyGeneratorClass() throws IOException {
testSimpleKeyGenerator(getPropertiesForSimpleKeyGen(true));
}
@Test
public void testSimpleKeyGeneratorWithKeyGeneratorType() throws IOException {
testSimpleKeyGenerator(getPropertiesForSimpleKeyGen(false));
}
public void testSimpleKeyGenerator(TypedProperties props) throws IOException {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686");
}
@Test
public void testTimestampBasedKeyGeneratorWithKeyGeneratorClass() throws IOException {
testTimestampBasedKeyGenerator(getPropertiesForTimestampBasedKeyGen(true));
}
@Test
public void testTimestampBasedKeyGeneratorWithKeyGeneratorType() throws IOException {
testTimestampBasedKeyGenerator(getPropertiesForTimestampBasedKeyGen(false));
}
public void testTimestampBasedKeyGenerator(TypedProperties props) throws IOException {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertEquals(key.getPartitionPath(), "ts_ms=20200321");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "ts_ms=20200321");
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "ts_ms=20200321");
}
@Test
public void testNonPartitionedKeyGeneratorWithKeyGeneratorClass() throws IOException {
testNonPartitionedKeyGenerator(getPropertiesForNonPartitionedKeyGen(true));
}
@Test
public void testNonPartitionedKeyGeneratorWithKeyGeneratorType() throws IOException {
testNonPartitionedKeyGenerator(getPropertiesForNonPartitionedKeyGen(false));
}
public void testNonPartitionedKeyGenerator(TypedProperties props) throws IOException {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertTrue(key.getPartitionPath().isEmpty());
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertTrue(keyGenerator.getPartitionPath(row).isEmpty());
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertTrue(keyGenerator.getPartitionPath(internalRow, row.schema()).isEmpty());
}
@Test
public void testInvalidPartitionKeyTypeWithKeyGeneratorClass() {
testInvalidPartitionKeyType(getInvalidPartitionKeyTypeProps(true));
}
@Test
public void testInvalidPartitionKeyTypeWithKeyGeneratorType() {
testInvalidPartitionKeyType(getInvalidPartitionKeyTypeProps(false));
}
public void testInvalidPartitionKeyType(TypedProperties props) {
try {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
keyGenerator.getKey(getRecord());
Assertions.fail("should fail when invalid PartitionKeyType is provided!");
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("No enum constant org.apache.hudi.keygen.CustomAvroKeyGenerator.PartitionKeyType.DUMMY"));
}
try {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
keyGenerator.getPartitionPath(row);
Assertions.fail("should fail when invalid PartitionKeyType is provided!");
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("No enum constant org.apache.hudi.keygen.CustomAvroKeyGenerator.PartitionKeyType.DUMMY"));
}
}
@Test
public void testNoRecordKeyFieldPropWithKeyGeneratorClass() {
testNoRecordKeyFieldProp(true);
}
@Test
public void testNoRecordKeyFieldPropWithKeyGeneratorType() {
testNoRecordKeyFieldProp(false);
}
public void testNoRecordKeyFieldProp(boolean useKeyGeneratorClassName) {
TypedProperties propsWithoutRecordKeyFieldProps = getPropsWithoutRecordKeyFieldProps(useKeyGeneratorClassName);
try {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(propsWithoutRecordKeyFieldProps);
keyGenerator.getKey(getRecord());
Assertions.fail("should fail when record key field is not provided!");
} catch (Exception e) {
if (useKeyGeneratorClassName) {
// "Property hoodie.datasource.write.recordkey.field not found" exception cause CustomKeyGenerator init fail
Assertions.assertTrue(e
.getCause()
.getCause()
.getCause()
.getMessage()
.contains("Property hoodie.datasource.write.recordkey.field not found"));
} else {
Assertions.assertTrue(stackTraceToString(e).contains("Property hoodie.datasource.write.recordkey.field not found"));
}
}
try {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(propsWithoutRecordKeyFieldProps);
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
keyGenerator.getRecordKey(row);
Assertions.fail("should fail when record key field is not provided!");
} catch (Exception e) {
if (useKeyGeneratorClassName) {
// "Property hoodie.datasource.write.recordkey.field not found" exception cause CustomKeyGenerator init fail
Assertions.assertTrue(e
.getCause()
.getCause()
.getCause()
.getMessage()
.contains("Property hoodie.datasource.write.recordkey.field not found"));
} else {
Assertions.assertTrue(stackTraceToString(e).contains("Property hoodie.datasource.write.recordkey.field not found"));
}
}
}
@Test
public void testPartitionFieldsInImproperFormatWithKeyGeneratorClass() {
testPartitionFieldsInImproperFormat(getImproperPartitionFieldFormatProp(true));
}
@Test
public void testPartitionFieldsInImproperFormatWithKeyGeneratorType() {
testPartitionFieldsInImproperFormat(getImproperPartitionFieldFormatProp(false));
}
public void testPartitionFieldsInImproperFormat(TypedProperties props) {
try {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
keyGenerator.getKey(getRecord());
Assertions.fail("should fail when partition key field is provided in improper format!");
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("Unable to find field names for partition path in proper format"));
}
try {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
keyGenerator.getPartitionPath(row);
Assertions.fail("should fail when partition key field is provided in improper format!");
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("Unable to find field names for partition path in proper format"));
}
}
@Test
public void testComplexRecordKeyWithSimplePartitionPathWithKeyGeneratorClass() throws IOException {
testComplexRecordKeyWithSimplePartitionPath(getComplexRecordKeyWithSimplePartitionProps(true));
}
@Test
public void testComplexRecordKeyWithSimplePartitionPathWithKeyGeneratorType() throws IOException {
testComplexRecordKeyWithSimplePartitionPath(getComplexRecordKeyWithSimplePartitionProps(false));
}
public void testComplexRecordKeyWithSimplePartitionPath(TypedProperties props) throws IOException {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(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");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686");
}
@Test
public void testComplexRecordKeysWithComplexPartitionPathWithKeyGeneratorClass() throws IOException {
testComplexRecordKeysWithComplexPartitionPath(getComplexRecordKeyAndPartitionPathProps(true));
}
@Test
public void testComplexRecordKeysWithComplexPartitionPathWithKeyGeneratorType() throws IOException {
testComplexRecordKeysWithComplexPartitionPath(getComplexRecordKeyAndPartitionPathProps(false));
}
public void testComplexRecordKeysWithComplexPartitionPath(TypedProperties props) throws IOException {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(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");
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");
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686/ts_ms=20200321");
}
}

View File

@@ -0,0 +1,86 @@
/*
* 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.avro.generic.GenericRecord;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
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.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
public class TestGlobalDeleteKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps(boolean getComplexRecordKey) {
TypedProperties properties = new TypedProperties();
if (getComplexRecordKey) {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key,pii_col");
} else {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
}
properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true");
return properties;
}
private TypedProperties getPropertiesWithoutRecordKeyProp() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
return properties;
}
private TypedProperties getWrongRecordKeyFieldProps() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_wrong_key");
return properties;
}
private TypedProperties getProps() {
TypedProperties properties = getCommonProps(true);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp,ts_ms");
return properties;
}
@Test
public void testNullRecordKeyFields() {
Assertions.assertThrows(IllegalArgumentException.class, () -> new GlobalDeleteKeyGenerator(getPropertiesWithoutRecordKeyProp()));
}
@Test
public void testWrongRecordKeyField() {
GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getWrongRecordKeyFieldProps());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testHappyFlow() {
GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getProps());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(key.getPartitionPath(), "");
keyGenerator.buildFieldPositionMapIfNeeded(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

@@ -0,0 +1,135 @@
/*
* 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.avro.generic.GenericRecord;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
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.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import static junit.framework.TestCase.assertEquals;
public class TestNonpartitionedKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps(boolean getComplexRecordKey) {
TypedProperties properties = new TypedProperties();
if (getComplexRecordKey) {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key, pii_col");
} else {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
}
properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true");
return properties;
}
private TypedProperties getPropertiesWithoutPartitionPathProp() {
return getCommonProps(false);
}
private TypedProperties getPropertiesWithPartitionPathProp() {
TypedProperties properties = getCommonProps(true);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp,ts_ms");
return properties;
}
private TypedProperties getPropertiesWithoutRecordKeyProp() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
return properties;
}
private TypedProperties getWrongRecordKeyFieldProps() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_wrong_key");
return properties;
}
@Test
public void testNullRecordKeyFields() {
Assertions.assertThrows(IllegalArgumentException.class, () -> new NonpartitionedKeyGenerator(getPropertiesWithoutRecordKeyProp()));
}
@Test
public void testNonNullPartitionPathFields() {
TypedProperties properties = getPropertiesWithPartitionPathProp();
NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties);
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(properties.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()), "timestamp,ts_ms");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "");
}
@Test
public void testNullPartitionPathFields() {
TypedProperties properties = getPropertiesWithoutPartitionPathProp();
NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties);
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "");
}
@Test
public void testWrongRecordKeyField() {
NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(getWrongRecordKeyFieldProps());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testSingleValueKeyGeneratorNonPartitioned() {
TypedProperties properties = new TypedProperties();
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.getPartitionPathFields().size(), 0);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
GenericRecord record = dataGenerator.generateGenericRecords(1).get(0);
String rowKey = record.get("timestamp").toString();
HoodieKey hoodieKey = keyGenerator.getKey(record);
assertEquals(rowKey, hoodieKey.getRecordKey());
assertEquals("", hoodieKey.getPartitionPath());
}
@Test
public void testMultipleValueKeyGeneratorNonPartitioned1() {
TypedProperties properties = new TypedProperties();
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.getPartitionPathFields().size(), 0);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
GenericRecord record = dataGenerator.generateGenericRecords(1).get(0);
String rowKey =
"timestamp" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("timestamp").toString() + ","
+ "driver" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("driver").toString();
String partitionPath = "";
HoodieKey hoodieKey = keyGenerator.getKey(record);
assertEquals(rowKey, hoodieKey.getRecordKey());
assertEquals(partitionPath, hoodieKey.getPartitionPath());
}
}

View File

@@ -0,0 +1,162 @@
/*
* 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.avro.generic.GenericRecord;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
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.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.MethodSource;
import java.util.stream.Stream;
import static org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH;
public class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true");
return properties;
}
private TypedProperties getPropertiesWithoutPartitionPathProp() {
return getCommonProps();
}
private TypedProperties getPropertiesWithoutRecordKeyProp() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
return properties;
}
private TypedProperties getWrongRecordKeyFieldProps() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_wrong_key");
return properties;
}
private TypedProperties getWrongPartitionPathFieldProps() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "_wrong_partition_path");
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
return properties;
}
private TypedProperties getComplexRecordKeyProp() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key,pii_col");
return properties;
}
private TypedProperties getProps() {
TypedProperties properties = getCommonProps();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
return properties;
}
private TypedProperties getPropsWithNestedPartitionPathField() {
TypedProperties properties = getCommonProps();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "nested_col.prop1");
return properties;
}
@Test
public void testNullPartitionPathFields() {
Assertions.assertThrows(IllegalArgumentException.class, () -> new SimpleKeyGenerator(getPropertiesWithoutPartitionPathProp()));
}
@Test
public void testNullRecordKeyFields() {
Assertions.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.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@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);
}
@Test
public void testComplexRecordKeyField() {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getComplexRecordKeyProp());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
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");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686");
}
private static Stream<GenericRecord> nestedColTestRecords() {
return Stream.of(null, getNestedColRecord(null, 10L),
getNestedColRecord("", 10L), getNestedColRecord("val1", 10L));
}
@ParameterizedTest
@MethodSource("nestedColTestRecords")
public void testNestedPartitionPathField(GenericRecord nestedColRecord) {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getPropsWithNestedPartitionPathField());
GenericRecord record = getRecord(nestedColRecord);
String partitionPathFieldValue = null;
if (nestedColRecord != null) {
partitionPathFieldValue = (String) nestedColRecord.get("prop1");
}
String expectedPartitionPath = "nested_col.prop1="
+ (partitionPathFieldValue != null && !partitionPathFieldValue.isEmpty() ? partitionPathFieldValue : HUDI_DEFAULT_PARTITION_PATH);
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals("key1", key.getRecordKey());
Assertions.assertEquals(expectedPartitionPath, key.getPartitionPath());
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals("key1", keyGenerator.getRecordKey(row));
Assertions.assertEquals(expectedPartitionPath, keyGenerator.getPartitionPath(row));
}
}

View File

@@ -0,0 +1,449 @@
/*
* 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.avro.Conversions;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericFixed;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.testutils.SchemaTestUtil;
import org.apache.hudi.exception.HoodieKeyGeneratorException;
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.sql.catalyst.expressions.GenericRowWithSchema;
import org.apache.spark.sql.types.StructType;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import scala.Function1;
import scala.Tuple2;
import java.io.IOException;
import java.math.BigDecimal;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class TestTimestampBasedKeyGenerator {
private GenericRecord baseRecord;
private TypedProperties properties = new TypedProperties();
private Schema schema;
private StructType structType;
private Row baseRow;
private InternalRow internalRow;
@BeforeEach
public void initialize() throws IOException {
schema = SchemaTestUtil.getTimestampEvolvedSchema();
structType = AvroConversionUtils.convertAvroSchemaToStructType(schema);
baseRecord = SchemaTestUtil
.generateAvroRecordFromJson(schema, 1, "001", "f1");
baseRow = genericRecordToRow(baseRecord);
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "field1");
properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "createTime");
properties.setProperty(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "false");
}
private Row genericRecordToRow(GenericRecord baseRecord) {
Function1<GenericRecord, Row> convertor = AvroConversionUtils.createConverterToRow(baseRecord.getSchema(), structType);
Row row = convertor.apply(baseRecord);
int fieldCount = structType.fieldNames().length;
Object[] values = new Object[fieldCount];
for (int i = 0; i < fieldCount; i++) {
values[i] = row.get(i);
}
return new GenericRowWithSchema(values, structType);
}
private TypedProperties getBaseKeyConfig(String partitionPathField, String timestampType, String dateFormat, String timezone, String scalarType) {
TypedProperties properties = new TypedProperties(this.properties);
properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), partitionPathField);
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_TYPE_FIELD_PROP, timestampType);
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, dateFormat);
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_TIMEZONE_FORMAT_PROP, timezone);
if (scalarType != null) {
properties.setProperty("hoodie.deltastreamer.keygen.timebased.timestamp.scalar.time.unit", scalarType);
}
return properties;
}
private TypedProperties getBaseKeyConfig(String partitionPathField,
String timestampType,
String inputFormatList,
String inputFormatDelimiterRegex,
String inputTimezone,
String outputFormat,
String outputTimezone) {
TypedProperties properties = new TypedProperties(this.properties);
properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), partitionPathField);
if (timestampType != null) {
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_TYPE_FIELD_PROP, timestampType);
}
if (inputFormatList != null) {
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP, inputFormatList);
}
if (inputFormatDelimiterRegex != null) {
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMITER_REGEX_PROP, inputFormatDelimiterRegex);
}
if (inputTimezone != null) {
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_INPUT_TIMEZONE_FORMAT_PROP, inputTimezone);
}
if (outputFormat != null) {
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, outputFormat);
}
if (outputTimezone != null) {
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP, outputTimezone);
}
return properties;
}
@Test
public void testTimestampBasedKeyGenerator() throws IOException {
// timezone is GMT+8:00
baseRecord.put("createTime", 1578283932000L);
properties = getBaseKeyConfig("createTime", "EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null);
TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", hk1.getPartitionPath());
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()));
// timezone is GMT+8:00, createTime is BigDecimal
BigDecimal decimal = new BigDecimal("1578283932000.0001");
Conversions.DecimalConversion conversion = new Conversions.DecimalConversion();
Tuple2<Object, Schema> resolvedNullableSchema = AvroConversionUtils.resolveAvroTypeNullability(schema.getField("createTimeDecimal").schema());
GenericFixed avroDecimal = conversion.toFixed(decimal, resolvedNullableSchema._2, LogicalTypes.decimal(20, 4));
baseRecord.put("createTimeDecimal", avroDecimal);
properties = getBaseKeyConfig("createTimeDecimal", "EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey bigDecimalKey = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", bigDecimalKey.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
// timezone is GMT
properties = getBaseKeyConfig("createTime", "EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT", null);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk2 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 04", hk2.getPartitionPath());
assertEquals("2020-01-06 04", keyGen.getPartitionPath(baseRow));
// timestamp is DATE_STRING, timezone is GMT+8:00
baseRecord.put("createTimeString", "2020-01-06 12:12:12");
properties = getBaseKeyConfig("createTimeString", "DATE_STRING", "yyyy-MM-dd hh", "GMT+8:00", null);
properties.setProperty("hoodie.deltastreamer.keygen.timebased.input.dateformat", "yyyy-MM-dd hh:mm:ss");
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk3 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", hk3.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
// timezone is GMT
properties = getBaseKeyConfig("createTimeString", "DATE_STRING", "yyyy-MM-dd hh", "GMT", null);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk4 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", hk4.getPartitionPath());
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
// timezone is GMT+8:00, createTime is null
baseRecord.put("createTime", null);
properties = getBaseKeyConfig("createTime", "EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk5 = keyGen.getKey(baseRecord);
assertEquals("1970-01-01 08", hk5.getPartitionPath());
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()));
// timestamp is DATE_STRING, timezone is GMT, createTime is null
baseRecord.put("createTimeString", null);
properties = getBaseKeyConfig("createTime", "DATE_STRING", "yyyy-MM-dd hh:mm:ss", "GMT", null);
properties.setProperty("hoodie.deltastreamer.keygen.timebased.input.dateformat", "yyyy-MM-dd hh:mm:ss");
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk6 = keyGen.getKey(baseRecord);
assertEquals("1970-01-01 12:00:00", hk6.getPartitionPath());
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()));
}
@Test
public void testScalar() throws IOException {
// timezone is GMT+8:00
baseRecord.put("createTime", 20000L);
// timezone is GMT
properties = getBaseKeyConfig("createTime", "SCALAR", "yyyy-MM-dd hh", "GMT", "days");
TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
assertEquals(hk1.getPartitionPath(), "2024-10-04 12");
// test w/ Row
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()));
// timezone is GMT, createTime is null
baseRecord.put("createTime", null);
properties = getBaseKeyConfig("createTime", "SCALAR", "yyyy-MM-dd hh", "GMT", "days");
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk2 = keyGen.getKey(baseRecord);
assertEquals("1970-01-02 12", hk2.getPartitionPath());
// test w/ Row
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()));
// timezone is GMT. number of days store integer in mysql
baseRecord.put("createTime", 18736L);
properties = getBaseKeyConfig("createTime", "SCALAR", "yyyy-MM-dd", "GMT", "DAYS");
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey scalarSecondsKey = keyGen.getKey(baseRecord);
assertEquals("2021-04-19", scalarSecondsKey.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("2021-04-19", keyGen.getPartitionPath(baseRow));
}
@Test
public void testScalarWithLogicalType() throws IOException {
schema = SchemaTestUtil.getTimestampWithLogicalTypeSchema();
structType = AvroConversionUtils.convertAvroSchemaToStructType(schema);
baseRecord = SchemaTestUtil.generateAvroRecordFromJson(schema, 1, "001", "f1");
baseRecord.put("createTime", 1638513806000000L);
properties = getBaseKeyConfig("createTime", "SCALAR", "yyyy/MM/dd", "GMT", "MICROSECONDS");
properties.setProperty(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), "true");
TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
assertEquals("2021/12/03", hk1.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("2021/12/03", keyGen.getPartitionPath(baseRow));
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
assertEquals("2021/12/03", keyGen.getPartitionPath(internalRow, baseRow.schema()));
// timezone is GMT, createTime is null
baseRecord.put("createTime", null);
properties = getBaseKeyConfig("createTime", "SCALAR", "yyyy/MM/dd", "GMT", "MICROSECONDS");
properties.setProperty(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), "true");
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk2 = keyGen.getKey(baseRecord);
assertEquals("1970/01/01", hk2.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("1970/01/01", keyGen.getPartitionPath(baseRow));
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
assertEquals("1970/01/01", keyGen.getPartitionPath(internalRow, baseRow.schema()));
}
@Test
public void test_ExpectsMatch_SingleInputFormat_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTimeString", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"createTimeString",
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"GMT");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_SingleInputFormats_ISO8601WithMsZ_OutputTimezoneAsInputDateTimeZone() throws IOException {
baseRecord.put("createTimeString", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"createTimeString",
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTimeString", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"createTimeString",
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsZ_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTimeString", "2020-04-01T13:01:33Z");
properties = this.getBaseKeyConfig(
"createTimeString",
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsWithOffset_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTimeString", "2020-04-01T13:01:33-05:00");
properties = this.getBaseKeyConfig(
"createTimeString",
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040118", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040118", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsWithOffset_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTimeString", "2020-04-01T13:01:33.123-05:00");
properties = this.getBaseKeyConfig(
"createTimeString",
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040118", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040118", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsEST() throws IOException {
baseRecord.put("createTimeString", "2020-04-01T13:01:33.123Z");
properties = this.getBaseKeyConfig(
"createTimeString",
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"EST");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040109", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040109", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_Throws_MultipleInputFormats_InputDateNotMatchingFormats() throws IOException {
baseRecord.put("createTimeString", "2020-04-01 13:01:33.123-05:00");
properties = this.getBaseKeyConfig(
"createTimeString",
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
Assertions.assertThrows(HoodieKeyGeneratorException.class, () -> keyGen.getKey(baseRecord));
baseRow = genericRecordToRow(baseRecord);
Assertions.assertThrows(HoodieKeyGeneratorException.class, () -> keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ShortDate_OutputCustomDate() throws IOException {
baseRecord.put("createTimeString", "20200401");
properties = this.getBaseKeyConfig(
"createTimeString",
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ,yyyyMMdd",
"",
"UTC",
"MM/dd/yyyy",
"UTC");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("04/01/2020", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("04/01/2020", keyGen.getPartitionPath(baseRow));
}
}

View File

@@ -0,0 +1,101 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.keygen.factory;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieKeyGeneratorException;
import org.apache.hudi.keygen.ComplexKeyGenerator;
import org.apache.hudi.keygen.CustomKeyGenerator;
import org.apache.hudi.keygen.GlobalDeleteKeyGenerator;
import org.apache.hudi.keygen.KeyGenerator;
import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.keygen.TimestampBasedKeyGenerator;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.keygen.constant.KeyGeneratorType;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import java.io.IOException;
import java.util.stream.Stream;
public class TestCreateKeyGeneratorByTypeWithFactory {
private TypedProperties props;
private static Stream<Arguments> configParams() {
String[] types = {KeyGeneratorType.SIMPLE.name(), KeyGeneratorType.TIMESTAMP.name(), KeyGeneratorType.COMPLEX.name(),
KeyGeneratorType.CUSTOM.name(), KeyGeneratorType.NON_PARTITION.name(), KeyGeneratorType.GLOBAL_DELETE.name()};
return Stream.of(types).map(Arguments::of);
}
@BeforeEach
public void init() {
props = new TypedProperties();
props.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
props.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true");
props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
// for timestamp based key generator
props.put("hoodie.deltastreamer.keygen.timebased.timestamp.type", "DATE_STRING");
props.put("hoodie.deltastreamer.keygen.timebased.input.dateformat", "yyyy-MM-dd");
props.put("hoodie.deltastreamer.keygen.timebased.output.dateformat", "yyyyMMdd");
}
@AfterEach
public void teardown() {
props = null;
}
@ParameterizedTest
@MethodSource("configParams")
public void testKeyGeneratorTypes(String keyGenType) throws IOException {
props.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), keyGenType);
KeyGeneratorType keyType = KeyGeneratorType.valueOf(keyGenType);
KeyGenerator keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
switch (keyType) {
case SIMPLE:
Assertions.assertEquals(SimpleKeyGenerator.class.getName(), keyGenerator.getClass().getName());
return;
case COMPLEX:
Assertions.assertEquals(ComplexKeyGenerator.class.getName(), keyGenerator.getClass().getName());
return;
case TIMESTAMP:
Assertions.assertEquals(TimestampBasedKeyGenerator.class.getName(), keyGenerator.getClass().getName());
return;
case CUSTOM:
Assertions.assertEquals(CustomKeyGenerator.class.getName(), keyGenerator.getClass().getName());
return;
case NON_PARTITION:
Assertions.assertEquals(NonpartitionedKeyGenerator.class.getName(), keyGenerator.getClass().getName());
return;
case GLOBAL_DELETE:
Assertions.assertEquals(GlobalDeleteKeyGenerator.class.getName(), keyGenerator.getClass().getName());
return;
default:
throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGenType);
}
}
}

View File

@@ -0,0 +1,82 @@
/*
* 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.factory;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieKeyGeneratorException;
import org.apache.hudi.keygen.KeyGenerator;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.keygen.TestComplexKeyGenerator;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.keygen.constant.KeyGeneratorType;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import static org.junit.jupiter.api.Assertions.assertThrows;
/**
* This class assist test KeyGenerator configuration(class name and type) priority.
* <p>
* The functional test of KeyGenerator is left to other unit tests. {@link TestComplexKeyGenerator etc.}.
*/
public class TestHoodieSparkKeyGeneratorFactory {
@Test
public void testKeyGeneratorFactory() throws IOException {
TypedProperties props = getCommonProps();
// set KeyGenerator type only
props.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), KeyGeneratorType.SIMPLE.name());
KeyGenerator keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
Assertions.assertEquals(SimpleKeyGenerator.class.getName(), keyGenerator.getClass().getName());
// set KeyGenerator class only
props = getCommonProps();
props.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), SimpleKeyGenerator.class.getName());
KeyGenerator keyGenerator2 = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
Assertions.assertEquals(SimpleKeyGenerator.class.getName(), keyGenerator2.getClass().getName());
// set both class name and keyGenerator type
props.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), KeyGeneratorType.CUSTOM.name());
KeyGenerator keyGenerator3 = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
// KEYGENERATOR_TYPE_PROP was overwritten by KEYGENERATOR_CLASS_PROP
Assertions.assertEquals(SimpleKeyGenerator.class.getName(), keyGenerator3.getClass().getName());
// set wrong class name
final TypedProperties props2 = getCommonProps();
props2.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), TestHoodieSparkKeyGeneratorFactory.class.getName());
assertThrows(IOException.class, () -> HoodieSparkKeyGeneratorFactory.createKeyGenerator(props2));
// set wrong keyGenerator type
final TypedProperties props3 = getCommonProps();
props3.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), "wrong_type");
assertThrows(HoodieKeyGeneratorException.class, () -> HoodieSparkKeyGeneratorFactory.createKeyGenerator(props3));
}
private TypedProperties getCommonProps() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true");
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
return properties;
}
}

View File

@@ -18,13 +18,13 @@
package org.apache.hudi
import java.time.LocalDate
import org.apache.avro.Schema
import org.apache.avro.generic.GenericData
import org.apache.spark.sql.catalyst.expressions.GenericRow
import org.scalatest.{FunSuite, Matchers}
import java.time.LocalDate
class TestAvroConversionHelper extends FunSuite with Matchers {
val dateSchema = s"""
@@ -42,7 +42,7 @@ class TestAvroConversionHelper extends FunSuite with Matchers {
test("Logical type: date") {
val schema = new Schema.Parser().parse(dateSchema)
val convertor = AvroConversionHelper.createConverterToRow(schema, AvroConversionUtils.convertAvroSchemaToStructType(schema))
val convertor = AvroConversionUtils.createConverterToRow(schema, AvroConversionUtils.convertAvroSchemaToStructType(schema))
val dateOutputData = dateInputData.map(x => {
val record = new GenericData.Record(schema) {{ put("date", x) }}

View File

@@ -243,7 +243,7 @@ class TestDataSourceDefaults {
val partitionPathProp: String = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD.key)
val STRUCT_NAME: String = "hoodieRowTopLevelField"
val NAMESPACE: String = "hoodieRow"
var converterFn: Function1[Any, Any] = _
var converterFn: Function1[Row, GenericRecord] = _
override def getKey(record: GenericRecord): HoodieKey = {
new HoodieKey(HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyProp, true, false),
@@ -251,13 +251,13 @@ class TestDataSourceDefaults {
}
override def getRecordKey(row: Row): String = {
if (null == converterFn) converterFn = AvroConversionHelper.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE)
if (null == converterFn) converterFn = AvroConversionUtils.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE)
val genericRecord = converterFn.apply(row).asInstanceOf[GenericRecord]
getKey(genericRecord).getRecordKey
}
override def getPartitionPath(row: Row): String = {
if (null == converterFn) converterFn = AvroConversionHelper.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE)
if (null == converterFn) converterFn = AvroConversionUtils.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE)
val genericRecord = converterFn.apply(row).asInstanceOf[GenericRecord]
getKey(genericRecord).getPartitionPath
}

View File

@@ -32,13 +32,13 @@ import org.apache.hudi.functional.TestBootstrap
import org.apache.hudi.hive.HiveSyncConfig
import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator}
import org.apache.hudi.testutils.DataSourceTestUtils
import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.api.java.JavaSparkContext
import org.apache.spark.sql._
import org.apache.spark.sql.functions.{expr, lit}
import org.apache.spark.sql.hudi.HoodieSparkSessionExtension
import org.apache.spark.sql.hudi.command.SqlKeyGenerator
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
import org.apache.spark.{SparkConf, SparkContext}
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue, fail}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.junit.jupiter.params.ParameterizedTest
@@ -827,33 +827,32 @@ class TestHoodieSparkSqlWriter {
/**
* Test case for non partition table with metatable support.
*/
@Test
def testNonPartitionTableWithMetatableSupport(): Unit = {
List(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL).foreach { tableType =>
val options = Map(DataSourceWriteOptions.TABLE_TYPE.key -> tableType,
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "col3",
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "keyid",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "",
DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
"hoodie.insert.shuffle.parallelism" -> "1",
"hoodie.metadata.enable" -> "true")
val df = spark.range(0, 10).toDF("keyid")
.withColumn("col3", expr("keyid"))
.withColumn("age", expr("keyid + 1000"))
df.write.format("hudi")
.options(options.updated(DataSourceWriteOptions.OPERATION.key, "insert"))
.mode(SaveMode.Overwrite).save(tempBasePath)
// upsert same record again
val df_update = spark.range(0, 10).toDF("keyid")
.withColumn("col3", expr("keyid"))
.withColumn("age", expr("keyid + 2000"))
df_update.write.format("hudi")
.options(options.updated(DataSourceWriteOptions.OPERATION.key, "upsert"))
.mode(SaveMode.Append).save(tempBasePath)
assert(spark.read.format("hudi").load(tempBasePath).count() == 10)
assert(spark.read.format("hudi").load(tempBasePath).where("age >= 2000").count() == 10)
}
@ParameterizedTest
@EnumSource(value = classOf[HoodieTableType])
def testNonPartitionTableWithMetatableSupport(tableType: HoodieTableType): Unit = {
val options = Map(DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name,
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "col3",
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "keyid",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "",
DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
"hoodie.insert.shuffle.parallelism" -> "1",
"hoodie.metadata.enable" -> "true")
val df = spark.range(0, 10).toDF("keyid")
.withColumn("col3", expr("keyid"))
.withColumn("age", expr("keyid + 1000"))
df.write.format("hudi")
.options(options.updated(DataSourceWriteOptions.OPERATION.key, "insert"))
.mode(SaveMode.Overwrite).save(tempBasePath)
// upsert same record again
val df_update = spark.range(0, 10).toDF("keyid")
.withColumn("col3", expr("keyid"))
.withColumn("age", expr("keyid + 2000"))
df_update.write.format("hudi")
.options(options.updated(DataSourceWriteOptions.OPERATION.key, "upsert"))
.mode(SaveMode.Append).save(tempBasePath)
assert(spark.read.format("hudi").load(tempBasePath).count() == 10)
assert(spark.read.format("hudi").load(tempBasePath).where("age >= 2000").count() == 10)
}
/**

View File

@@ -20,22 +20,18 @@ package org.apache.hudi
import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hudi.exception.SchemaCompatibilityException
import org.apache.hudi.testutils.DataSourceTestUtils
import org.apache.spark.sql.types.{StructType, TimestampType}
import org.apache.spark.sql.{Row, SparkSession}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.junit.jupiter.api.io.TempDir
import java.io.File
import java.nio.file.Paths
import scala.collection.JavaConverters
class TestHoodieSparkUtils {
@@ -232,8 +228,9 @@ class TestHoodieSparkUtils {
fail("createRdd should fail, because records don't have a column which is not nullable in the passed in schema")
} catch {
case e: Exception =>
e.getCause.asInstanceOf[NullPointerException]
assertTrue(e.getMessage.contains("null of string in field new_nested_col of"))
val cause = e.getCause
assertTrue(cause.isInstanceOf[SchemaCompatibilityException])
assertTrue(e.getMessage.contains("Unable to validate the rewritten record {\"innerKey\": \"innerKey1_2\", \"innerValue\": 2} against schema"))
}
spark.stop()
}

View File

@@ -17,11 +17,11 @@
package org.apache.hudi.functional
import org.apache.avro.generic.GenericRecord
import org.apache.hadoop.fs.Path
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieTableType}
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord, HoodieRecordPayload, HoodieTableType}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
@@ -30,18 +30,18 @@ import org.apache.hudi.index.HoodieIndex.IndexType
import org.apache.hudi.keygen.NonpartitionedKeyGenerator
import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config
import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieClientTestBase}
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkUtils}
import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkUtils}
import org.apache.log4j.LogManager
import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types.BooleanType
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.CsvSource
import java.util
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
@@ -349,11 +349,15 @@ class TestMORDataSource extends HoodieClientTestBase {
// First Operation:
// Producing parquet files to three default partitions.
// SNAPSHOT view on MOR table with parquet files only.
// Overriding the partition-path field
val opts = commonOpts + (DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition_path")
val hoodieRecords1 = dataGen.generateInserts("001", 100)
val records1 = recordsToStrings(hoodieRecords1).toList
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
val inputDF1 = toDataset(hoodieRecords1)
inputDF1.write.format("org.apache.hudi")
.options(commonOpts)
.options(opts)
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
@@ -376,11 +380,10 @@ class TestMORDataSource extends HoodieClientTestBase {
// Second Operation:
// Upsert 50 update records
// Snopshot view should read 100 records
val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 50))
.toList
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
val records2 = dataGen.generateUniqueUpdates("002", 50)
val inputDF2 = toDataset(records2)
inputDF2.write.format("org.apache.hudi")
.options(commonOpts)
.options(opts)
.mode(SaveMode.Append)
.save(basePath)
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
@@ -424,17 +427,31 @@ class TestMORDataSource extends HoodieClientTestBase {
verifyShow(hudiIncDF2)
verifyShow(hudiIncDF1Skipmerge)
val record3 = recordsToStrings(dataGen.generateUpdatesWithTS("003", hoodieRecords1, -1))
spark.read.json(spark.sparkContext.parallelize(record3, 2))
.write.format("org.apache.hudi").options(commonOpts)
val record3 = dataGen.generateUpdatesWithTS("003", hoodieRecords1, -1)
val inputDF3 = toDataset(record3)
inputDF3.write.format("org.apache.hudi").options(opts)
.mode(SaveMode.Append).save(basePath)
val hudiSnapshotDF3 = spark.read.format("org.apache.hudi")
.option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
.load(basePath + "/*/*/*/*")
verifyShow(hudiSnapshotDF3);
assertEquals(100, hudiSnapshotDF3.count())
assertEquals(0, hudiSnapshotDF3.filter("rider = 'rider-003'").count())
}
private def toDataset(records: util.List[HoodieRecord[_]]) = {
val avroRecords = records.map(_.getData
.asInstanceOf[HoodieRecordPayload[_]]
.getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA)
.get
.asInstanceOf[GenericRecord])
val rdd: RDD[GenericRecord] = spark.sparkContext.parallelize(avroRecords, 2)
AvroConversionUtils.createDataFrame(rdd, HoodieTestDataGenerator.AVRO_SCHEMA.toString, spark)
}
@Test
def testVectorizedReader() {
spark.conf.set("spark.sql.parquet.enableVectorizedReader", true)
@@ -553,15 +570,10 @@ class TestMORDataSource extends HoodieClientTestBase {
.orderBy(desc("_hoodie_commit_time"))
.head()
assertEquals(sampleRow.getDouble(0), sampleRow.get(0))
assertEquals(sampleRow.getLong(1), sampleRow.get(1))
assertEquals(sampleRow.getDate(1), sampleRow.get(1))
assertEquals(sampleRow.getString(2), sampleRow.get(2))
assertEquals(sampleRow.getSeq(3), sampleRow.get(3))
if (HoodieSparkUtils.gteqSpark3_2) {
// Since Spark3.2, the `nation` column is parsed as String, not Struct.
assertEquals(sampleRow.getString(4), sampleRow.get(4))
} else {
assertEquals(sampleRow.getStruct(4), sampleRow.get(4))
}
assertEquals(sampleRow.getAs[Array[Byte]](4), sampleRow.get(4))
}
def verifyShow(df: DataFrame): Unit = {

View File

@@ -0,0 +1,98 @@
/*
* 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.functional
import org.apache.hudi.DataSourceWriteOptions
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.testutils.SparkClientFunctionalTestHarness
import org.apache.spark.sql.SaveMode
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{Tag, Test}
import scala.collection.JavaConverters._
@Tag("functional")
class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarness {
val hudi = "org.apache.hudi"
var commonOpts = Map(
"hoodie.insert.shuffle.parallelism" -> "4",
"hoodie.upsert.shuffle.parallelism" -> "4",
"hoodie.bulkinsert.shuffle.parallelism" -> "2",
"hoodie.delete.shuffle.parallelism" -> "1",
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
)
@Test
def testReadability(): Unit = {
val dataGen = new HoodieTestDataGenerator()
val opts: Map[String, String] = commonOpts ++ Map(
HoodieMetadataConfig.ENABLE.key -> "true",
HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1"
)
// Insert records
val newRecords = dataGen.generateInserts("001", 100)
val newRecordsDF = parseRecords(recordsToStrings(newRecords).asScala)
newRecordsDF.write.format(hudi)
.options(opts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.mode(SaveMode.Append)
.save(basePath)
// Update records
val updatedRecords = dataGen.generateUpdates("002", newRecords)
val updatedRecordsDF = parseRecords(recordsToStrings(updatedRecords).asScala)
updatedRecordsDF.write.format(hudi)
.options(opts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
.mode(SaveMode.Append)
.save(basePath)
val metadataDF = spark.read.format(hudi).load(s"$basePath/.hoodie/metadata")
// Smoke test
metadataDF.show()
// Query w/ 0 requested columns should be working fine
assertEquals(4, metadataDF.count())
val expectedKeys = Seq("2015/03/16", "2015/03/17", "2016/03/15", "__all_partitions__")
val keys = metadataDF.select("key")
.collect()
.map(_.getString(0))
.toSeq
.sorted
assertEquals(expectedKeys, keys)
}
private def parseRecords(records: Seq[String]) = {
spark.read.json(spark.sparkContext.parallelize(records, 2))
}
}