Perform consistency checks during write finalize
- Check to ensure written files are listable on storage - Docs reflected to capture how this helps with s3 storage - Unit tests added, corrections to existing tests - Fix DeltaStreamer to manage archived commits in a separate folder
This commit is contained in:
committed by
vinoth chandar
parent
4c74dd4cad
commit
9ca6f91e97
@@ -54,7 +54,6 @@ import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -159,11 +158,8 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
}
|
||||
}
|
||||
} else {
|
||||
Properties properties = new Properties();
|
||||
properties.put(HoodieWriteConfig.TABLE_NAME, cfg.targetTableName);
|
||||
HoodieTableMetaClient
|
||||
.initializePathAsHoodieDataset(jssc.hadoopConfiguration(), cfg.targetBasePath,
|
||||
properties);
|
||||
HoodieTableMetaClient.initTableType(jssc.hadoopConfiguration(), cfg.targetBasePath,
|
||||
cfg.storageType, cfg.targetTableName, "archived");
|
||||
}
|
||||
log.info("Checkpoint to resume from : " + resumeCheckpointStr);
|
||||
|
||||
@@ -247,6 +243,10 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
@Parameter(names = {"--target-table"}, description = "name of the target table in Hive", required = true)
|
||||
public String targetTableName;
|
||||
|
||||
@Parameter(names = {"--storage-type"}, description = "Type of Storage. "
|
||||
+ "COPY_ON_WRITE (or) MERGE_ON_READ", required = true)
|
||||
public String storageType;
|
||||
|
||||
@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 "
|
||||
+ "used, but recommend use to provide basic things like metrics endpoints, hive configs etc. For sources, refer"
|
||||
|
||||
@@ -89,6 +89,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config();
|
||||
cfg.targetBasePath = basePath;
|
||||
cfg.targetTableName = "hoodie_trips";
|
||||
cfg.storageType = "COPY_ON_WRITE";
|
||||
cfg.sourceClassName = TestDataSource.class.getName();
|
||||
cfg.operation = op;
|
||||
cfg.sourceOrderingField = "timestamp";
|
||||
|
||||
Reference in New Issue
Block a user