yihua commented on a change in pull request #4090:
URL: https://github.com/apache/hudi/pull/4090#discussion_r756282739
##########
File path:
hudi-common/src/main/java/org/apache/hudi/common/config/DFSPropertiesConfiguration.java
##########
@@ -119,13 +123,13 @@ public void addPropsFromFile(Path filePath) {
if (visitedFilePaths.contains(filePath.toString())) {
throw new IllegalStateException("Loop detected; file " + filePath + "
already referenced");
}
- FileSystem fileSystem;
- try {
- fileSystem = fs != null ? fs : filePath.getFileSystem(new
Configuration());
- } catch (IOException e) {
- throw new IllegalArgumentException("Cannot get the file system from file
path", e);
- }
- try (BufferedReader reader = new BufferedReader(new
InputStreamReader(fileSystem.open(filePath)))) {
+
+ FileSystem fs = FSUtils.getFs(
+ filePath.toString(),
+ Option.ofNullable(hadoopConfig).orElseGet(Configuration::new)
+ );
Review comment:
@nsivabalan This is changed to get the `FileSystem` instance each time
for loading a props file (`fs` is passed in from the constructor before).
Since it's only for configuration, it shouldn't incur too much overhead?
##########
File path:
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -1150,11 +1150,11 @@ public int getArchiveDeleteParallelism() {
}
public boolean inlineClusteringEnabled() {
- return getBoolean(HoodieClusteringConfig.INLINE_CLUSTERING);
+ return getBooleanOrDefault(HoodieClusteringConfig.INLINE_CLUSTERING);
}
public boolean isAsyncClusteringEnabled() {
- return getBoolean(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE);
+ return getBooleanOrDefault(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE);
Review comment:
I believe the defaults are set when building the clustering and write
config so it's not necessary to `getBooleanOrDefault()`, since there is always
a value corresponding to each config key.
##########
File path:
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
##########
@@ -680,36 +682,49 @@ private HoodieWriteConfig
getHoodieClientConfig(SchemaProvider schemaProvider) {
private HoodieWriteConfig getHoodieClientConfig(Schema schema) {
final boolean combineBeforeUpsert = true;
final boolean autoCommit = false;
+
+ // NOTE: Provided that we're injection combined properties
+ // (from {@code props}, including CLI overrides), there's no
+ // need to explicitly set up some configuration aspects that
+ // are based on these (for ex Clustering configuration)
HoodieWriteConfig.Builder builder =
-
HoodieWriteConfig.newBuilder().withPath(cfg.targetBasePath).combineInput(cfg.filterDupes,
combineBeforeUpsert)
-
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName)
- // Inline compaction is disabled for continuous mode.
otherwise enabled for MOR
- .withInlineCompaction(cfg.isInlineCompactionEnabled()).build())
- .withClusteringConfig(HoodieClusteringConfig.newBuilder()
- .withInlineClustering(cfg.isInlineClusteringEnabled())
- .withAsyncClustering(cfg.isAsyncClusteringEnabled()).build())
-
.withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(cfg.sourceOrderingField)
- .build())
+ HoodieWriteConfig.newBuilder()
+ .withPath(cfg.targetBasePath)
+ .combineInput(cfg.filterDupes, combineBeforeUpsert)
+ .withCompactionConfig(
+ HoodieCompactionConfig.newBuilder()
+ .withPayloadClass(cfg.payloadClassName)
+ .withInlineCompaction(cfg.isInlineCompactionEnabled())
+ .build()
+ )
+ .withPayloadConfig(
+ HoodiePayloadConfig.newBuilder()
+ .withPayloadOrderingField(cfg.sourceOrderingField)
+ .build())
.forTable(cfg.targetTableName)
- .withAutoCommit(autoCommit).withProps(props);
+ .withAutoCommit(autoCommit)
+ .withProps(props);
- if (null != schema) {
- builder = builder.withSchema(schema.toString());
+ if (schema != null) {
+ builder.withSchema(schema.toString());
}
+
HoodieWriteConfig config = builder.build();
// set default value for {@link HoodieWriteCommitKafkaCallbackConfig} if
needed.
if (config.writeCommitCallbackOn() &&
HoodieWriteCommitKafkaCallback.class.getName().equals(config.getCallbackClass()))
{
HoodieWriteCommitKafkaCallbackConfig.setCallbackKafkaConfigIfNeeded(config);
}
+ HoodieClusteringConfig clusteringConfig =
HoodieClusteringConfig.from(props);
+
// Validate what deltastreamer assumes of write-config to be really safe
ValidationUtils.checkArgument(config.inlineCompactionEnabled() ==
cfg.isInlineCompactionEnabled(),
String.format("%s should be set to %s", INLINE_COMPACT.key(),
cfg.isInlineCompactionEnabled()));
- ValidationUtils.checkArgument(config.inlineClusteringEnabled() ==
cfg.isInlineClusteringEnabled(),
- String.format("%s should be set to %s", INLINE_CLUSTERING.key(),
cfg.isInlineClusteringEnabled()));
- ValidationUtils.checkArgument(config.isAsyncClusteringEnabled() ==
cfg.isAsyncClusteringEnabled(),
- String.format("%s should be set to %s", ASYNC_CLUSTERING_ENABLE.key(),
cfg.isAsyncClusteringEnabled()));
+ ValidationUtils.checkArgument(config.inlineClusteringEnabled() ==
clusteringConfig.isInlineClusteringEnabled(),
+ String.format("%s should be set to %s", INLINE_CLUSTERING.key(),
clusteringConfig.isInlineClusteringEnabled()));
+ ValidationUtils.checkArgument(config.isAsyncClusteringEnabled() ==
clusteringConfig.isAsyncClusteringEnabled(),
+ String.format("%s should be set to %s", ASYNC_CLUSTERING_ENABLE.key(),
clusteringConfig.isAsyncClusteringEnabled()));
Review comment:
To clarify, is this to make sure the properties loaded from the file
matches what's in the write config after config resolution?
##########
File path:
hudi-common/src/main/java/org/apache/hudi/common/config/DFSPropertiesConfiguration.java
##########
@@ -119,13 +123,13 @@ public void addPropsFromFile(Path filePath) {
if (visitedFilePaths.contains(filePath.toString())) {
throw new IllegalStateException("Loop detected; file " + filePath + "
already referenced");
}
- FileSystem fileSystem;
- try {
- fileSystem = fs != null ? fs : filePath.getFileSystem(new
Configuration());
- } catch (IOException e) {
- throw new IllegalArgumentException("Cannot get the file system from file
path", e);
- }
- try (BufferedReader reader = new BufferedReader(new
InputStreamReader(fileSystem.open(filePath)))) {
+
+ FileSystem fs = FSUtils.getFs(
+ filePath.toString(),
+ Option.ofNullable(hadoopConfig).orElseGet(Configuration::new)
+ );
Review comment:
Most changes in this PR are related to this so just to make sure there
is no hidden implication.
##########
File path:
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
##########
@@ -680,36 +682,49 @@ private HoodieWriteConfig
getHoodieClientConfig(SchemaProvider schemaProvider) {
private HoodieWriteConfig getHoodieClientConfig(Schema schema) {
final boolean combineBeforeUpsert = true;
final boolean autoCommit = false;
+
+ // NOTE: Provided that we're injection combined properties
Review comment:
nit: typo, `injection` -> `injecting`
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]