Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -482,12 +482,6 @@ private FlinkOptions() {
// Compaction Options
// ------------------------------------------------------------------------

public static final ConfigOption<Double> COMPACTION_MEMORY_FRACTION_PROP = ConfigOptions
.key(HoodieRealtimeConfig.COMPACTION_MEMORY_FRACTION_PROP)
.doubleType()
.defaultValue(0.1)
.withDescription("Compaction memory fraction of Task Manager managed memory size, default 0.1.");

public static final ConfigOption<Boolean> COMPACTION_SCHEDULE_ENABLED = ConfigOptions
.key("compaction.schedule.enabled")
.booleanType()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,10 +109,6 @@ public class FlinkCompactionConfig extends Configuration {
description = "Min compaction interval of async compaction service, default 10 minutes")
public Integer minCompactionIntervalSeconds = 600;

@Parameter(names = {"--compaction-memory-fraction-prop"},
description = "Compaction memory fraction of Task Manager managed memory size, default 0.1")
public double compactionMemoryFractionProp = 0.1;

/**
* Transforms a {@code HoodieFlinkCompaction.config} into {@code Configuration}.
* The latter is more suitable for the table APIs. It reads all the properties
Expand All @@ -133,7 +129,6 @@ public static org.apache.flink.configuration.Configuration toFlinkConfig(FlinkCo
conf.setLong(FlinkOptions.COMPACTION_TARGET_IO, config.compactionTargetIo);
conf.setInteger(FlinkOptions.COMPACTION_TASKS, config.compactionTasks);
conf.setBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED, config.cleanAsyncEnable);
conf.setDouble(FlinkOptions.COMPACTION_MEMORY_FRACTION_PROP, config.compactionMemoryFractionProp);
// use synchronous compaction always
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false);
conf.setBoolean(FlinkOptions.COMPACTION_SCHEDULE_ENABLED, config.schedule);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -250,10 +250,6 @@ public class FlinkStreamerConfig extends Configuration {
@Parameter(names = {"--compaction-target-io"}, description = "Target IO per compaction (both read and write), default 500 GB")
public Long compactionTargetIo = 512000L;

@Parameter(names = {"--compaction-memory-fraction-prop"},
description = "Compaction memory fraction of Task Manager managed memory size, default 0.1")
public double compactionMemoryFractionProp = 0.1;

@Parameter(names = {"--clean-async-enabled"}, description = "Whether to cleanup the old commits immediately on new commits, enabled by default")
public Boolean cleanAsyncEnabled = true;

Expand Down Expand Up @@ -388,7 +384,6 @@ public static org.apache.flink.configuration.Configuration toFlinkConfig(FlinkSt
conf.setInteger(FlinkOptions.COMPACTION_DELTA_SECONDS, config.compactionDeltaSeconds);
conf.setInteger(FlinkOptions.COMPACTION_MAX_MEMORY, config.compactionMaxMemory);
conf.setLong(FlinkOptions.COMPACTION_TARGET_IO, config.compactionTargetIo);
conf.setDouble(FlinkOptions.COMPACTION_MEMORY_FRACTION_PROP, config.compactionMemoryFractionProp);
conf.setBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED, config.cleanAsyncEnabled);
conf.setInteger(FlinkOptions.CLEAN_RETAIN_COMMITS, config.cleanRetainCommits);
conf.setInteger(FlinkOptions.ARCHIVE_MAX_COMMITS, config.archiveMaxCommits);
Expand Down
12 changes: 1 addition & 11 deletions hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,6 @@

package org.apache.hudi.util;

import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
import org.apache.hudi.client.FlinkTaskContextSupplier;
import org.apache.hudi.client.HoodieFlinkWriteClient;
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
Expand Down Expand Up @@ -516,14 +514,6 @@ public static boolean haveSuccessfulCommits(HoodieTableMetaClient metaClient) {
* Returns the max compaction memory in bytes with given conf.
*/
public static long getMaxCompactionMemoryInBytes(Configuration conf) {
if (conf.contains(FlinkOptions.COMPACTION_MAX_MEMORY)) {
return conf.get(FlinkOptions.COMPACTION_MAX_MEMORY) * 1024 * 1024;
}
return (long)Math
.ceil(conf.getDouble(FlinkOptions.COMPACTION_MEMORY_FRACTION_PROP)
* TaskExecutorProcessUtils.processSpecFromConfig(
TaskExecutorProcessUtils.getConfigurationMapLegacyTaskManagerHeapSizeToConfigOption(
conf, TaskManagerOptions.TOTAL_PROCESS_MEMORY))
.getManagedMemorySize().getBytes());
return conf.getInteger(FlinkOptions.COMPACTION_MAX_MEMORY) * 1024 * 1024;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,6 @@ void beforeEach() throws IOException {
this.conf = new Configuration();
this.conf.setString(FlinkOptions.PATH, tempFile.getAbsolutePath());
this.conf.setString(FlinkOptions.TABLE_NAME, "t1");
this.conf.set(FlinkOptions.COMPACTION_MAX_MEMORY, 1024);
StreamerUtil.initTableIfNotExists(this.conf);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,6 @@ public class TestHoodieTableSource {
void beforeEach() throws Exception {
final String path = tempFile.getAbsolutePath();
conf = TestConfigurations.getDefaultConf(path);
conf.set(FlinkOptions.COMPACTION_MAX_MEMORY, 1024);
TestData.writeData(TestData.DATA_SET_INSERT, conf);
}

Expand Down Expand Up @@ -123,7 +122,6 @@ void testGetTableAvroSchema() {
final String path = tempFile.getAbsolutePath();
conf = TestConfigurations.getDefaultConf(path);
conf.setBoolean(FlinkOptions.READ_AS_STREAMING, true);
conf.set(FlinkOptions.COMPACTION_MAX_MEMORY, 1024);

HoodieTableSource tableSource = new HoodieTableSource(
TestConfigurations.TABLE_SCHEMA,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,6 @@ void beforeEach(HoodieTableType tableType, Map<String, String> options) throws I
conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setString(FlinkOptions.TABLE_TYPE, tableType.name());
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false); // close the async compaction
conf.set(FlinkOptions.COMPACTION_MAX_MEMORY, 1024);
options.forEach((key, value) -> conf.setString(key, value));

StreamerUtil.initTableIfNotExists(conf);
Expand Down