1
0

[HUDI-3772] Fixing auto adjustment of lock configs for deltastreamer (#5207)

This commit is contained in:
Sivabalan Narayanan
2022-04-02 23:44:10 -07:00
committed by GitHub
parent cc3737be50
commit 84064a9b08
5 changed files with 102 additions and 33 deletions

View File

@@ -43,8 +43,8 @@ import org.apache.hudi.exception.HoodieIOException;
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.exception.HoodieSourcePostProcessException;
import org.apache.hudi.utilities.exception.HoodieSchemaPostProcessException;
import org.apache.hudi.utilities.exception.HoodieSourcePostProcessException;
import org.apache.hudi.utilities.schema.ChainedSchemaPostProcessor;
import org.apache.hudi.utilities.schema.DelegatingSchemaProvider;
import org.apache.hudi.utilities.schema.RowBasedSchemaProvider;
@@ -272,6 +272,7 @@ public class UtilHelpers {
sparkConf.set("spark.eventLog.overwrite", "true");
sparkConf.set("spark.eventLog.enabled", "true");
}
sparkConf.set("spark.ui.port", "8090");
sparkConf.setIfMissing("spark.driver.maxResultSize", "2g");
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
sparkConf.set("spark.hadoop.mapred.output.compress", "true");

View File

@@ -35,6 +35,7 @@ import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieTableType;
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.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
@@ -127,7 +128,6 @@ public class HoodieDeltaStreamer implements Serializable {
public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf,
Option<TypedProperties> propsOverride) throws IOException {
this.properties = combineProperties(cfg, propsOverride, jssc.hadoopConfiguration());
if (cfg.initialCheckpointProvider != null && cfg.checkpoint == null) {
InitialCheckPointProvider checkPointProvider =
UtilHelpers.createInitialCheckpointProvider(cfg.initialCheckpointProvider, this.properties);
@@ -156,7 +156,14 @@ public class HoodieDeltaStreamer implements Serializable {
hoodieConfig.setAll(UtilHelpers.readConfig(hadoopConf, new Path(cfg.propsFilePath), cfg.configs).getProps());
}
// set any configs that Deltastreamer has to override explicitly
hoodieConfig.setDefaultValue(DataSourceWriteOptions.RECONCILE_SCHEMA());
// we need auto adjustment enabled for deltastreamer since async table services are feasible within the same JVM.
hoodieConfig.setValue(HoodieWriteConfig.AUTO_ADJUST_LOCK_CONFIGS.key(), "true");
if (cfg.tableType.equals(HoodieTableType.MERGE_ON_READ.name())) {
// Explicitly set the table type
hoodieConfig.setValue(HoodieTableConfig.TYPE.key(), HoodieTableType.MERGE_ON_READ.name());
}
return hoodieConfig.getProps(true);
}