[HUDI-552] Fix the schema mismatch in Row-to-Avro conversion (#1246)
This commit is contained in:
committed by
vinoth chandar
parent
3f4966ddea
commit
d0ee95ed16
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.utilities;
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.keygen.SimpleKeyGenerator;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
@@ -44,6 +45,7 @@ import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.hudi.utilities.sources.DistributedTestDataSource;
|
||||
import org.apache.hudi.utilities.sources.HoodieIncrSource;
|
||||
import org.apache.hudi.utilities.sources.InputBatch;
|
||||
import org.apache.hudi.utilities.sources.ParquetDFSSource;
|
||||
import org.apache.hudi.utilities.sources.TestDataSource;
|
||||
import org.apache.hudi.utilities.sources.config.TestSourceConfig;
|
||||
import org.apache.hudi.utilities.transform.SqlQueryBasedTransformer;
|
||||
@@ -96,8 +98,13 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
private static final Random RANDOM = new Random();
|
||||
private static final String PROPS_FILENAME_TEST_SOURCE = "test-source.properties";
|
||||
private static final String PROPS_FILENAME_TEST_INVALID = "test-invalid.properties";
|
||||
private static final String PROPS_FILENAME_TEST_PARQUET = "test-parquet-dfs-source.properties";
|
||||
private static final String PARQUET_SOURCE_ROOT = dfsBasePath + "/parquetFiles";
|
||||
private static final int PARQUET_NUM_RECORDS = 5;
|
||||
private static final Logger LOG = LogManager.getLogger(TestHoodieDeltaStreamer.class);
|
||||
|
||||
private static int parquetTestNum = 1;
|
||||
|
||||
@BeforeClass
|
||||
public static void initClass() throws Exception {
|
||||
UtilitiesTestBase.initClass(true);
|
||||
@@ -146,6 +153,8 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
invalidProps.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
|
||||
invalidProps.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc");
|
||||
UtilitiesTestBase.Helpers.savePropsToDFS(invalidProps, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_INVALID);
|
||||
|
||||
prepareParquetDFSFiles(PARQUET_NUM_RECORDS);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
@@ -186,17 +195,24 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
static HoodieDeltaStreamer.Config makeConfig(String basePath, Operation op, String transformerClassName,
|
||||
String propsFilename, boolean enableHiveSync, boolean useSchemaProviderClass, boolean updatePayloadClass,
|
||||
String payloadClassName, String tableType) {
|
||||
return makeConfig(basePath, op, TestDataSource.class.getName(), transformerClassName, propsFilename, enableHiveSync,
|
||||
useSchemaProviderClass, 1000, updatePayloadClass, payloadClassName, tableType);
|
||||
}
|
||||
|
||||
static HoodieDeltaStreamer.Config makeConfig(String basePath, Operation op, String sourceClassName,
|
||||
String transformerClassName, String propsFilename, boolean enableHiveSync, boolean useSchemaProviderClass,
|
||||
int sourceLimit, boolean updatePayloadClass, String payloadClassName, String tableType) {
|
||||
HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config();
|
||||
cfg.targetBasePath = basePath;
|
||||
cfg.targetTableName = "hoodie_trips";
|
||||
cfg.tableType = tableType == null ? "COPY_ON_WRITE" : tableType;
|
||||
cfg.sourceClassName = TestDataSource.class.getName();
|
||||
cfg.sourceClassName = sourceClassName;
|
||||
cfg.transformerClassName = transformerClassName;
|
||||
cfg.operation = op;
|
||||
cfg.enableHiveSync = enableHiveSync;
|
||||
cfg.sourceOrderingField = "timestamp";
|
||||
cfg.propsFilePath = dfsBasePath + "/" + propsFilename;
|
||||
cfg.sourceLimit = 1000;
|
||||
cfg.sourceLimit = sourceLimit;
|
||||
if (updatePayloadClass) {
|
||||
cfg.payloadClassName = payloadClassName;
|
||||
}
|
||||
@@ -620,6 +636,62 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
Assert.assertEquals(1000, c);
|
||||
}
|
||||
|
||||
private static void prepareParquetDFSFiles(int numRecords) throws IOException {
|
||||
String path = PARQUET_SOURCE_ROOT + "/1.parquet";
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
|
||||
Helpers.saveParquetToDFS(Helpers.toGenericRecords(
|
||||
dataGenerator.generateInserts("000", numRecords), dataGenerator), new Path(path));
|
||||
}
|
||||
|
||||
private void prepareParquetDFSSource(boolean useSchemaProvider, boolean hasTransformer) throws IOException {
|
||||
// Properties used for testing delta-streamer with Parquet source
|
||||
TypedProperties parquetProps = new TypedProperties();
|
||||
parquetProps.setProperty("include", "base.properties");
|
||||
parquetProps.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
|
||||
parquetProps.setProperty("hoodie.datasource.write.partitionpath.field", "not_there");
|
||||
if (useSchemaProvider) {
|
||||
parquetProps.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
|
||||
if (hasTransformer) {
|
||||
parquetProps.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/target.avsc");
|
||||
}
|
||||
}
|
||||
parquetProps.setProperty("hoodie.deltastreamer.source.dfs.root", PARQUET_SOURCE_ROOT);
|
||||
|
||||
UtilitiesTestBase.Helpers.savePropsToDFS(parquetProps, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_PARQUET);
|
||||
}
|
||||
|
||||
private void testParquetDFSSource(boolean useSchemaProvider, String transformerClassName) throws Exception {
|
||||
prepareParquetDFSSource(useSchemaProvider, transformerClassName != null);
|
||||
String tableBasePath = dfsBasePath + "/test_parquet_table" + parquetTestNum;
|
||||
HoodieDeltaStreamer deltaStreamer = new HoodieDeltaStreamer(
|
||||
TestHelpers.makeConfig(tableBasePath, Operation.INSERT, ParquetDFSSource.class.getName(),
|
||||
transformerClassName, PROPS_FILENAME_TEST_PARQUET, false,
|
||||
useSchemaProvider, 100000, false, null, null), jsc);
|
||||
deltaStreamer.sync();
|
||||
TestHelpers.assertRecordCount(PARQUET_NUM_RECORDS, tableBasePath + "/*/*.parquet", sqlContext);
|
||||
parquetTestNum++;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParquetDFSSourceWithoutSchemaProviderAndNoTransformer() throws Exception {
|
||||
testParquetDFSSource(false, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParquetDFSSourceWithoutSchemaProviderAndTransformer() throws Exception {
|
||||
testParquetDFSSource(false, TripsWithDistanceTransformer.class.getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParquetDFSSourceWithSourceSchemaFileAndNoTransformer() throws Exception {
|
||||
testParquetDFSSource(true, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParquetDFSSourceWithSchemaFilesAndTransformer() throws Exception {
|
||||
testParquetDFSSource(true, TripsWithDistanceTransformer.class.getName());
|
||||
}
|
||||
|
||||
/**
|
||||
* UDF to calculate Haversine distance.
|
||||
*/
|
||||
|
||||
Reference in New Issue
Block a user