[HUDI-2161] Adding support to disable meta columns with bulk insert operation (#3247)
This commit is contained in:
committed by
GitHub
parent
2099bf41db
commit
d5026e9a24
@@ -16,7 +16,7 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
package org.apache.hudi.io.storage.row;
|
||||
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
@@ -16,7 +16,7 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io;
|
||||
package org.apache.hudi.io.storage.row;
|
||||
|
||||
import org.apache.hudi.client.HoodieInternalWriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.keygen;
|
||||
|
||||
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;
|
||||
@@ -27,6 +28,7 @@ import org.apache.avro.generic.GenericRecord;
|
||||
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;
|
||||
|
||||
@@ -95,6 +97,9 @@ public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
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
|
||||
@@ -112,6 +117,12 @@ public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
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
|
||||
@@ -131,6 +142,13 @@ public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
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
|
||||
@@ -150,5 +168,12 @@ public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -28,6 +28,7 @@ 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;
|
||||
|
||||
@@ -141,6 +142,8 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
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
|
||||
@@ -164,6 +167,8 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
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
|
||||
@@ -187,6 +192,9 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
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
|
||||
@@ -335,6 +343,9 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
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
|
||||
@@ -359,5 +370,8 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
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");
|
||||
}
|
||||
}
|
||||
|
||||
@@ -21,11 +21,12 @@ package org.apache.hudi.keygen;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
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;
|
||||
@@ -130,6 +131,9 @@ public class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities {
|
||||
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() {
|
||||
|
||||
@@ -28,7 +28,10 @@ import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
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;
|
||||
@@ -50,6 +53,7 @@ public class TestTimestampBasedKeyGenerator {
|
||||
private Schema schema;
|
||||
private StructType structType;
|
||||
private Row baseRow;
|
||||
private InternalRow internalRow;
|
||||
|
||||
@BeforeEach
|
||||
public void initialize() throws IOException {
|
||||
@@ -58,6 +62,7 @@ public class TestTimestampBasedKeyGenerator {
|
||||
baseRecord = SchemaTestUtil
|
||||
.generateAvroRecordFromJson(schema, 1, "001", "f1");
|
||||
baseRow = genericRecordToRow(baseRecord);
|
||||
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
|
||||
|
||||
properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "field1");
|
||||
properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "createTime");
|
||||
@@ -77,7 +82,7 @@ public class TestTimestampBasedKeyGenerator {
|
||||
}
|
||||
|
||||
private Row genericRecordToRow(GenericRecord baseRecord) {
|
||||
Function1<Object, Object> convertor = AvroConversionHelper.createConverterToRow(schema, structType);
|
||||
Function1<Object, Object> convertor = AvroConversionHelper.createConverterToRow(baseRecord.getSchema(), structType);
|
||||
Row row = (Row) convertor.apply(baseRecord);
|
||||
int fieldCount = structType.fieldNames().length;
|
||||
Object[] values = new Object[fieldCount];
|
||||
@@ -117,6 +122,10 @@ public class TestTimestampBasedKeyGenerator {
|
||||
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
|
||||
baseRecord.put("createTime", new BigDecimal(1578283932000.00001));
|
||||
@@ -169,6 +178,8 @@ public class TestTimestampBasedKeyGenerator {
|
||||
// test w/ Row
|
||||
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("createTime", null);
|
||||
@@ -181,6 +192,8 @@ public class TestTimestampBasedKeyGenerator {
|
||||
// test w/ Row
|
||||
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
|
||||
@@ -197,6 +210,8 @@ public class TestTimestampBasedKeyGenerator {
|
||||
// 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);
|
||||
@@ -208,6 +223,8 @@ public class TestTimestampBasedKeyGenerator {
|
||||
// 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", 18736);
|
||||
@@ -216,6 +233,9 @@ public class TestTimestampBasedKeyGenerator {
|
||||
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
|
||||
|
||||
@@ -107,6 +107,11 @@ public class HoodieClientTestUtils {
|
||||
|
||||
public static Dataset<Row> readCommit(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
|
||||
String instantTime) {
|
||||
return readCommit(basePath, sqlContext, commitTimeline, instantTime, true);
|
||||
}
|
||||
|
||||
public static Dataset<Row> readCommit(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
|
||||
String instantTime, boolean filterByCommitTime) {
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime);
|
||||
if (!commitTimeline.containsInstant(commitInstant)) {
|
||||
throw new HoodieException("No commit exists at " + instantTime);
|
||||
@@ -115,14 +120,21 @@ public class HoodieClientTestUtils {
|
||||
HashMap<String, String> paths =
|
||||
getLatestFileIDsToFullPath(basePath, commitTimeline, Arrays.asList(commitInstant));
|
||||
LOG.info("Path :" + paths.values());
|
||||
Dataset<Row> unFilteredRows = null;
|
||||
if (HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().equals(HoodieFileFormat.PARQUET)) {
|
||||
return sqlContext.read().parquet(paths.values().toArray(new String[paths.size()]))
|
||||
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, instantTime));
|
||||
unFilteredRows = sqlContext.read().parquet(paths.values().toArray(new String[paths.size()]));
|
||||
} else if (HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().equals(HoodieFileFormat.ORC)) {
|
||||
return sqlContext.read().orc(paths.values().toArray(new String[paths.size()]))
|
||||
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, instantTime));
|
||||
unFilteredRows = sqlContext.read().orc(paths.values().toArray(new String[paths.size()]));
|
||||
}
|
||||
if (unFilteredRows != null) {
|
||||
if (filterByCommitTime) {
|
||||
return unFilteredRows.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, instantTime));
|
||||
} else {
|
||||
return unFilteredRows;
|
||||
}
|
||||
} else {
|
||||
return sqlContext.emptyDataFrame();
|
||||
}
|
||||
return sqlContext.emptyDataFrame();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error reading commit " + instantTime, e);
|
||||
}
|
||||
|
||||
@@ -24,11 +24,24 @@ import org.apache.hudi.AvroConversionUtils;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.package$;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
|
||||
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
|
||||
import org.apache.spark.sql.catalyst.encoders.RowEncoder;
|
||||
import org.apache.spark.sql.catalyst.expressions.Attribute;
|
||||
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Function1;
|
||||
import scala.collection.JavaConversions;
|
||||
import scala.collection.JavaConverters;
|
||||
|
||||
public class KeyGeneratorTestUtilities {
|
||||
|
||||
@@ -60,7 +73,7 @@ public class KeyGeneratorTestUtilities {
|
||||
|
||||
public static GenericRecord getRecord(GenericRecord nestedColRecord) {
|
||||
GenericRecord record = new GenericData.Record(new Schema.Parser().parse(EXAMPLE_SCHEMA));
|
||||
record.put("timestamp", 4357686);
|
||||
record.put("timestamp", 4357686L);
|
||||
record.put("_row_key", "key1");
|
||||
record.put("ts_ms", "2020-03-21");
|
||||
record.put("pii_col", "pi");
|
||||
@@ -82,4 +95,39 @@ public class KeyGeneratorTestUtilities {
|
||||
}
|
||||
return new GenericRowWithSchema(values, structType);
|
||||
}
|
||||
|
||||
public static InternalRow getInternalRow(Row row) {
|
||||
try {
|
||||
return getInternalRow(row, getEncoder(row.schema()));
|
||||
} catch (Exception e) {
|
||||
throw new IllegalStateException("Exception thrown while converting Row to InternalRow", e);
|
||||
}
|
||||
}
|
||||
|
||||
private static ExpressionEncoder getEncoder(StructType schema) {
|
||||
List<Attribute> attributes = JavaConversions.asJavaCollection(schema.toAttributes()).stream()
|
||||
.map(Attribute::toAttribute).collect(Collectors.toList());
|
||||
return RowEncoder.apply(schema)
|
||||
.resolveAndBind(JavaConverters.asScalaBufferConverter(attributes).asScala().toSeq(),
|
||||
SimpleAnalyzer$.MODULE$);
|
||||
}
|
||||
|
||||
public static InternalRow getInternalRow(Row row, ExpressionEncoder<Row> encoder) throws ClassNotFoundException, InvocationTargetException, IllegalAccessException, NoSuchMethodException {
|
||||
return serializeRow(encoder, row);
|
||||
}
|
||||
|
||||
private static InternalRow serializeRow(ExpressionEncoder encoder, Row row)
|
||||
throws InvocationTargetException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException {
|
||||
// TODO remove reflection if Spark 2.x support is dropped
|
||||
if (package$.MODULE$.SPARK_VERSION().startsWith("2.")) {
|
||||
Method spark2method = encoder.getClass().getMethod("toRow", Object.class);
|
||||
return (InternalRow) spark2method.invoke(encoder, row);
|
||||
} else {
|
||||
Class<?> serializerClass = Class.forName("org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer");
|
||||
Object serializer = encoder.getClass().getMethod("createSerializer").invoke(encoder);
|
||||
Method aboveSpark2method = serializerClass.getMethod("apply", Object.class);
|
||||
return (InternalRow) aboveSpark2method.invoke(serializer, row);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -59,12 +59,17 @@ import static org.apache.hudi.common.testutils.FileSystemTestUtils.RANDOM;
|
||||
*/
|
||||
public class SparkDatasetTestUtils {
|
||||
|
||||
public static final String RECORD_KEY_FIELD_NAME = "record_key";
|
||||
public static final String PARTITION_PATH_FIELD_NAME = "partition_path";
|
||||
|
||||
public static final StructType STRUCT_TYPE = new StructType(new StructField[] {
|
||||
new StructField(HoodieRecord.COMMIT_TIME_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.RECORD_KEY_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.FILENAME_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(RECORD_KEY_FIELD_NAME, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(PARTITION_PATH_FIELD_NAME, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField("randomInt", DataTypes.IntegerType, false, Metadata.empty()),
|
||||
new StructField("randomLong", DataTypes.LongType, false, Metadata.empty())});
|
||||
|
||||
@@ -74,6 +79,8 @@ public class SparkDatasetTestUtils {
|
||||
new StructField(HoodieRecord.RECORD_KEY_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.FILENAME_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(RECORD_KEY_FIELD_NAME, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(PARTITION_PATH_FIELD_NAME, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField("randomInt", DataTypes.IntegerType, false, Metadata.empty()),
|
||||
new StructField("randomStr", DataTypes.StringType, false, Metadata.empty())});
|
||||
|
||||
@@ -117,7 +124,7 @@ public class SparkDatasetTestUtils {
|
||||
*/
|
||||
public static Row getRandomValue(String partitionPath, boolean isError) {
|
||||
// order commit time, seq no, record key, partition path, file name
|
||||
Object[] values = new Object[7];
|
||||
Object[] values = new Object[9];
|
||||
values[0] = ""; //commit time
|
||||
if (!isError) {
|
||||
values[1] = ""; // commit seq no
|
||||
@@ -127,11 +134,13 @@ public class SparkDatasetTestUtils {
|
||||
values[2] = UUID.randomUUID().toString();
|
||||
values[3] = partitionPath;
|
||||
values[4] = ""; // filename
|
||||
values[5] = RANDOM.nextInt();
|
||||
values[5] = UUID.randomUUID().toString();
|
||||
values[6] = partitionPath;
|
||||
values[7] = RANDOM.nextInt();
|
||||
if (!isError) {
|
||||
values[6] = RANDOM.nextLong();
|
||||
values[8] = RANDOM.nextLong();
|
||||
} else {
|
||||
values[6] = UUID.randomUUID().toString();
|
||||
values[8] = UUID.randomUUID().toString();
|
||||
}
|
||||
return new GenericRow(values);
|
||||
}
|
||||
@@ -154,14 +163,16 @@ public class SparkDatasetTestUtils {
|
||||
public static InternalRow getInternalRowWithError(String partitionPath) {
|
||||
// order commit time, seq no, record key, partition path, file name
|
||||
String recordKey = UUID.randomUUID().toString();
|
||||
Object[] values = new Object[7];
|
||||
Object[] values = new Object[9];
|
||||
values[0] = "";
|
||||
values[1] = "";
|
||||
values[2] = recordKey;
|
||||
values[3] = partitionPath;
|
||||
values[4] = "";
|
||||
values[5] = RANDOM.nextInt();
|
||||
values[6] = RANDOM.nextBoolean();
|
||||
values[5] = recordKey;
|
||||
values[6] = partitionPath;
|
||||
values[7] = RANDOM.nextInt();
|
||||
values[8] = RANDOM.nextBoolean();
|
||||
return new GenericInternalRow(values);
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user