1
0

[HUDI-242] Support for RFC-12/Bootstrapping of external datasets to hudi (#1876)

- [HUDI-418] Bootstrap Index Implementation using HFile with unit-test
 - [HUDI-421] FileSystem View Changes to support Bootstrap with unit-tests
 - [HUDI-424] Implement Query Side Integration for querying tables containing bootstrap file slices
 - [HUDI-423] Implement upsert functionality for handling updates to these bootstrap file slices
 - [HUDI-421] Bootstrap Write Client with tests
 - [HUDI-425] Added HoodieDeltaStreamer support
 - [HUDI-899] Add a knob to change partition-path style while performing metadata bootstrap
 - [HUDI-900] Metadata Bootstrap Key Generator needs to handle complex keys correctly
 - [HUDI-424] Simplify Record reader implementation
 - [HUDI-423] Implement upsert functionality for handling updates to these bootstrap file slices
 - [HUDI-420] Hoodie Demo working with hive and sparkSQL. Also, Hoodie CLI working with bootstrap tables

Co-authored-by: Mehrotra <uditme@amazon.com>
Co-authored-by: Vinoth Chandar <vinoth@apache.org>
Co-authored-by: Balaji Varadarajan <varadarb@uber.com>
This commit is contained in:
vinoth chandar
2020-08-03 20:19:21 -07:00
committed by GitHub
parent 266bce12b3
commit 539621bd33
175 changed files with 7540 additions and 779 deletions

View File

@@ -18,12 +18,12 @@
package org.apache.hudi.utilities.checkpointing;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.exception.HoodieException;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.exception.HoodieException;
/**
* Provide the initial checkpoint for delta streamer.
@@ -51,7 +51,13 @@ public abstract class InitialCheckPointProvider {
*
* @param config Hadoop configuration
*/
public abstract void init(Configuration config) throws HoodieException;
public void init(Configuration config) throws HoodieException {
try {
this.fs = FileSystem.get(config);
} catch (IOException e) {
throw new HoodieException("CheckpointProvider initialization failed");
}
}
/**
* Get checkpoint string recognizable for delta streamer.

View File

@@ -0,0 +1,64 @@
/*
* 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.checkpointing;
import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.exception.HoodieException;
import java.io.IOException;
import java.util.Objects;
import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY;
/**
* This is used to set a checkpoint from latest commit of another (mirror) hudi dataset.
* Used by integration test.
*/
public class InitialCheckpointFromAnotherHoodieTimelineProvider extends InitialCheckPointProvider {
private HoodieTableMetaClient anotherDsHoodieMetaclient;
public InitialCheckpointFromAnotherHoodieTimelineProvider(TypedProperties props) {
super(props);
}
@Override
public void init(Configuration config) throws HoodieException {
super.init(config);
this.anotherDsHoodieMetaclient = new HoodieTableMetaClient(config, path.toString());
}
@Override
public String getCheckpoint() throws HoodieException {
return anotherDsHoodieMetaclient.getCommitsTimeline().filterCompletedInstants().getReverseOrderedInstants()
.map(instant -> {
try {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(anotherDsHoodieMetaclient.getActiveTimeline().getInstantDetails(instant).get(),
HoodieCommitMetadata.class);
return commitMetadata.getMetadata(CHECKPOINT_KEY);
} catch (IOException e) {
return null;
}
}).filter(Objects::nonNull).findFirst().get();
}
}

View File

@@ -0,0 +1,181 @@
/*
* 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.deltastreamer;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.hive.HiveSyncTool;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.utilities.UtilHelpers;
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.IOException;
import java.io.Serializable;
import java.util.HashMap;
/**
* Performs bootstrap from a non-hudi source.
*/
public class BootstrapExecutor implements Serializable {
private static final Logger LOG = LogManager.getLogger(BootstrapExecutor.class);
/**
* Config.
*/
private final HoodieDeltaStreamer.Config cfg;
/**
* Schema provider that supplies the command for reading the input and writing out the target table.
*/
private transient SchemaProvider schemaProvider;
/**
* Spark context.
*/
private transient JavaSparkContext jssc;
/**
* Bag of properties with source, hoodie client, key generator etc.
*/
private final TypedProperties props;
/**
* Hadoop Configuration.
*/
private final Configuration configuration;
/**
* Bootstrap Configuration.
*/
private final HoodieWriteConfig bootstrapConfig;
/**
* FileSystem instance.
*/
private transient FileSystem fs;
private String bootstrapBasePath;
/**
* Bootstrap Executor.
* @param cfg DeltaStreamer Config
* @param jssc Java Spark Context
* @param fs File System
* @param properties Bootstrap Writer Properties
* @throws IOException
*/
public BootstrapExecutor(HoodieDeltaStreamer.Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf,
TypedProperties properties) throws IOException {
this.cfg = cfg;
this.jssc = jssc;
this.fs = fs;
this.configuration = conf;
this.props = properties;
ValidationUtils.checkArgument(properties.containsKey(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH),
HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH + " must be specified.");
this.bootstrapBasePath = properties.getString(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH);
// Add more defaults if full bootstrap requested
this.props.putIfAbsent(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_PAYLOAD_OPT_VAL());
this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc);
HoodieWriteConfig.Builder builder =
HoodieWriteConfig.newBuilder().withPath(cfg.targetBasePath)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withInlineCompaction(false).build())
.forTable(cfg.targetTableName)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withAutoCommit(true)
.withProps(props);
if (null != schemaProvider && null != schemaProvider.getTargetSchema()) {
builder = builder.withSchema(schemaProvider.getTargetSchema().toString());
}
this.bootstrapConfig = builder.build();
LOG.info("Created bootstrap executor with configs : " + bootstrapConfig.getProps());
}
/**
* Executes Bootstrap.
*/
public void execute() throws IOException {
initializeTable();
HoodieWriteClient bootstrapClient = new HoodieWriteClient(jssc, bootstrapConfig, true);
try {
HashMap<String, String> checkpointCommitMetadata = new HashMap<>();
checkpointCommitMetadata.put(HoodieDeltaStreamer.CHECKPOINT_KEY, cfg.checkpoint);
if (cfg.checkpoint != null) {
checkpointCommitMetadata.put(HoodieDeltaStreamer.CHECKPOINT_RESET_KEY, cfg.checkpoint);
}
bootstrapClient.bootstrap(Option.of(checkpointCommitMetadata));
syncHive();
} finally {
bootstrapClient.close();
}
}
/**
* Sync to Hive.
*/
private void syncHive() {
if (cfg.enableHiveSync) {
HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath, cfg.baseFileFormat);
LOG.info("Syncing target hoodie table with hive table(" + hiveSyncConfig.tableName + "). Hive metastore URL :"
+ hiveSyncConfig.jdbcUrl + ", basePath :" + cfg.targetBasePath);
new HiveSyncTool(hiveSyncConfig, new HiveConf(configuration, HiveConf.class), fs).syncHoodieTable();
}
}
private void initializeTable() throws IOException {
if (fs.exists(new Path(cfg.targetBasePath))) {
throw new HoodieException("target base path already exists at " + cfg.targetBasePath
+ ". Cannot bootstrap data on top of an existing table");
}
HoodieTableMetaClient.initTableTypeWithBootstrap(new Configuration(jssc.hadoopConfiguration()),
cfg.targetBasePath, HoodieTableType.valueOf(cfg.tableType), cfg.targetTableName, "archived", cfg.payloadClassName,
cfg.baseFileFormat, cfg.bootstrapIndexClass, bootstrapBasePath);
}
public HoodieWriteConfig getBootstrapConfig() {
return bootstrapConfig;
}
}

View File

@@ -20,12 +20,14 @@ package org.apache.hudi.utilities.deltastreamer;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -73,6 +75,8 @@ import java.util.stream.Collectors;
import scala.collection.JavaConversions;
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.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;
@@ -87,8 +91,6 @@ public class DeltaSync implements Serializable {
private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(DeltaSync.class);
public static final String CHECKPOINT_KEY = "deltastreamer.checkpoint.key";
public static final String CHECKPOINT_RESET_KEY = "deltastreamer.checkpoint.reset_key";
/**
* Delta Sync Config.
@@ -206,14 +208,14 @@ public class DeltaSync implements Serializable {
} else {
this.commitTimelineOpt = Option.empty();
HoodieTableMetaClient.initTableType(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath,
cfg.tableType, cfg.targetTableName, "archived", cfg.payloadClassName, cfg.baseFileFormat);
HoodieTableType.valueOf(cfg.tableType), cfg.targetTableName, "archived", cfg.payloadClassName, cfg.baseFileFormat);
}
}
/**
* Run one round of delta sync and return new compaction instant if one got scheduled.
*/
public Pair<Option<String>, JavaRDD<WriteStatus>> syncOnce() throws Exception {
public Pair<Option<String>, JavaRDD<WriteStatus>> syncOnce() throws IOException {
Pair<Option<String>, JavaRDD<WriteStatus>> result = null;
HoodieDeltaStreamerMetrics metrics = new HoodieDeltaStreamerMetrics(getHoodieClientConfig(schemaProvider));
Timer.Context overallTimerContext = metrics.getOverallTimerContext();
@@ -249,8 +251,7 @@ public class DeltaSync implements Serializable {
* of schemaProvider, checkpointStr and hoodieRecord
* @throws Exception in case of any Exception
*/
public Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> readFromSource(
Option<HoodieTimeline> commitTimelineOpt) throws Exception {
public Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> readFromSource(Option<HoodieTimeline> commitTimelineOpt) throws IOException {
// Retrieve the previous round checkpoints, if any
Option<String> resumeCheckpointStr = Option.empty();
if (commitTimelineOpt.isPresent()) {
@@ -265,7 +266,8 @@ public class DeltaSync implements Serializable {
if (!commitMetadata.getMetadata(CHECKPOINT_KEY).isEmpty()) {
resumeCheckpointStr = Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY));
}
} else {
} else if (HoodieTimeline.compareTimestamps(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
HoodieTimeline.LESSER_THAN, lastCommit.get().getTimestamp())) {
throw new HoodieDeltaStreamerException(
"Unable to find previous checkpoint. Please double check if this table "
+ "was indeed built via delta streamer. Last Commit :" + lastCommit + ", Instants :"
@@ -275,7 +277,7 @@ public class DeltaSync implements Serializable {
}
} else {
HoodieTableMetaClient.initTableType(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath,
cfg.tableType, cfg.targetTableName, "archived", cfg.payloadClassName, cfg.baseFileFormat);
HoodieTableType.valueOf(cfg.tableType), cfg.targetTableName, "archived", cfg.payloadClassName, cfg.baseFileFormat);
}
if (!resumeCheckpointStr.isPresent() && cfg.checkpoint != null) {
@@ -338,12 +340,12 @@ public class DeltaSync implements Serializable {
}
JavaRDD<GenericRecord> avroRDD = avroRDDOptional.get();
String deleteMakrerField = props.getString(HoodieWriteConfig.DELETE_MARKER_FIELD_PROP,
String deleteMarkerField = props.getString(HoodieWriteConfig.DELETE_MARKER_FIELD_PROP,
HoodieWriteConfig.DEFAULT_DELETE_MARKER_FIELD);
JavaRDD<HoodieRecord> records = avroRDD.map(gr -> {
HoodieRecordPayload payload = DataSourceUtils.createPayload(cfg.payloadClassName, gr,
(Comparable) DataSourceUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false),
deleteMakrerField);
(Comparable) HoodieAvroUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false),
deleteMarkerField);
return new HoodieRecord<>(keyGenerator.getKey(gr), payload);
});
@@ -360,8 +362,8 @@ public class DeltaSync implements Serializable {
* @return Option Compaction instant if one is scheduled
*/
private Pair<Option<String>, JavaRDD<WriteStatus>> writeToSink(JavaRDD<HoodieRecord> records, String checkpointStr,
HoodieDeltaStreamerMetrics metrics, Timer.Context overallTimerContext) throws Exception {
HoodieDeltaStreamerMetrics metrics,
Timer.Context overallTimerContext) {
Option<String> scheduledCompactionInstant = Option.empty();
// filter dupes if needed
if (cfg.filterDupes) {
@@ -476,7 +478,7 @@ public class DeltaSync implements Serializable {
/**
* Sync to Hive.
*/
public void syncHiveIfNeeded() throws ClassNotFoundException {
public void syncHiveIfNeeded() {
if (cfg.enableHiveSync) {
syncHive();
}
@@ -591,3 +593,4 @@ public class DeltaSync implements Serializable {
return commitTimelineOpt;
}
}

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.utilities.deltastreamer;
import org.apache.hudi.async.AbstractAsyncService;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieTableType;
@@ -83,11 +84,16 @@ public class HoodieDeltaStreamer implements Serializable {
private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(HoodieDeltaStreamer.class);
public static String CHECKPOINT_KEY = "deltastreamer.checkpoint.key";
public static final String CHECKPOINT_KEY = "deltastreamer.checkpoint.key";
public static final String CHECKPOINT_RESET_KEY = "deltastreamer.checkpoint.reset_key";
protected final transient Config cfg;
protected transient DeltaSyncService deltaSyncService;
private final TypedProperties properties;
protected transient Option<DeltaSyncService> deltaSyncService;
private final Option<BootstrapExecutor> bootstrapExecutor;
public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc) throws IOException {
this(cfg, jssc, FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()),
@@ -104,19 +110,27 @@ public class HoodieDeltaStreamer implements Serializable {
}
public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf,
TypedProperties properties) throws IOException {
TypedProperties props) throws IOException {
// Resolving the properties first in a consistent way
this.properties = props != null ? props : UtilHelpers.readConfig(
FSUtils.getFs(cfg.propsFilePath, jssc.hadoopConfiguration()),
new Path(cfg.propsFilePath), cfg.configs).getConfig();
if (cfg.initialCheckpointProvider != null && cfg.checkpoint == null) {
InitialCheckPointProvider checkPointProvider =
UtilHelpers.createInitialCheckpointProvider(cfg.initialCheckpointProvider, properties);
UtilHelpers.createInitialCheckpointProvider(cfg.initialCheckpointProvider, this.properties);
checkPointProvider.init(conf);
cfg.checkpoint = checkPointProvider.getCheckpoint();
}
this.cfg = cfg;
this.deltaSyncService = new DeltaSyncService(cfg, jssc, fs, conf, properties);
this.bootstrapExecutor = Option.ofNullable(
cfg.runBootstrap ? new BootstrapExecutor(cfg, jssc, fs, conf, this.properties) : null);
this.deltaSyncService = Option.ofNullable(
cfg.runBootstrap ? null : new DeltaSyncService(cfg, jssc, fs, conf, this.properties));
}
public void shutdownGracefully() {
deltaSyncService.shutdown(false);
deltaSyncService.ifPresent(ds -> ds.shutdown(false));
}
/**
@@ -125,20 +139,37 @@ public class HoodieDeltaStreamer implements Serializable {
* @throws Exception
*/
public void sync() throws Exception {
if (cfg.continuousMode) {
deltaSyncService.start(this::onDeltaSyncShutdown);
deltaSyncService.waitForShutdown();
LOG.info("Delta Sync shutting down");
if (bootstrapExecutor.isPresent()) {
LOG.info("Performing bootstrap. Source=" + bootstrapExecutor.get().getBootstrapConfig().getBootstrapSourceBasePath());
bootstrapExecutor.get().execute();
} else {
LOG.info("Delta Streamer running only single round");
try {
deltaSyncService.getDeltaSync().syncOnce();
} catch (Exception ex) {
LOG.error("Got error running delta sync once. Shutting down", ex);
throw ex;
} finally {
deltaSyncService.close();
LOG.info("Shut down delta streamer");
if (cfg.continuousMode) {
deltaSyncService.ifPresent(ds -> {
ds.start(this::onDeltaSyncShutdown);
try {
ds.waitForShutdown();
} catch (Exception e) {
throw new HoodieException(e.getMessage(), e);
}
});
LOG.info("Delta Sync shutting down");
} else {
LOG.info("Delta Streamer running only single round");
try {
deltaSyncService.ifPresent(ds -> {
try {
ds.getDeltaSync().syncOnce();
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
});
} catch (Exception ex) {
LOG.error("Got error running delta sync once. Shutting down", ex);
throw ex;
} finally {
deltaSyncService.ifPresent(DeltaSyncService::close);
LOG.info("Shut down delta streamer");
}
}
}
}
@@ -149,7 +180,7 @@ public class HoodieDeltaStreamer implements Serializable {
private boolean onDeltaSyncShutdown(boolean error) {
LOG.info("DeltaSync shutdown. Closing write client. Error?" + error);
deltaSyncService.close();
deltaSyncService.ifPresent(DeltaSyncService::close);
return true;
}
@@ -181,7 +212,7 @@ public class HoodieDeltaStreamer implements Serializable {
public String tableType;
@Parameter(names = {"--base-file-format"}, description = "File format for the base files. PARQUET (or) HFILE", required = false)
public String baseFileFormat;
public String baseFileFormat = "PARQUET";
@Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for "
+ "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are "
@@ -294,6 +325,12 @@ public class HoodieDeltaStreamer implements Serializable {
+ "Use this field only when switching source, for example, from DFS source to Kafka Source.")
public String initialCheckpointProvider = null;
@Parameter(names = {"--run-bootstrap"}, description = "Run bootstrap if bootstrap index is not found")
public Boolean runBootstrap = false;
@Parameter(names = {"--bootstrap-index-class"}, description = "subclass of BootstrapIndex")
public String bootstrapIndexClass = HFileBootstrapIndex.class.getName();
@Parameter(names = {"--help", "-h"}, help = true)
public Boolean help = false;
@@ -404,9 +441,7 @@ public class HoodieDeltaStreamer implements Serializable {
ValidationUtils.checkArgument(!cfg.filterDupes || cfg.operation != Operation.UPSERT,
"'--filter-dupes' needs to be disabled when '--op' is 'UPSERT' to ensure updates are not missed.");
this.props = properties != null ? properties : UtilHelpers.readConfig(
FSUtils.getFs(cfg.propsFilePath, jssc.hadoopConfiguration()),
new Path(cfg.propsFilePath), cfg.configs).getConfig();
this.props = properties;
LOG.info("Creating delta streamer with configs : " + props.toString());
this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc);
@@ -627,6 +662,6 @@ public class HoodieDeltaStreamer implements Serializable {
}
public DeltaSyncService getDeltaSyncService() {
return deltaSyncService;
return deltaSyncService.get();
}
}

View File

@@ -23,6 +23,7 @@ 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.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.table.HoodieTableConfig;
@@ -70,6 +71,7 @@ import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.api.java.UDF4;
import org.apache.spark.sql.functions;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.sql.types.StructField;
import org.apache.spark.streaming.kafka010.KafkaTestUtils;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach;
@@ -443,7 +445,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
}
@Test
public void testBulkInsertsAndUpserts() throws Exception {
public void testBulkInsertsAndUpsertsWithBootstrap() throws Exception {
String tableBasePath = dfsBasePath + "/test_table";
// Initial bulk insert
@@ -469,6 +471,34 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
TestHelpers.assertCommitMetadata("00001", tableBasePath, dfs, 2);
List<Row> counts = TestHelpers.countsPerCommit(tableBasePath + "/*/*.parquet", sqlContext);
assertEquals(1950, counts.stream().mapToLong(entry -> entry.getLong(1)).sum());
// Perform bootstrap with tableBasePath as source
String bootstrapSourcePath = dfsBasePath + "/src_bootstrapped";
sqlContext.read().format("org.apache.hudi").load(tableBasePath + "/*/*.parquet").write().format("parquet")
.save(bootstrapSourcePath);
String newDatasetBasePath = dfsBasePath + "/test_dataset_bootstrapped";
cfg.runBootstrap = true;
cfg.configs.add(String.format("hoodie.bootstrap.base.path=%s", bootstrapSourcePath));
cfg.configs.add(String.format("hoodie.bootstrap.keygen.class=%s", SimpleKeyGenerator.class.getName()));
cfg.configs.add("hoodie.bootstrap.parallelism=5");
cfg.targetBasePath = newDatasetBasePath;
new HoodieDeltaStreamer(cfg, jsc).sync();
Dataset<Row> res = sqlContext.read().format("org.apache.hudi").load(newDatasetBasePath + "/*.parquet");
LOG.info("Schema :");
res.printSchema();
TestHelpers.assertRecordCount(1950, newDatasetBasePath + "/*.parquet", sqlContext);
res.registerTempTable("bootstrapped");
assertEquals(1950, sqlContext.sql("select distinct _hoodie_record_key from bootstrapped").count());
StructField[] fields = res.schema().fields();
assertEquals(5, fields.length);
assertEquals(HoodieRecord.COMMIT_TIME_METADATA_FIELD, fields[0].name());
assertEquals(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, fields[1].name());
assertEquals(HoodieRecord.RECORD_KEY_METADATA_FIELD, fields[2].name());
assertEquals(HoodieRecord.PARTITION_PATH_METADATA_FIELD, fields[3].name());
assertEquals(HoodieRecord.FILENAME_METADATA_FIELD, fields[4].name());
}
@Test

View File

@@ -68,8 +68,7 @@
},{
"name" : "current_ts",
"type" : {
"type" : "long",
"logicalType" : "timestamp-micros"
"type" : "long"
}
},{
"name" : "height",

View File

@@ -68,8 +68,7 @@
},{
"name" : "current_ts",
"type" : {
"type" : "long",
"logicalType" : "timestamp-micros"
"type" : "long"
}
}, {
"name" : "height",

View File

@@ -17,6 +17,7 @@
###
log4j.rootLogger=WARN, CONSOLE
log4j.logger.org.apache.hudi=DEBUG
log4j.logger.org.apache.hadoop.hbase=ERROR
# CONSOLE is set to be a ConsoleAppender.
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender

View File

@@ -18,6 +18,7 @@
log4j.rootLogger=WARN, CONSOLE
log4j.logger.org.apache=INFO
log4j.logger.org.apache.hudi=DEBUG
log4j.logger.org.apache.hadoop.hbase=ERROR
# A1 is set to be a ConsoleAppender.
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender