[HUDI-76] Add CSV Source support for Hudi Delta Streamer
This commit is contained in:
@@ -19,8 +19,8 @@
|
||||
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.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
||||
@@ -42,6 +42,7 @@ import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
|
||||
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Operation;
|
||||
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.hudi.utilities.sources.CsvDFSSource;
|
||||
import org.apache.hudi.utilities.sources.DistributedTestDataSource;
|
||||
import org.apache.hudi.utilities.sources.HoodieIncrSource;
|
||||
import org.apache.hudi.utilities.sources.InputBatch;
|
||||
@@ -60,6 +61,7 @@ import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.AnalysisException;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
@@ -98,12 +100,14 @@ 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_CSV = "test-csv-dfs-source.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 int CSV_NUM_RECORDS = 3;
|
||||
private static final Logger LOG = LogManager.getLogger(TestHoodieDeltaStreamer.class);
|
||||
|
||||
private static int parquetTestNum = 1;
|
||||
private static int testNum = 1;
|
||||
|
||||
@BeforeClass
|
||||
public static void initClass() throws Exception {
|
||||
@@ -114,7 +118,9 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/sql-transformer.properties", dfs,
|
||||
dfsBasePath + "/sql-transformer.properties");
|
||||
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source.avsc", dfs, dfsBasePath + "/source.avsc");
|
||||
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source-flattened.avsc", dfs, dfsBasePath + "/source-flattened.avsc");
|
||||
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/target.avsc", dfs, dfsBasePath + "/target.avsc");
|
||||
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/target-flattened.avsc", dfs, dfsBasePath + "/target-flattened.avsc");
|
||||
|
||||
TypedProperties props = new TypedProperties();
|
||||
props.setProperty("include", "sql-transformer.properties");
|
||||
@@ -197,12 +203,12 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
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);
|
||||
useSchemaProviderClass, 1000, updatePayloadClass, payloadClassName, tableType, "timestamp");
|
||||
}
|
||||
|
||||
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) {
|
||||
int sourceLimit, boolean updatePayloadClass, String payloadClassName, String tableType, String sourceOrderingField) {
|
||||
HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config();
|
||||
cfg.targetBasePath = basePath;
|
||||
cfg.targetTableName = "hoodie_trips";
|
||||
@@ -211,7 +217,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
cfg.transformerClassName = transformerClassName;
|
||||
cfg.operation = op;
|
||||
cfg.enableHiveSync = enableHiveSync;
|
||||
cfg.sourceOrderingField = "timestamp";
|
||||
cfg.sourceOrderingField = sourceOrderingField;
|
||||
cfg.propsFilePath = dfsBasePath + "/" + propsFilename;
|
||||
cfg.sourceLimit = sourceLimit;
|
||||
if (updatePayloadClass) {
|
||||
@@ -653,7 +659,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
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.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc");
|
||||
}
|
||||
}
|
||||
parquetProps.setProperty("hoodie.deltastreamer.source.dfs.root", PARQUET_SOURCE_ROOT);
|
||||
@@ -663,14 +669,14 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
|
||||
private void testParquetDFSSource(boolean useSchemaProvider, String transformerClassName) throws Exception {
|
||||
prepareParquetDFSSource(useSchemaProvider, transformerClassName != null);
|
||||
String tableBasePath = dfsBasePath + "/test_parquet_table" + parquetTestNum;
|
||||
String tableBasePath = dfsBasePath + "/test_parquet_table" + testNum;
|
||||
HoodieDeltaStreamer deltaStreamer = new HoodieDeltaStreamer(
|
||||
TestHelpers.makeConfig(tableBasePath, Operation.INSERT, ParquetDFSSource.class.getName(),
|
||||
transformerClassName, PROPS_FILENAME_TEST_PARQUET, false,
|
||||
useSchemaProvider, 100000, false, null, null), jsc);
|
||||
useSchemaProvider, 100000, false, null, null, "timestamp"), jsc);
|
||||
deltaStreamer.sync();
|
||||
TestHelpers.assertRecordCount(PARQUET_NUM_RECORDS, tableBasePath + "/*/*.parquet", sqlContext);
|
||||
parquetTestNum++;
|
||||
testNum++;
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -693,6 +699,146 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
testParquetDFSSource(true, TripsWithDistanceTransformer.class.getName());
|
||||
}
|
||||
|
||||
private void prepareCsvDFSSource(
|
||||
boolean hasHeader, char sep, boolean useSchemaProvider, boolean hasTransformer) throws IOException {
|
||||
String sourceRoot = dfsBasePath + "/csvFiles";
|
||||
String recordKeyField = (hasHeader || useSchemaProvider) ? "_row_key" : "_c0";
|
||||
|
||||
// Properties used for testing delta-streamer with CSV source
|
||||
TypedProperties csvProps = new TypedProperties();
|
||||
csvProps.setProperty("include", "base.properties");
|
||||
csvProps.setProperty("hoodie.datasource.write.recordkey.field", recordKeyField);
|
||||
csvProps.setProperty("hoodie.datasource.write.partitionpath.field", "not_there");
|
||||
if (useSchemaProvider) {
|
||||
csvProps.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source-flattened.avsc");
|
||||
if (hasTransformer) {
|
||||
csvProps.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target-flattened.avsc");
|
||||
}
|
||||
}
|
||||
csvProps.setProperty("hoodie.deltastreamer.source.dfs.root", sourceRoot);
|
||||
|
||||
if (sep != ',') {
|
||||
if (sep == '\t') {
|
||||
csvProps.setProperty("hoodie.deltastreamer.csv.sep", "\\t");
|
||||
} else {
|
||||
csvProps.setProperty("hoodie.deltastreamer.csv.sep", Character.toString(sep));
|
||||
}
|
||||
}
|
||||
if (hasHeader) {
|
||||
csvProps.setProperty("hoodie.deltastreamer.csv.header", Boolean.toString(hasHeader));
|
||||
}
|
||||
|
||||
UtilitiesTestBase.Helpers.savePropsToDFS(csvProps, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_CSV);
|
||||
|
||||
String path = sourceRoot + "/1.csv";
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
|
||||
UtilitiesTestBase.Helpers.saveCsvToDFS(
|
||||
hasHeader, sep,
|
||||
Helpers.jsonifyRecords(dataGenerator.generateInserts("000", CSV_NUM_RECORDS, true)),
|
||||
dfs, path);
|
||||
}
|
||||
|
||||
private void testCsvDFSSource(
|
||||
boolean hasHeader, char sep, boolean useSchemaProvider, String transformerClassName) throws Exception {
|
||||
prepareCsvDFSSource(hasHeader, sep, useSchemaProvider, transformerClassName != null);
|
||||
String tableBasePath = dfsBasePath + "/test_csv_table" + testNum;
|
||||
String sourceOrderingField = (hasHeader || useSchemaProvider) ? "timestamp" : "_c0";
|
||||
HoodieDeltaStreamer deltaStreamer =
|
||||
new HoodieDeltaStreamer(TestHelpers.makeConfig(
|
||||
tableBasePath, Operation.INSERT, CsvDFSSource.class.getName(),
|
||||
transformerClassName, PROPS_FILENAME_TEST_CSV, false,
|
||||
useSchemaProvider, 1000, false, null, null, sourceOrderingField), jsc);
|
||||
deltaStreamer.sync();
|
||||
TestHelpers.assertRecordCount(CSV_NUM_RECORDS, tableBasePath + "/*/*.parquet", sqlContext);
|
||||
testNum++;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCsvDFSSourceWithHeaderWithoutSchemaProviderAndNoTransformer() throws Exception {
|
||||
// The CSV files have header, the columns are separated by ',', the default separator
|
||||
// No schema provider is specified, no transformer is applied
|
||||
// In this case, the source schema comes from the inferred schema of the CSV files
|
||||
testCsvDFSSource(true, ',', false, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCsvDFSSourceWithHeaderAndSepWithoutSchemaProviderAndNoTransformer() throws Exception {
|
||||
// The CSV files have header, the columns are separated by '\t',
|
||||
// which is passed in through the Hudi CSV properties
|
||||
// No schema provider is specified, no transformer is applied
|
||||
// In this case, the source schema comes from the inferred schema of the CSV files
|
||||
testCsvDFSSource(true, '\t', false, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCsvDFSSourceWithHeaderAndSepWithSchemaProviderAndNoTransformer() throws Exception {
|
||||
// The CSV files have header, the columns are separated by '\t'
|
||||
// File schema provider is used, no transformer is applied
|
||||
// In this case, the source schema comes from the source Avro schema file
|
||||
testCsvDFSSource(true, '\t', true, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCsvDFSSourceWithHeaderAndSepWithoutSchemaProviderAndWithTransformer() throws Exception {
|
||||
// The CSV files have header, the columns are separated by '\t'
|
||||
// No schema provider is specified, transformer is applied
|
||||
// In this case, the source schema comes from the inferred schema of the CSV files.
|
||||
// Target schema is determined based on the Dataframe after transformation
|
||||
testCsvDFSSource(true, '\t', false, TripsWithDistanceTransformer.class.getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCsvDFSSourceWithHeaderAndSepWithSchemaProviderAndTransformer() throws Exception {
|
||||
// The CSV files have header, the columns are separated by '\t'
|
||||
// File schema provider is used, transformer is applied
|
||||
// In this case, the source and target schema come from the Avro schema files
|
||||
testCsvDFSSource(true, '\t', true, TripsWithDistanceTransformer.class.getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCsvDFSSourceNoHeaderWithoutSchemaProviderAndNoTransformer() throws Exception {
|
||||
// The CSV files do not have header, the columns are separated by '\t',
|
||||
// which is passed in through the Hudi CSV properties
|
||||
// No schema provider is specified, no transformer is applied
|
||||
// In this case, the source schema comes from the inferred schema of the CSV files
|
||||
// No CSV header and no schema provider at the same time are not recommended
|
||||
// as the column names are not informative
|
||||
testCsvDFSSource(false, '\t', false, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCsvDFSSourceNoHeaderWithSchemaProviderAndNoTransformer() throws Exception {
|
||||
// The CSV files do not have header, the columns are separated by '\t'
|
||||
// File schema provider is used, no transformer is applied
|
||||
// In this case, the source schema comes from the source Avro schema file
|
||||
testCsvDFSSource(false, '\t', true, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCsvDFSSourceNoHeaderWithoutSchemaProviderAndWithTransformer() throws Exception {
|
||||
// The CSV files do not have header, the columns are separated by '\t'
|
||||
// No schema provider is specified, transformer is applied
|
||||
// In this case, the source schema comes from the inferred schema of the CSV files.
|
||||
// Target schema is determined based on the Dataframe after transformation
|
||||
// No CSV header and no schema provider at the same time are not recommended,
|
||||
// as the transformer behavior may be unexpected
|
||||
try {
|
||||
testCsvDFSSource(false, '\t', false, TripsWithDistanceTransformer.class.getName());
|
||||
fail("Should error out when doing the transformation.");
|
||||
} catch (AnalysisException e) {
|
||||
LOG.error("Expected error during transformation", e);
|
||||
assertTrue(e.getMessage().contains("cannot resolve '`begin_lat`' given input columns:"));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCsvDFSSourceNoHeaderWithSchemaProviderAndTransformer() throws Exception {
|
||||
// The CSV files do not have header, the columns are separated by '\t'
|
||||
// File schema provider is used, transformer is applied
|
||||
// In this case, the source and target schema come from the Avro schema files
|
||||
testCsvDFSSource(false, '\t', true, TripsWithDistanceTransformer.class.getName());
|
||||
}
|
||||
|
||||
/**
|
||||
* UDF to calculate Haversine distance.
|
||||
*/
|
||||
|
||||
@@ -27,11 +27,20 @@ import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.HoodieHiveClient;
|
||||
import org.apache.hudi.hive.util.HiveTestService;
|
||||
import org.apache.hudi.utilities.sources.TestDataSource;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.ObjectWriter;
|
||||
import com.fasterxml.jackson.databind.node.ArrayNode;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import com.fasterxml.jackson.dataformat.csv.CsvMapper;
|
||||
import com.fasterxml.jackson.dataformat.csv.CsvSchema;
|
||||
import com.fasterxml.jackson.dataformat.csv.CsvSchema.Builder;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
@@ -42,6 +51,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hive.service.server.HiveServer2;
|
||||
import org.apache.parquet.avro.AvroParquetWriter;
|
||||
import org.apache.parquet.hadoop.ParquetFileWriter.Mode;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
@@ -56,6 +66,7 @@ import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.io.PrintStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
@@ -72,6 +83,7 @@ public class UtilitiesTestBase {
|
||||
protected transient SparkSession sparkSession = null;
|
||||
protected transient SQLContext sqlContext;
|
||||
protected static HiveServer2 hiveServer;
|
||||
private static ObjectMapper mapper = new ObjectMapper();
|
||||
|
||||
@BeforeClass
|
||||
public static void initClass() throws Exception {
|
||||
@@ -193,9 +205,47 @@ public class UtilitiesTestBase {
|
||||
os.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the json records into CSV format and writes to a file.
|
||||
*
|
||||
* @param hasHeader whether the CSV file should have a header line.
|
||||
* @param sep the column separator to use.
|
||||
* @param lines the records in JSON format.
|
||||
* @param fs {@link FileSystem} instance.
|
||||
* @param targetPath File path.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void saveCsvToDFS(
|
||||
boolean hasHeader, char sep,
|
||||
String[] lines, FileSystem fs, String targetPath) throws IOException {
|
||||
Builder csvSchemaBuilder = CsvSchema.builder();
|
||||
|
||||
ArrayNode arrayNode = mapper.createArrayNode();
|
||||
Arrays.stream(lines).forEachOrdered(
|
||||
line -> {
|
||||
try {
|
||||
arrayNode.add(mapper.readValue(line, ObjectNode.class));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Error converting json records into CSV format: " + e.getMessage());
|
||||
}
|
||||
});
|
||||
arrayNode.get(0).fieldNames().forEachRemaining(csvSchemaBuilder::addColumn);
|
||||
ObjectWriter csvObjWriter = new CsvMapper()
|
||||
.writerFor(JsonNode.class)
|
||||
.with(csvSchemaBuilder.setUseHeader(hasHeader).setColumnSeparator(sep).build());
|
||||
PrintStream os = new PrintStream(fs.create(new Path(targetPath), true));
|
||||
csvObjWriter.writeValue(os, arrayNode);
|
||||
os.flush();
|
||||
os.close();
|
||||
}
|
||||
|
||||
public static void saveParquetToDFS(List<GenericRecord> records, Path targetFile) throws IOException {
|
||||
try (ParquetWriter<GenericRecord> writer = AvroParquetWriter.<GenericRecord>builder(targetFile)
|
||||
.withSchema(HoodieTestDataGenerator.AVRO_SCHEMA).withConf(HoodieTestUtils.getDefaultHadoopConf()).build()) {
|
||||
.withSchema(HoodieTestDataGenerator.AVRO_SCHEMA)
|
||||
.withConf(HoodieTestUtils.getDefaultHadoopConf())
|
||||
.withWriteMode(Mode.OVERWRITE)
|
||||
.build()) {
|
||||
for (GenericRecord record : records) {
|
||||
writer.write(record);
|
||||
}
|
||||
@@ -203,9 +253,13 @@ public class UtilitiesTestBase {
|
||||
}
|
||||
|
||||
public static TypedProperties setupSchemaOnDFS() throws IOException {
|
||||
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source.avsc", dfs, dfsBasePath + "/source.avsc");
|
||||
return setupSchemaOnDFS("source.avsc");
|
||||
}
|
||||
|
||||
public static TypedProperties setupSchemaOnDFS(String filename) throws IOException {
|
||||
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/" + filename, dfs, dfsBasePath + "/" + filename);
|
||||
TypedProperties props = new TypedProperties();
|
||||
props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
|
||||
props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/" + filename);
|
||||
return props;
|
||||
}
|
||||
|
||||
|
||||
@@ -123,7 +123,7 @@ public abstract class AbstractBaseTestSource extends AvroSource {
|
||||
updateStream = dataGenerator.generateUniqueUpdatesStream(commitTime, numUpdates)
|
||||
.map(hr -> AbstractBaseTestSource.toGenericRecord(hr, dataGenerator));
|
||||
}
|
||||
Stream<GenericRecord> insertStream = dataGenerator.generateInsertsStream(commitTime, numInserts)
|
||||
Stream<GenericRecord> insertStream = dataGenerator.generateInsertsStream(commitTime, numInserts, false)
|
||||
.map(hr -> AbstractBaseTestSource.toGenericRecord(hr, dataGenerator));
|
||||
return Stream.concat(deleteStream, Stream.concat(updateStream, insertStream));
|
||||
}
|
||||
|
||||
@@ -56,6 +56,7 @@ public abstract class AbstractDFSSourceTestBase extends UtilitiesTestBase {
|
||||
String dfsRoot;
|
||||
String fileSuffix;
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
|
||||
boolean useFlattenedSchema = false;
|
||||
|
||||
@BeforeClass
|
||||
public static void initClass() throws Exception {
|
||||
@@ -105,7 +106,7 @@ public abstract class AbstractDFSSourceTestBase extends UtilitiesTestBase {
|
||||
*/
|
||||
Path generateOneFile(String filename, String commitTime, int n) throws IOException {
|
||||
Path path = new Path(dfsRoot, filename + fileSuffix);
|
||||
writeNewDataToFile(dataGenerator.generateInserts(commitTime, n), path);
|
||||
writeNewDataToFile(dataGenerator.generateInserts(commitTime, n, useFlattenedSchema), path);
|
||||
return path;
|
||||
}
|
||||
|
||||
|
||||
@@ -0,0 +1,61 @@
|
||||
/*
|
||||
* 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.utilities.sources;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.utilities.UtilitiesTestBase;
|
||||
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Basic tests for {@link CsvDFSSource}.
|
||||
*/
|
||||
public class TestCsvDFSSource extends AbstractDFSSourceTestBase {
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
super.setup();
|
||||
this.dfsRoot = dfsBasePath + "/jsonFiles";
|
||||
this.fileSuffix = ".json";
|
||||
this.useFlattenedSchema = true;
|
||||
this.schemaProvider = new FilebasedSchemaProvider(
|
||||
Helpers.setupSchemaOnDFS("source-flattened.avsc"), jsc);
|
||||
}
|
||||
|
||||
@Override
|
||||
Source prepareDFSSource() {
|
||||
TypedProperties props = new TypedProperties();
|
||||
props.setProperty("hoodie.deltastreamer.source.dfs.root", dfsRoot);
|
||||
props.setProperty("hoodie.deltastreamer.csv.header", Boolean.toString(true));
|
||||
props.setProperty("hoodie.deltastreamer.csv.sep", "\t");
|
||||
return new CsvDFSSource(props, jsc, sparkSession, schemaProvider);
|
||||
}
|
||||
|
||||
@Override
|
||||
void writeNewDataToFile(List<HoodieRecord> records, Path path) throws IOException {
|
||||
UtilitiesTestBase.Helpers.saveCsvToDFS(
|
||||
true, '\t', Helpers.jsonifyRecords(records), dfs, path.toString());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,57 @@
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
{
|
||||
"type" : "record",
|
||||
"name" : "triprec",
|
||||
"fields" : [
|
||||
{
|
||||
"name" : "timestamp",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "_row_key",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "rider",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "driver",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "begin_lat",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "begin_lon",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "end_lat",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "end_lon",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "fare",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "currency",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "_hoodie_is_deleted",
|
||||
"type" : "boolean",
|
||||
"default" : false
|
||||
} ]
|
||||
}
|
||||
@@ -0,0 +1,60 @@
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
{
|
||||
"type" : "record",
|
||||
"name" : "triprec",
|
||||
"fields" : [
|
||||
{
|
||||
"name" : "timestamp",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "_row_key",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "rider",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "driver",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "begin_lat",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "begin_lon",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "end_lat",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "end_lon",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "fare",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "currency",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "_hoodie_is_deleted",
|
||||
"type" : "boolean",
|
||||
"default" : false
|
||||
}, {
|
||||
"name" : "haversine_distance",
|
||||
"type" : "double"
|
||||
}]
|
||||
}
|
||||
Reference in New Issue
Block a user