1
0

Restore 0.8.0 config keys with deprecated annotation (#3506)

Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com>
Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
Udit Mehrotra
2021-08-19 13:36:40 -07:00
committed by GitHub
parent 37c29e75dc
commit c350d05dd3
137 changed files with 3460 additions and 1527 deletions

View File

@@ -74,7 +74,7 @@ public class HoodieSnapshotCopier implements Serializable {
public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
@Parameter(names = {"--verify-metadata-file-listing"}, description = "Verify file listing from Hudi's metadata against file system")
public Boolean verifyMetadataFileListing = HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue();
public Boolean verifyMetadataFileListing = HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue();
}
public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDir,

View File

@@ -35,12 +35,12 @@ import org.apache.log4j.Logger;
import java.util.Properties;
import static org.apache.hudi.config.HoodieWriteConfig.TABLE_NAME;
import static org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig.CALLBACK_KAFKA_ACKS;
import static org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig.CALLBACK_KAFKA_BOOTSTRAP_SERVERS;
import static org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig.CALLBACK_KAFKA_PARTITION;
import static org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig.CALLBACK_KAFKA_RETRIES;
import static org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig.CALLBACK_KAFKA_TOPIC;
import static org.apache.hudi.config.HoodieWriteConfig.TBL_NAME;
import static org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig.ACKS;
import static org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig.BOOTSTRAP_SERVERS;
import static org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig.PARTITION;
import static org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig.RETRIES;
import static org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig.TOPIC;
/**
* Kafka implementation of {@link HoodieWriteCommitCallback}.
@@ -55,8 +55,8 @@ public class HoodieWriteCommitKafkaCallback implements HoodieWriteCommitCallback
public HoodieWriteCommitKafkaCallback(HoodieWriteConfig config) {
this.hoodieConfig = config;
this.bootstrapServers = config.getString(CALLBACK_KAFKA_BOOTSTRAP_SERVERS);
this.topic = config.getString(CALLBACK_KAFKA_TOPIC);
this.bootstrapServers = config.getString(BOOTSTRAP_SERVERS);
this.topic = config.getString(TOPIC);
validateKafkaConfig();
}
@@ -85,10 +85,10 @@ public class HoodieWriteCommitKafkaCallback implements HoodieWriteCommitCallback
kafkaProducerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
// default "all" to ensure no message loss
kafkaProducerProps.setProperty(ProducerConfig.ACKS_CONFIG, hoodieConfig
.getString(CALLBACK_KAFKA_ACKS));
.getString(ACKS));
// retries 3 times by default
kafkaProducerProps.setProperty(ProducerConfig.RETRIES_CONFIG, hoodieConfig
.getString(CALLBACK_KAFKA_RETRIES));
.getString(RETRIES));
kafkaProducerProps.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
"org.apache.kafka.common.serialization.StringSerializer");
kafkaProducerProps.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,
@@ -109,13 +109,13 @@ public class HoodieWriteCommitKafkaCallback implements HoodieWriteCommitCallback
* @return Callback {@link ProducerRecord}
*/
private ProducerRecord<String, String> buildProducerRecord(HoodieConfig hoodieConfig, String callbackMsg) {
String partition = hoodieConfig.getString(CALLBACK_KAFKA_PARTITION);
String partition = hoodieConfig.getString(PARTITION);
if (null != partition) {
return new ProducerRecord<String, String>(topic, Integer.valueOf(partition), hoodieConfig
.getString(TABLE_NAME),
.getString(TBL_NAME),
callbackMsg);
} else {
return new ProducerRecord<String, String>(topic, hoodieConfig.getString(TABLE_NAME), callbackMsg);
return new ProducerRecord<String, String>(topic, hoodieConfig.getString(TBL_NAME), callbackMsg);
}
}
@@ -125,9 +125,9 @@ public class HoodieWriteCommitKafkaCallback implements HoodieWriteCommitCallback
*/
private void validateKafkaConfig() {
ValidationUtils.checkArgument(!StringUtils.isNullOrEmpty(bootstrapServers), String.format("Config %s can not be "
+ "null or empty", CALLBACK_KAFKA_BOOTSTRAP_SERVERS.key()));
+ "null or empty", BOOTSTRAP_SERVERS.key()));
ValidationUtils.checkArgument(!StringUtils.isNullOrEmpty(topic), String.format("Config %s can not be null or empty",
CALLBACK_KAFKA_TOPIC.key()));
TOPIC.key()));
}
/**

View File

@@ -32,19 +32,19 @@ import static org.apache.hudi.config.HoodieWriteCommitCallbackConfig.CALLBACK_PR
description = "Controls notifications sent to Kafka, on events happening to a hudi table.")
public class HoodieWriteCommitKafkaCallbackConfig extends HoodieConfig {
public static final ConfigProperty<String> CALLBACK_KAFKA_BOOTSTRAP_SERVERS = ConfigProperty
public static final ConfigProperty<String> BOOTSTRAP_SERVERS = ConfigProperty
.key(CALLBACK_PREFIX + "kafka.bootstrap.servers")
.noDefaultValue()
.sinceVersion("0.7.0")
.withDocumentation("Bootstrap servers of kafka cluster, to be used for publishing commit metadata.");
public static final ConfigProperty<String> CALLBACK_KAFKA_TOPIC = ConfigProperty
public static final ConfigProperty<String> TOPIC = ConfigProperty
.key(CALLBACK_PREFIX + "kafka.topic")
.noDefaultValue()
.sinceVersion("0.7.0")
.withDocumentation("Kafka topic name to publish timeline activity into.");
public static final ConfigProperty<String> CALLBACK_KAFKA_PARTITION = ConfigProperty
public static final ConfigProperty<String> PARTITION = ConfigProperty
.key(CALLBACK_PREFIX + "kafka.partition")
.noDefaultValue()
.sinceVersion("0.7.0")
@@ -52,13 +52,13 @@ public class HoodieWriteCommitKafkaCallbackConfig extends HoodieConfig {
+ " for providing strict ordering. By default, Kafka messages are keyed by table name, which "
+ " guarantees ordering at the table level, but not globally (or when new partitions are added)");
public static final ConfigProperty<String> CALLBACK_KAFKA_ACKS = ConfigProperty
public static final ConfigProperty<String> ACKS = ConfigProperty
.key(CALLBACK_PREFIX + "kafka.acks")
.defaultValue("all")
.sinceVersion("0.7.0")
.withDocumentation("kafka acks level, all by default to ensure strong durability.");
public static final ConfigProperty<Integer> CALLBACK_KAFKA_RETRIES = ConfigProperty
public static final ConfigProperty<Integer> RETRIES = ConfigProperty
.key(CALLBACK_PREFIX + "kafka.retries")
.defaultValue(3)
.sinceVersion("0.7.0")
@@ -68,8 +68,43 @@ public class HoodieWriteCommitKafkaCallbackConfig extends HoodieConfig {
* Set default value for {@link HoodieWriteCommitKafkaCallbackConfig} if needed.
*/
public static void setCallbackKafkaConfigIfNeeded(HoodieConfig config) {
config.setDefaultValue(CALLBACK_KAFKA_ACKS);
config.setDefaultValue(CALLBACK_KAFKA_RETRIES);
config.setDefaultValue(ACKS);
config.setDefaultValue(RETRIES);
}
/**
* @deprecated Use {@link #BOOTSTRAP_SERVERS} and its methods.
*/
@Deprecated
public static final String CALLBACK_KAFKA_BOOTSTRAP_SERVERS = BOOTSTRAP_SERVERS.key();
/**
* @deprecated Use {@link #TOPIC} and its methods.
*/
@Deprecated
public static final String CALLBACK_KAFKA_TOPIC = TOPIC.key();
/**
* @deprecated Use {@link #PARTITION} and its methods.
*/
@Deprecated
public static final String CALLBACK_KAFKA_PARTITION = PARTITION.key();
/**
* @deprecated Use {@link #ACKS} and its methods.
*/
@Deprecated
public static final String CALLBACK_KAFKA_ACKS = ACKS.key();
/**
* @deprecated Use {@link #ACKS} and its methods.
*/
@Deprecated
public static final String DEFAULT_CALLBACK_KAFKA_ACKS = ACKS.defaultValue();
/**
* @deprecated Use {@link #RETRIES} and its methods.
*/
@Deprecated
public static final String CALLBACK_KAFKA_RETRIES = RETRIES.key();
/**
* @deprecated Use {@link #RETRIES} and its methods.
*/
@Deprecated
public static final int DEFAULT_CALLBACK_KAFKA_RETRIES = RETRIES.defaultValue();
}

View File

@@ -49,7 +49,7 @@ import java.io.IOException;
import java.io.Serializable;
import java.util.HashMap;
import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP;
import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER;
/**
* Performs bootstrap from a non-hudi source.
@@ -111,14 +111,14 @@ public class BootstrapExecutor implements Serializable {
this.configuration = conf;
this.props = properties;
ValidationUtils.checkArgument(properties.containsKey(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP
ValidationUtils.checkArgument(properties.containsKey(HoodieTableConfig.BOOTSTRAP_BASE_PATH
.key()),
HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP.key() + " must be specified.");
this.bootstrapBasePath = properties.getString(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP.key());
HoodieTableConfig.BOOTSTRAP_BASE_PATH.key() + " must be specified.");
this.bootstrapBasePath = properties.getString(HoodieTableConfig.BOOTSTRAP_BASE_PATH.key());
// Add more defaults if full bootstrap requested
this.props.putIfAbsent(DataSourceWriteOptions.PAYLOAD_CLASS().key(),
DataSourceWriteOptions.PAYLOAD_CLASS().defaultValue());
this.props.putIfAbsent(DataSourceWriteOptions.PAYLOAD_CLASS_NAME().key(),
DataSourceWriteOptions.PAYLOAD_CLASS_NAME().defaultValue());
this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc);
HoodieWriteConfig.Builder builder =
HoodieWriteConfig.newBuilder().withPath(cfg.targetBasePath)
@@ -175,7 +175,7 @@ public class BootstrapExecutor implements Serializable {
HoodieTableMetaClient.withPropertyBuilder()
.setTableType(cfg.tableType)
.setTableName(cfg.targetTableName)
.setArchiveLogFolder(HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue())
.setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue())
.setPayloadClassName(cfg.payloadClassName)
.setBaseFileFormat(cfg.baseFileFormat)
.setBootstrapIndexClass(cfg.bootstrapIndexClass)

View File

@@ -99,13 +99,13 @@ 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.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER;
import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE;
import static org.apache.hudi.config.HoodieClusteringConfig.INLINE_CLUSTERING;
import static org.apache.hudi.config.HoodieCompactionConfig.INLINE_COMPACT;
import static org.apache.hudi.config.HoodieWriteConfig.AUTO_COMMIT_ENABLE;
import static org.apache.hudi.config.HoodieWriteConfig.COMBINE_BEFORE_INSERT;
import static org.apache.hudi.config.HoodieWriteConfig.COMBINE_BEFORE_UPSERT;
import static org.apache.hudi.config.HoodieWriteConfig.HOODIE_AUTO_COMMIT;
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;
@@ -252,14 +252,14 @@ public class DeltaSync implements Serializable {
HoodieTableMetaClient.withPropertyBuilder()
.setTableType(cfg.tableType)
.setTableName(cfg.targetTableName)
.setArchiveLogFolder(HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue())
.setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue())
.setPayloadClassName(cfg.payloadClassName)
.setBaseFileFormat(cfg.baseFileFormat)
.setPartitionFields(partitionColumns)
.setRecordKeyFields(props.getProperty(DataSourceWriteOptions.RECORDKEY_FIELD().key()))
.setPopulateMetaFields(props.getBoolean(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(),
Boolean.parseBoolean(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue())))
.setKeyGeneratorClassProp(props.getProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(),
.setPopulateMetaFields(props.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS.key(),
Boolean.parseBoolean(HoodieTableConfig.POPULATE_META_FIELDS.defaultValue())))
.setKeyGeneratorClassProp(props.getProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(),
SimpleKeyGenerator.class.getName()))
.setPreCombineField(cfg.sourceOrderingField)
.initTable(new Configuration(jssc.hadoopConfiguration()),
@@ -356,14 +356,14 @@ public class DeltaSync implements Serializable {
HoodieTableMetaClient.withPropertyBuilder()
.setTableType(cfg.tableType)
.setTableName(cfg.targetTableName)
.setArchiveLogFolder(HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue())
.setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue())
.setPayloadClassName(cfg.payloadClassName)
.setBaseFileFormat(cfg.baseFileFormat)
.setPartitionFields(partitionColumns)
.setRecordKeyFields(props.getProperty(DataSourceWriteOptions.RECORDKEY_FIELD().key()))
.setPopulateMetaFields(props.getBoolean(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(),
Boolean.parseBoolean(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue())))
.setKeyGeneratorClassProp(props.getProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(),
.setPopulateMetaFields(props.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS.key(),
Boolean.parseBoolean(HoodieTableConfig.POPULATE_META_FIELDS.defaultValue())))
.setKeyGeneratorClassProp(props.getProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(),
SimpleKeyGenerator.class.getName()))
.initTable(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath);
}
@@ -711,7 +711,7 @@ public class DeltaSync implements Serializable {
ValidationUtils.checkArgument(config.isAsyncClusteringEnabled() == cfg.isAsyncClusteringEnabled(),
String.format("%s should be set to %s", ASYNC_CLUSTERING_ENABLE.key(), cfg.isAsyncClusteringEnabled()));
ValidationUtils.checkArgument(!config.shouldAutoCommit(),
String.format("%s should be set to %s", HOODIE_AUTO_COMMIT.key(), autoCommit));
String.format("%s should be set to %s", AUTO_COMMIT_ENABLE.key(), autoCommit));
ValidationUtils.checkArgument(config.shouldCombineBeforeInsert() == cfg.filterDupes,
String.format("%s should be set to %s", COMBINE_BEFORE_INSERT.key(), cfg.filterDupes));
ValidationUtils.checkArgument(config.shouldCombineBeforeUpsert(),

View File

@@ -297,10 +297,10 @@ public class TimelineServerPerf implements Serializable {
@Parameter(names = {"--base-store-path", "-sp"},
description = "Directory where spilled view entries will be stored. Used for SPILLABLE_DISK storage type")
public String baseStorePathForFileGroups = FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue();
public String baseStorePathForFileGroups = FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue();
@Parameter(names = {"--rocksdb-path", "-rp"}, description = "Root directory for RocksDB")
public String rocksDBPath = FileSystemViewStorageConfig.ROCKSDB_BASE_PATH_PROP.defaultValue();
public String rocksDBPath = FileSystemViewStorageConfig.ROCKSDB_BASE_PATH.defaultValue();
@Parameter(names = {"--wait-for-manual-queries", "-ww"})
public Boolean waitForManualQueries = false;
@@ -309,7 +309,7 @@ public class TimelineServerPerf implements Serializable {
public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
@Parameter(names = {"--verify-metadata-file-listing"}, description = "Verify file listing from Hudi's metadata against file system")
public Boolean verifyMetadataFileListing = HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue();
public Boolean verifyMetadataFileListing = HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue();
@Parameter(names = {"--help", "-h"})
public Boolean help = false;

View File

@@ -34,7 +34,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
public class TestKafkaConnectHdfsProvider extends HoodieCommonTestHarness {
private static final String BASE_FILE_EXTENSION = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension();
private static final String BASE_FILE_EXTENSION = HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension();
@Test
public void testValidKafkaConnectPath() throws Exception {

View File

@@ -983,13 +983,13 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING.key(), inlineCluster));
}
if (!StringUtils.isNullOrEmpty(inlineClusterMaxCommit)) {
configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT.key(), inlineClusterMaxCommit));
configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMITS.key(), inlineClusterMaxCommit));
}
if (!StringUtils.isNullOrEmpty(asyncCluster)) {
configs.add(String.format("%s=%s", HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE.key(), asyncCluster));
}
if (!StringUtils.isNullOrEmpty(asyncClusterMaxCommit)) {
configs.add(String.format("%s=%s", HoodieClusteringConfig.ASYNC_CLUSTERING_MAX_COMMIT_PROP.key(), asyncClusterMaxCommit));
configs.add(String.format("%s=%s", HoodieClusteringConfig.ASYNC_CLUSTERING_MAX_COMMITS.key(), asyncClusterMaxCommit));
}
return configs;
}
@@ -1250,7 +1250,7 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
props.load(inputStream);
}
assertEquals(new HoodieConfig(props).getString(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP), DummyAvroPayload.class.getName());
assertEquals(new HoodieConfig(props).getString(HoodieTableConfig.PAYLOAD_CLASS_NAME), DummyAvroPayload.class.getName());
}
@Test
@@ -1276,7 +1276,7 @@ public class TestHoodieDeltaStreamer extends TestHoodieDeltaStreamerBase {
props.load(inputStream);
}
assertFalse(props.containsKey(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP.key()));
assertFalse(props.containsKey(HoodieTableConfig.PAYLOAD_CLASS_NAME.key()));
}
@Test

View File

@@ -111,7 +111,7 @@ public class TestHoodieMultiTableDeltaStreamer extends TestHoodieDeltaStreamerBa
assertEquals("uber_db.dummy_table_uber", executionContext.getConfig().targetTableName);
assertEquals("topic1", executionContext.getProperties().getString(HoodieMultiTableDeltaStreamer.Constants.KAFKA_TOPIC_PROP));
assertEquals("_row_key", executionContext.getProperties().getString(DataSourceWriteOptions.RECORDKEY_FIELD().key()));
assertEquals(TestHoodieDeltaStreamer.TestGenerator.class.getName(), executionContext.getProperties().getString(DataSourceWriteOptions.KEYGENERATOR_CLASS().key()));
assertEquals(TestHoodieDeltaStreamer.TestGenerator.class.getName(), executionContext.getProperties().getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key()));
assertEquals("uber_hive_dummy_table", executionContext.getProperties().getString(HoodieMultiTableDeltaStreamer.Constants.HIVE_SYNC_TABLE_PROP));
}
@@ -224,11 +224,11 @@ public class TestHoodieMultiTableDeltaStreamer extends TestHoodieDeltaStreamerBa
tableExecutionContexts.forEach(tableExecutionContext -> {
switch (tableExecutionContext.getTableName()) {
case "dummy_table_short_trip":
String tableLevelKeyGeneratorClass = tableExecutionContext.getProperties().getString(DataSourceWriteOptions.KEYGENERATOR_CLASS().key());
String tableLevelKeyGeneratorClass = tableExecutionContext.getProperties().getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key());
assertEquals(TestHoodieDeltaStreamer.TestTableLevelGenerator.class.getName(), tableLevelKeyGeneratorClass);
break;
default:
String defaultKeyGeneratorClass = tableExecutionContext.getProperties().getString(DataSourceWriteOptions.KEYGENERATOR_CLASS().key());
String defaultKeyGeneratorClass = tableExecutionContext.getProperties().getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key());
assertEquals(TestHoodieDeltaStreamer.TestGenerator.class.getName(), defaultKeyGeneratorClass);
}
});

View File

@@ -70,7 +70,7 @@ public class TestHoodieSnapshotCopier extends FunctionalTestHarness {
HoodieSnapshotCopier copier = new HoodieSnapshotCopier();
copier.snapshot(jsc(), basePath, outputPath, true,
HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue());
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue());
// Nothing changed; we just bail out
assertEquals(fs.listStatus(new Path(basePath)).length, 1);
@@ -124,7 +124,7 @@ public class TestHoodieSnapshotCopier extends FunctionalTestHarness {
// Do a snapshot copy
HoodieSnapshotCopier copier = new HoodieSnapshotCopier();
copier.snapshot(jsc(), basePath, outputPath, false, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue());
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue());
// Check results
assertTrue(fs.exists(new Path(outputPath + "/2016/05/01/" + file11.getName())));