1
0

[HUDI-1129] Improving schema evolution support in hudi (#2927)

* Adding support to ingest records with old schema after table's schema is evolved

* Rebasing against latest master

- Trimming test file to be < 800 lines
- Renaming config names

* Addressing feedback

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
Sivabalan Narayanan
2021-08-10 12:15:37 -04:00
committed by GitHub
parent 73d898322b
commit 1196736185
22 changed files with 778 additions and 213 deletions

View File

@@ -24,7 +24,11 @@ import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.config.DFSPropertiesConfiguration;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.util.Functions.Function1;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.StringUtils;
@@ -38,12 +42,12 @@ import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.utilities.checkpointing.InitialCheckPointProvider;
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics;
import org.apache.hudi.utilities.schema.DelegatingSchemaProvider;
import org.apache.hudi.utilities.schema.RowBasedSchemaProvider;
import org.apache.hudi.utilities.schema.SchemaPostProcessor;
import org.apache.hudi.utilities.schema.SchemaPostProcessor.Config;
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.schema.SchemaProviderWithPostProcessor;
import org.apache.hudi.utilities.schema.SparkAvroPostProcessor;
import org.apache.hudi.utilities.schema.RowBasedSchemaProvider;
import org.apache.hudi.utilities.sources.Source;
import org.apache.hudi.utilities.transform.ChainedTransformer;
import org.apache.hudi.utilities.transform.Transformer;
@@ -425,11 +429,53 @@ public class UtilHelpers {
}
public static SchemaProvider createRowBasedSchemaProvider(StructType structType,
TypedProperties cfg, JavaSparkContext jssc) {
TypedProperties cfg, JavaSparkContext jssc) {
SchemaProvider rowSchemaProvider = new RowBasedSchemaProvider(structType);
return wrapSchemaProviderWithPostProcessor(rowSchemaProvider, cfg, jssc, null);
}
/**
* Create latest schema provider for Target schema.
*
* @param structType spark data type of incoming batch.
* @param jssc instance of {@link JavaSparkContext}.
* @param fs instance of {@link FileSystem}.
* @param basePath base path of the table.
* @return the schema provider where target schema refers to latest schema(either incoming schema or table schema).
*/
public static SchemaProvider createLatestSchemaProvider(StructType structType,
JavaSparkContext jssc, FileSystem fs, String basePath) {
SchemaProvider rowSchemaProvider = new RowBasedSchemaProvider(structType);
Schema writeSchema = rowSchemaProvider.getTargetSchema();
Schema latestTableSchema = writeSchema;
try {
if (FSUtils.isTableExists(basePath, fs)) {
HoodieTableMetaClient tableMetaClient = HoodieTableMetaClient.builder().setConf(jssc.sc().hadoopConfiguration()).setBasePath(basePath).build();
TableSchemaResolver
tableSchemaResolver = new TableSchemaResolver(tableMetaClient);
latestTableSchema = tableSchemaResolver.getLatestSchema(writeSchema, true, (Function1<Schema, Schema>) v1 -> AvroConversionUtils.convertStructTypeToAvroSchema(
AvroConversionUtils.convertAvroSchemaToStructType(v1), RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME,
RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE));
}
} catch (IOException e) {
LOG.warn("Could not fetch table schema. Falling back to writer schema");
}
final Schema finalLatestTableSchema = latestTableSchema;
return new SchemaProvider(new TypedProperties()) {
@Override
public Schema getSourceSchema() {
return rowSchemaProvider.getSourceSchema();
}
@Override
public Schema getTargetSchema() {
return finalLatestTableSchema;
}
};
}
@FunctionalInterface
public interface CheckedSupplier<T> {
T get() throws Throwable;

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.utilities.deltastreamer;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.HoodieSparkUtils;
import org.apache.hudi.HoodieWriterUtils;
import org.apache.hudi.avro.HoodieAvroUtils;
@@ -48,6 +49,7 @@ import org.apache.hudi.config.HoodieClusteringConfig;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodiePayloadConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieDeltaStreamerException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.hive.HiveSyncTool;
@@ -55,10 +57,9 @@ import org.apache.hudi.keygen.KeyGenerator;
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
import org.apache.hudi.sync.common.AbstractSyncTool;
import org.apache.hudi.utilities.UtilHelpers;
import org.apache.hudi.exception.HoodieDeltaStreamerException;
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
import org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallback;
import org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig;
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
import org.apache.hudi.utilities.schema.DelegatingSchemaProvider;
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.schema.SchemaSet;
@@ -86,26 +87,26 @@ import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.function.Function;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Objects;
import java.util.Properties;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;
import scala.collection.JavaConversions;
import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP;
import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE;
import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY;
import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_RESET_KEY;
import static org.apache.hudi.config.HoodieClusteringConfig.INLINE_CLUSTERING_PROP;
import static org.apache.hudi.config.HoodieCompactionConfig.INLINE_COMPACT_PROP;
import static org.apache.hudi.config.HoodieWriteConfig.COMBINE_BEFORE_INSERT_PROP;
import static org.apache.hudi.config.HoodieWriteConfig.COMBINE_BEFORE_UPSERT_PROP;
import static org.apache.hudi.config.HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP;
import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY;
import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_RESET_KEY;
import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE;
import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME;
@@ -365,7 +366,7 @@ public class DeltaSync implements Serializable {
final Option<JavaRDD<GenericRecord>> avroRDDOptional;
final String checkpointStr;
final SchemaProvider schemaProvider;
SchemaProvider schemaProvider;
if (transformer.isPresent()) {
// Transformation is needed. Fetch New rows in Row Format, apply transformation and then convert them
// to generic records for writing
@@ -374,28 +375,40 @@ public class DeltaSync implements Serializable {
Option<Dataset<Row>> transformed =
dataAndCheckpoint.getBatch().map(data -> transformer.get().apply(jssc, sparkSession, data, props));
checkpointStr = dataAndCheckpoint.getCheckpointForNextBatch();
boolean reconcileSchema = props.getBoolean(DataSourceWriteOptions.RECONCILE_SCHEMA().key());
if (this.userProvidedSchemaProvider != null && this.userProvidedSchemaProvider.getTargetSchema() != null) {
// If the target schema is specified through Avro schema,
// pass in the schema for the Row-to-Avro conversion
// to avoid nullability mismatch between Avro schema and Row schema
avroRDDOptional = transformed
.map(t -> HoodieSparkUtils.createRdd(
t, this.userProvidedSchemaProvider.getTargetSchema(),
HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD());
t, HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE, reconcileSchema,
Option.of(this.userProvidedSchemaProvider.getTargetSchema())
).toJavaRDD());
schemaProvider = this.userProvidedSchemaProvider;
} else {
// Use Transformed Row's schema if not overridden. If target schema is not specified
// default to RowBasedSchemaProvider
schemaProvider =
transformed
.map(r -> (SchemaProvider) new DelegatingSchemaProvider(props, jssc,
dataAndCheckpoint.getSchemaProvider(),
UtilHelpers.createRowBasedSchemaProvider(r.schema(), props, jssc)))
.map(r -> {
// determine the targetSchemaProvider. use latestTableSchema if reconcileSchema is enabled.
SchemaProvider targetSchemaProvider = null;
if (reconcileSchema) {
targetSchemaProvider = UtilHelpers.createLatestSchemaProvider(r.schema(), jssc, fs, cfg.targetBasePath);
} else {
targetSchemaProvider = UtilHelpers.createRowBasedSchemaProvider(r.schema(), props, jssc);
}
return (SchemaProvider) new DelegatingSchemaProvider(props, jssc,
dataAndCheckpoint.getSchemaProvider(), targetSchemaProvider); })
.orElse(dataAndCheckpoint.getSchemaProvider());
avroRDDOptional = transformed
.map(t -> HoodieSparkUtils.createRdd(
t, HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD());
t, HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE, reconcileSchema,
Option.ofNullable(schemaProvider.getTargetSchema())
).toJavaRDD());
}
} else {
// Pull the data from the source & prepare the write

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.utilities.deltastreamer;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.async.AsyncClusteringService;
import org.apache.hudi.async.AsyncCompactService;
import org.apache.hudi.async.HoodieAsyncService;
@@ -120,13 +121,13 @@ public class HoodieDeltaStreamer implements Serializable {
Option<TypedProperties> props) throws IOException {
// Resolving the properties first in a consistent way
if (props.isPresent()) {
this.properties = props.get();
this.properties = setDefaults(props.get());
} else if (cfg.propsFilePath.equals(Config.DEFAULT_DFS_SOURCE_PROPERTIES)) {
this.properties = UtilHelpers.getConfig(cfg.configs).getConfig();
this.properties = setDefaults(UtilHelpers.getConfig(cfg.configs).getConfig());
} else {
this.properties = UtilHelpers.readConfig(
this.properties = setDefaults(UtilHelpers.readConfig(
FSUtils.getFs(cfg.propsFilePath, jssc.hadoopConfiguration()),
new Path(cfg.propsFilePath), cfg.configs).getConfig();
new Path(cfg.propsFilePath), cfg.configs).getConfig());
}
if (cfg.initialCheckpointProvider != null && cfg.checkpoint == null) {
@@ -146,6 +147,13 @@ public class HoodieDeltaStreamer implements Serializable {
deltaSyncService.ifPresent(ds -> ds.shutdown(false));
}
private TypedProperties setDefaults(TypedProperties props) {
if (!props.containsKey(DataSourceWriteOptions.RECONCILE_SCHEMA().key())) {
props.setProperty(DataSourceWriteOptions.RECONCILE_SCHEMA().key(), DataSourceWriteOptions.RECONCILE_SCHEMA().defaultValue().toString());
}
return props;
}
/**
* Main method to start syncing.
*

View File

@@ -74,9 +74,10 @@ public final class SourceFormatAdapter {
// If the source schema is specified through Avro schema,
// pass in the schema for the Row-to-Avro conversion
// to avoid nullability mismatch between Avro schema and Row schema
? HoodieSparkUtils.createRdd(rdd, r.getSchemaProvider().getSourceSchema(),
HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD() : HoodieSparkUtils.createRdd(rdd,
HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD();
? HoodieSparkUtils.createRdd(rdd, HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE, true,
org.apache.hudi.common.util.Option.ofNullable(r.getSchemaProvider().getSourceSchema())
).toJavaRDD() : HoodieSparkUtils.createRdd(rdd,
HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE, false, Option.empty()).toJavaRDD();
})
.orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider());
}

View File

@@ -40,8 +40,8 @@ public class SparkAvroPostProcessor extends SchemaPostProcessor {
@Override
public Schema processSchema(Schema schema) {
return AvroConversionUtils.convertStructTypeToAvroSchema(
return schema != null ? AvroConversionUtils.convertStructTypeToAvroSchema(
AvroConversionUtils.convertAvroSchemaToStructType(schema), RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME,
RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE);
RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE) : null;
}
}

View File

@@ -18,10 +18,8 @@
package org.apache.hudi.utilities.functional;
import java.sql.Connection;
import java.sql.DriverManager;
import java.util.ConcurrentModificationException;
import java.util.concurrent.ExecutorService;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.config.DFSPropertiesConfiguration;
import org.apache.hudi.common.config.HoodieConfig;
import org.apache.hudi.common.config.TypedProperties;
@@ -33,6 +31,7 @@ import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
@@ -49,6 +48,7 @@ import org.apache.hudi.utilities.DummySchemaProvider;
import org.apache.hudi.utilities.HoodieClusteringJob;
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
import org.apache.hudi.utilities.schema.SparkAvroPostProcessor;
import org.apache.hudi.utilities.sources.CsvDFSSource;
import org.apache.hudi.utilities.sources.HoodieIncrSource;
import org.apache.hudi.utilities.sources.InputBatch;
@@ -63,6 +63,7 @@ import org.apache.hudi.utilities.testutils.sources.config.SourceConfigs;
import org.apache.hudi.utilities.transform.SqlQueryBasedTransformer;
import org.apache.hudi.utilities.transform.Transformer;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
@@ -92,11 +93,15 @@ import org.junit.jupiter.params.provider.MethodSource;
import org.junit.jupiter.params.provider.ValueSource;
import java.io.IOException;
import java.sql.Connection;
import java.sql.DriverManager;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.ConcurrentModificationException;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
@@ -105,6 +110,8 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY;
import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE;
import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -115,6 +122,7 @@ import static org.junit.jupiter.api.Assertions.fail;
/**
* Basic tests against {@link HoodieDeltaStreamer}, by issuing bulk_inserts, upserts, inserts. Check counts at the end.
*/
public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
private static final Logger LOG = LogManager.getLogger(TestHoodieDeltaStreamer.class);
@@ -157,7 +165,7 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
protected HoodieClusteringJob initialHoodieClusteringJob(String tableBasePath, String clusteringInstantTime, boolean runSchedule, String scheduleAndExecute) {
HoodieClusteringJob.Config scheduleClusteringConfig = buildHoodieClusteringUtilConfig(tableBasePath,
clusteringInstantTime, runSchedule, scheduleAndExecute);
clusteringInstantTime, runSchedule, scheduleAndExecute);
return new HoodieClusteringJob(jsc, scheduleClusteringConfig);
}
@@ -226,7 +234,7 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
cfg.payloadClassName = payloadClassName;
}
if (useSchemaProviderClass) {
cfg.schemaProviderClassName = FilebasedSchemaProvider.class.getName();
cfg.schemaProviderClassName = defaultSchemaProviderClassName;
}
return cfg;
}
@@ -391,6 +399,23 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
return base;
}
/**
* args for schema evolution test.
*
* @return
*/
private static Stream<Arguments> schemaEvolArgs() {
return Stream.of(
Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), true, true),
Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), true, false),
Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), false, true),
Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), false, false),
Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), true, true),
Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), true, false),
Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), false, true),
Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), false, false));
}
private static Stream<Arguments> provideValidCliArgs() {
HoodieDeltaStreamer.Config base = getBaseConfig();
@@ -425,41 +450,41 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
conf.enableHiveSync = true;
conf.configs = Arrays.asList(HOODIE_CONF_VALUE1, HOODIE_CONF_VALUE2);
String[] allConfig = new String[]{TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE, SOURCE_LIMIT_PARAM,
String[] allConfig = new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE, SOURCE_LIMIT_PARAM,
SOURCE_LIMIT_VALUE, TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE,
BASE_FILE_FORMAT_PARAM, BASE_FILE_FORMAT_VALUE, ENABLE_HIVE_SYNC_PARAM, HOODIE_CONF_PARAM, HOODIE_CONF_VALUE1,
HOODIE_CONF_PARAM, HOODIE_CONF_VALUE2};
return Stream.of(
// Base
Arguments.of(new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE,
TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE}, base),
// String
Arguments.of(new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE,
TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE,
BASE_FILE_FORMAT_PARAM, BASE_FILE_FORMAT_VALUE}, conf1),
// Integer
Arguments.of(new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE,
TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE,
SOURCE_LIMIT_PARAM, SOURCE_LIMIT_VALUE}, conf2),
// Boolean
Arguments.of(new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE,
TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE,
ENABLE_HIVE_SYNC_PARAM}, conf3),
// Array List 1
Arguments.of(new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE,
TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE,
HOODIE_CONF_PARAM, HOODIE_CONF_VALUE1}, conf4),
// Array List with comma
Arguments.of(new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE,
TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE,
HOODIE_CONF_PARAM, HOODIE_CONF_VALUE2}, conf5),
// Array list with multiple values
Arguments.of(new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE,
TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE,
HOODIE_CONF_PARAM, HOODIE_CONF_VALUE1, HOODIE_CONF_PARAM, HOODIE_CONF_VALUE2}, conf6),
// All
Arguments.of(allConfig, conf)
// Base
Arguments.of(new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE,
TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE}, base),
// String
Arguments.of(new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE,
TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE,
BASE_FILE_FORMAT_PARAM, BASE_FILE_FORMAT_VALUE}, conf1),
// Integer
Arguments.of(new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE,
TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE,
SOURCE_LIMIT_PARAM, SOURCE_LIMIT_VALUE}, conf2),
// Boolean
Arguments.of(new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE,
TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE,
ENABLE_HIVE_SYNC_PARAM}, conf3),
// Array List 1
Arguments.of(new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE,
TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE,
HOODIE_CONF_PARAM, HOODIE_CONF_VALUE1}, conf4),
// Array List with comma
Arguments.of(new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE,
TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE,
HOODIE_CONF_PARAM, HOODIE_CONF_VALUE2}, conf5),
// Array list with multiple values
Arguments.of(new String[] {TGT_BASE_PATH_PARAM, TGT_BASE_PATH_VALUE,
TABLE_TYPE_PARAM, TABLE_TYPE_VALUE, TARGET_TABLE_PARAM, TARGET_TABLE_VALUE,
HOODIE_CONF_PARAM, HOODIE_CONF_VALUE1, HOODIE_CONF_PARAM, HOODIE_CONF_VALUE2}, conf6),
// All
Arguments.of(allConfig, conf)
);
}
@@ -494,7 +519,7 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
@Test
public void testPropsWithInvalidKeyGenerator() throws Exception {
Exception e = assertThrows(IOException.class, () -> {
String tableBasePath = dfsBasePath + "/test_table";
String tableBasePath = dfsBasePath + "/test_table_invalid_key_gen";
HoodieDeltaStreamer deltaStreamer =
new HoodieDeltaStreamer(TestHelpers.makeConfig(tableBasePath, WriteOperationType.BULK_INSERT,
Collections.singletonList(TripsWithDistanceTransformer.class.getName()), PROPS_FILENAME_TEST_INVALID, false), jsc);
@@ -548,8 +573,8 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
// Perform bootstrap with tableBasePath as source
String bootstrapSourcePath = dfsBasePath + "/src_bootstrapped";
Dataset<Row> sourceDf = sqlContext.read()
.format("org.apache.hudi")
.load(tableBasePath + "/*/*.parquet");
.format("org.apache.hudi")
.load(tableBasePath + "/*/*.parquet");
sourceDf.write().format("parquet").save(bootstrapSourcePath);
String newDatasetBasePath = dfsBasePath + "/test_dataset_bootstrapped";
@@ -575,6 +600,83 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
assertTrue(fieldNames.containsAll(expectedFieldNames));
}
@ParameterizedTest
@MethodSource("schemaEvolArgs")
public void testSchemaEvolution(String tableType, boolean useUserProvidedSchema, boolean useSchemaPostProcessor) throws Exception {
String tableBasePath = dfsBasePath + "/test_table_schema_evolution" + tableType + "_" + useUserProvidedSchema + "_" + useSchemaPostProcessor;
defaultSchemaProviderClassName = FilebasedSchemaProvider.class.getName();
// Insert data produced with Schema A, pass Schema A
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT, Collections.singletonList(TestIdentityTransformer.class.getName()),
PROPS_FILENAME_TEST_SOURCE, false, true, false, null, tableType);
cfg.configs.add("hoodie.deltastreamer.schemaprovider.source.schema.file=" + dfsBasePath + "/source.avsc");
cfg.configs.add("hoodie.deltastreamer.schemaprovider.target.schema.file=" + dfsBasePath + "/source.avsc");
cfg.configs.add(DataSourceWriteOptions.RECONCILE_SCHEMA().key() + "=true");
if (!useSchemaPostProcessor) {
cfg.configs.add(SparkAvroPostProcessor.Config.SPARK_AVRO_POST_PROCESSOR_PROP_ENABLE + "=false");
}
new HoodieDeltaStreamer(cfg, jsc).sync();
TestHelpers.assertRecordCount(1000, tableBasePath + "/*/*", sqlContext);
TestHelpers.assertCommitMetadata("00000", tableBasePath, dfs, 1);
// Upsert data produced with Schema B, pass Schema B
cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, Collections.singletonList(TripsWithEvolvedOptionalFieldTransformer.class.getName()),
PROPS_FILENAME_TEST_SOURCE, false, true, false, null, tableType);
cfg.configs.add("hoodie.deltastreamer.schemaprovider.source.schema.file=" + dfsBasePath + "/source.avsc");
cfg.configs.add("hoodie.deltastreamer.schemaprovider.target.schema.file=" + dfsBasePath + "/source_evolved.avsc");
cfg.configs.add(DataSourceWriteOptions.RECONCILE_SCHEMA().key() + "=true");
if (!useSchemaPostProcessor) {
cfg.configs.add(SparkAvroPostProcessor.Config.SPARK_AVRO_POST_PROCESSOR_PROP_ENABLE + "=false");
}
new HoodieDeltaStreamer(cfg, jsc).sync();
// out of 1000 new records, 500 are inserts, 450 are updates and 50 are deletes.
TestHelpers.assertRecordCount(1450, tableBasePath + "/*/*", sqlContext);
TestHelpers.assertCommitMetadata("00001", tableBasePath, dfs, 2);
List<Row> counts = TestHelpers.countsPerCommit(tableBasePath + "/*/*", sqlContext);
assertEquals(1450, counts.stream().mapToLong(entry -> entry.getLong(1)).sum());
sqlContext.read().format("org.apache.hudi").load(tableBasePath + "/*/*").createOrReplaceTempView("tmp_trips");
long recordCount =
sqlContext.sparkSession().sql("select * from tmp_trips where evoluted_optional_union_field is not NULL").count();
assertEquals(950, recordCount);
// Upsert data produced with Schema A, pass Schema B
if (!useUserProvidedSchema) {
defaultSchemaProviderClassName = TestFileBasedSchemaProviderNullTargetSchema.class.getName();
}
cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, Collections.singletonList(TestIdentityTransformer.class.getName()),
PROPS_FILENAME_TEST_SOURCE, false, true, false, null, tableType);
cfg.configs.add("hoodie.deltastreamer.schemaprovider.source.schema.file=" + dfsBasePath + "/source.avsc");
if (useUserProvidedSchema) {
cfg.configs.add("hoodie.deltastreamer.schemaprovider.target.schema.file=" + dfsBasePath + "/source_evolved.avsc");
}
if (!useSchemaPostProcessor) {
cfg.configs.add(SparkAvroPostProcessor.Config.SPARK_AVRO_POST_PROCESSOR_PROP_ENABLE + "=false");
}
cfg.configs.add(DataSourceWriteOptions.RECONCILE_SCHEMA().key() + "=true");
new HoodieDeltaStreamer(cfg, jsc).sync();
// again, 1000 new records, 500 are inserts, 450 are updates and 50 are deletes.
TestHelpers.assertRecordCount(1900, tableBasePath + "/*/*", sqlContext);
TestHelpers.assertCommitMetadata("00002", tableBasePath, dfs, 3);
counts = TestHelpers.countsPerCommit(tableBasePath + "/*/*", sqlContext);
assertEquals(1900, counts.stream().mapToLong(entry -> entry.getLong(1)).sum());
TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(HoodieTableMetaClient.builder().setBasePath(tableBasePath).setConf(dfs.getConf()).build());
Schema tableSchema = tableSchemaResolver.getTableAvroSchemaWithoutMetadataFields();
assertNotNull(tableSchema);
Schema expectedSchema = new Schema.Parser().parse(dfs.open(new Path(dfsBasePath + "/source_evolved.avsc")));
if (!useUserProvidedSchema || useSchemaPostProcessor) {
expectedSchema = AvroConversionUtils.convertStructTypeToAvroSchema(
AvroConversionUtils.convertAvroSchemaToStructType(expectedSchema), HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE);
}
assertEquals(tableSchema, expectedSchema);
// clean up and reinit
UtilitiesTestBase.Helpers.deleteFileFromDfs(FSUtils.getFs(cfg.targetBasePath, jsc.hadoopConfiguration()), dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE);
writeCommonPropsToFile();
defaultSchemaProviderClassName = FilebasedSchemaProvider.class.getName();
}
@Test
public void testUpsertsCOWContinuousMode() throws Exception {
testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow");
@@ -782,8 +884,8 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
}
private void runJobsInParallel(String tableBasePath, HoodieTableType tableType, int totalRecords,
HoodieDeltaStreamer ingestionJob, HoodieDeltaStreamer.Config cfgIngestionJob, HoodieDeltaStreamer backfillJob,
HoodieDeltaStreamer.Config cfgBackfillJob, boolean expectConflict) throws Exception {
HoodieDeltaStreamer ingestionJob, HoodieDeltaStreamer.Config cfgIngestionJob, HoodieDeltaStreamer backfillJob,
HoodieDeltaStreamer.Config cfgBackfillJob, boolean expectConflict) throws Exception {
ExecutorService service = Executors.newFixedThreadPool(2);
HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(dfs.getConf()).setBasePath(tableBasePath).build();
HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
@@ -1262,7 +1364,7 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
}
private void prepareParquetDFSSource(boolean useSchemaProvider, boolean hasTransformer, String sourceSchemaFile, String targetSchemaFile,
String propsFileName, String parquetSourceRoot, boolean addCommonProps) throws IOException {
String propsFileName, String parquetSourceRoot, boolean addCommonProps) throws IOException {
// Properties used for testing delta-streamer with Parquet source
TypedProperties parquetProps = new TypedProperties();
@@ -1271,7 +1373,7 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
}
parquetProps.setProperty("include", "base.properties");
parquetProps.setProperty("hoodie.embed.timeline.server","false");
parquetProps.setProperty("hoodie.embed.timeline.server", "false");
parquetProps.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
parquetProps.setProperty("hoodie.datasource.write.partitionpath.field", "not_there");
if (useSchemaProvider) {
@@ -1301,7 +1403,7 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
TypedProperties props = new TypedProperties();
populateAllCommonProps(props);
props.setProperty("include", "base.properties");
props.setProperty("hoodie.embed.timeline.server","false");
props.setProperty("hoodie.embed.timeline.server", "false");
props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
props.setProperty("hoodie.datasource.write.partitionpath.field", "not_there");
props.setProperty("hoodie.deltastreamer.source.dfs.root", JSON_KAFKA_SOURCE_ROOT);
@@ -1316,6 +1418,7 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
/**
* Tests Deltastreamer with parquet dfs source and transitions to JsonKafkaSource.
*
* @param autoResetToLatest true if auto reset value to be set to LATEST. false to leave it as default(i.e. EARLIEST)
* @throws Exception
*/
@@ -1325,7 +1428,7 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
int parquetRecords = 10;
prepareParquetDFSFiles(parquetRecords, PARQUET_SOURCE_ROOT, FIRST_PARQUET_FILE_NAME, true, HoodieTestDataGenerator.TRIP_SCHEMA, HoodieTestDataGenerator.AVRO_TRIP_SCHEMA);
prepareParquetDFSSource(true, false,"source_uber.avsc", "target_uber.avsc", PROPS_FILENAME_TEST_PARQUET,
prepareParquetDFSSource(true, false, "source_uber.avsc", "target_uber.avsc", PROPS_FILENAME_TEST_PARQUET,
PARQUET_SOURCE_ROOT, false);
// delta streamer w/ parquest source
String tableBasePath = dfsBasePath + "/test_dfs_to_kakfa" + testNum;
@@ -1388,19 +1491,19 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
prepareJsonKafkaDFSSource(PROPS_FILENAME_TEST_JSON_KAFKA, "earliest", topicName);
String tableBasePath = dfsBasePath + "/test_json_kafka_table" + testNum;
HoodieDeltaStreamer deltaStreamer = new HoodieDeltaStreamer(
TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, JsonKafkaSource.class.getName(),
Collections.EMPTY_LIST, PROPS_FILENAME_TEST_JSON_KAFKA, false,
true, 100000, false, null,
null, "timestamp", String.valueOf(System.currentTimeMillis())), jsc);
TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, JsonKafkaSource.class.getName(),
Collections.EMPTY_LIST, PROPS_FILENAME_TEST_JSON_KAFKA, false,
true, 100000, false, null,
null, "timestamp", String.valueOf(System.currentTimeMillis())), jsc);
deltaStreamer.sync();
TestHelpers.assertRecordCount(JSON_KAFKA_NUM_RECORDS, tableBasePath + "/*/*.parquet", sqlContext);
prepareJsonKafkaDFSFiles(JSON_KAFKA_NUM_RECORDS, false, topicName);
deltaStreamer = new HoodieDeltaStreamer(
TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, JsonKafkaSource.class.getName(),
Collections.EMPTY_LIST, PROPS_FILENAME_TEST_JSON_KAFKA, false,
true, 100000, false, null, null,
"timestamp", String.valueOf(System.currentTimeMillis())), jsc);
TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, JsonKafkaSource.class.getName(),
Collections.EMPTY_LIST, PROPS_FILENAME_TEST_JSON_KAFKA, false,
true, 100000, false, null, null,
"timestamp", String.valueOf(System.currentTimeMillis())), jsc);
deltaStreamer.sync();
TestHelpers.assertRecordCount(JSON_KAFKA_NUM_RECORDS * 2, tableBasePath + "/*/*.parquet", sqlContext);
}
@@ -1717,8 +1820,36 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
@Override
public Dataset<Row> apply(JavaSparkContext jsc, SparkSession sparkSession, Dataset<Row> rowDataset,
TypedProperties properties) {
TypedProperties properties) {
return rowDataset;
}
}
/**
* Add new field evoluted_optional_union_field with value of the field rider.
*/
public static class TripsWithEvolvedOptionalFieldTransformer implements Transformer {
@Override
public Dataset<Row> apply(JavaSparkContext jsc, SparkSession sparkSession, Dataset<Row> rowDataset,
TypedProperties properties) {
return rowDataset.withColumn("evoluted_optional_union_field", functions.col("rider"));
}
}
/**
* {@link FilebasedSchemaProvider} to be used in tests where target schema is null.
*/
public static class TestFileBasedSchemaProviderNullTargetSchema extends FilebasedSchemaProvider {
public TestFileBasedSchemaProviderNullTargetSchema(TypedProperties props, JavaSparkContext jssc) {
super(props, jssc);
}
@Override
public Schema getTargetSchema() {
return null;
}
}
}

View File

@@ -22,6 +22,7 @@ import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
import org.apache.avro.Schema;
@@ -76,7 +77,7 @@ public class TestHoodieDeltaStreamerBase extends UtilitiesTestBase {
static final Logger LOG = LogManager.getLogger(TestHoodieDeltaStreamerBase.class);
public static KafkaTestUtils testUtils;
protected static String topicName;
protected static String defaultSchemaProviderClassName = FilebasedSchemaProvider.class.getName();
protected static int testNum = 1;
@BeforeAll
@@ -94,6 +95,7 @@ public class TestHoodieDeltaStreamerBase 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_evolved.avsc", dfs, dfsBasePath + "/source_evolved.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");
@@ -107,22 +109,7 @@ public class TestHoodieDeltaStreamerBase extends UtilitiesTestBase {
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/short_trip_uber_config.properties", dfs, dfsBasePath + "/config/short_trip_uber_config.properties");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/clusteringjob.properties", dfs, dfsBasePath + "/clusteringjob.properties");
TypedProperties props = new TypedProperties();
props.setProperty("include", "sql-transformer.properties");
props.setProperty("hoodie.datasource.write.keygenerator.class", TestHoodieDeltaStreamer.TestGenerator.class.getName());
props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
props.setProperty("hoodie.datasource.write.partitionpath.field", "not_there");
props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc");
// Hive Configs
props.setProperty(DataSourceWriteOptions.HIVE_URL().key(), "jdbc:hive2://127.0.0.1:9999/");
props.setProperty(DataSourceWriteOptions.HIVE_DATABASE().key(), "testdb1");
props.setProperty(DataSourceWriteOptions.HIVE_TABLE().key(), "hive_trips");
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "datestr");
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
MultiPartKeysValueExtractor.class.getName());
UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE);
writeCommonPropsToFile();
// Properties used for the delta-streamer which incrementally pulls from upstream Hudi source table and writes to
// downstream hudi table
@@ -162,6 +149,25 @@ public class TestHoodieDeltaStreamerBase extends UtilitiesTestBase {
prepareParquetDFSFiles(PARQUET_NUM_RECORDS, PARQUET_SOURCE_ROOT);
}
protected static void writeCommonPropsToFile() throws IOException {
TypedProperties props = new TypedProperties();
props.setProperty("include", "sql-transformer.properties");
props.setProperty("hoodie.datasource.write.keygenerator.class", TestHoodieDeltaStreamer.TestGenerator.class.getName());
props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
props.setProperty("hoodie.datasource.write.partitionpath.field", "not_there");
props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc");
// Hive Configs
props.setProperty(DataSourceWriteOptions.HIVE_URL().key(), "jdbc:hive2://127.0.0.1:9999/");
props.setProperty(DataSourceWriteOptions.HIVE_DATABASE().key(), "testdb1");
props.setProperty(DataSourceWriteOptions.HIVE_TABLE().key(), "hive_trips");
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "datestr");
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
MultiPartKeysValueExtractor.class.getName());
UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE);
}
@BeforeEach
public void setup() throws Exception {
super.setup();
@@ -241,5 +247,4 @@ public class TestHoodieDeltaStreamerBase extends UtilitiesTestBase {
dataGenerator.generateInserts("000", numRecords)), new Path(path));
}
}
}

View File

@@ -243,6 +243,12 @@ public class UtilitiesTestBase {
os.close();
}
public static void deleteFileFromDfs(FileSystem fs, String targetPath) throws IOException {
if (fs.exists(new Path(targetPath))) {
fs.delete(new Path(targetPath), true);
}
}
public static void savePropsToDFS(TypedProperties props, FileSystem fs, String targetPath) throws IOException {
String[] lines = props.keySet().stream().map(k -> String.format("%s=%s", k, props.get(k))).toArray(String[]::new);
saveStringsToDFS(lines, fs, targetPath);

View File

@@ -0,0 +1,139 @@
/*
* 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" : "long"
}, {
"name" : "_row_key",
"type" : "string"
}, {
"name" : "partition_path",
"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" : "distance_in_meters",
"type" : "int"
}, {
"name" : "seconds_since_epoch",
"type" : "long"
}, {
"name" : "weight",
"type" : "float"
},{
"name" : "nation",
"type" : "bytes"
},{
"name" : "current_date",
"type" : {
"type" : "int",
"logicalType" : "date"
}
},{
"name" : "current_ts",
"type" : {
"type" : "long"
}
},{
"name" : "height",
"type" : {
"type" : "fixed",
"name" : "abc",
"size" : 5,
"logicalType" : "decimal",
"precision" : 10,
"scale": 6
}
}, {
"name" :"city_to_state",
"type" : {
"type" : "map",
"values": "string"
}
},
{
"name" : "fare",
"type" : {
"type" : "record",
"name" : "fare",
"fields" : [
{
"name" : "amount",
"type" : "double"
},
{
"name" : "currency",
"type" : "string"
}
]
}
},
{
"name" : "tip_history",
"type" : {
"type" : "array",
"items" : {
"type" : "record",
"name" : "tip_history",
"fields" : [
{
"name" : "amount",
"type" : "double"
},
{
"name" : "currency",
"type" : "string"
}
]
}
}
},
{
"name" : "_hoodie_is_deleted",
"type" : "boolean",
"default" : false
},
{
"name": "evoluted_optional_union_field",
"type": [
"null",
"string"
],
"default": null
}]
}