|
54 | 54 | import org.apache.hudi.common.util.ReflectionUtils;
|
55 | 55 | import org.apache.hudi.common.util.StringUtils;
|
56 | 56 | import org.apache.hudi.common.util.ValidationUtils;
|
| 57 | +import org.apache.hudi.common.util.VisibleForTesting; |
| 58 | +import org.apache.hudi.common.util.queue.DisruptorWaitStrategyType; |
57 | 59 | import org.apache.hudi.common.util.queue.ExecutorType;
|
58 | 60 | import org.apache.hudi.config.metrics.HoodieMetricsCloudWatchConfig;
|
59 | 61 | import org.apache.hudi.config.metrics.HoodieMetricsConfig;
|
|
97 | 99 | import java.util.function.Supplier;
|
98 | 100 | import java.util.stream.Collectors;
|
99 | 101 |
|
100 |
| -import static org.apache.hudi.common.util.queue.ExecutorType.BOUNDED_IN_MEMORY; |
101 |
| -import static org.apache.hudi.common.util.queue.ExecutorType.DISRUPTOR; |
| 102 | +import static org.apache.hudi.common.util.queue.ExecutorType.SIMPLE; |
102 | 103 | import static org.apache.hudi.config.HoodieCleanConfig.CLEANER_POLICY;
|
103 | 104 | import static org.apache.hudi.table.marker.ConflictDetectionUtils.getDefaultEarlyConflictDetectionStrategy;
|
104 | 105 |
|
@@ -158,10 +159,10 @@ public class HoodieWriteConfig extends HoodieConfig {
|
158 | 159 | .withDocumentation("Key generator class, that implements `org.apache.hudi.keygen.KeyGenerator` "
|
159 | 160 | + "extract a key out of incoming records.");
|
160 | 161 |
|
161 |
| - public static final ConfigProperty<String> EXECUTOR_TYPE = ConfigProperty |
| 162 | + public static final ConfigProperty<String> WRITE_EXECUTOR_TYPE = ConfigProperty |
162 | 163 | .key("hoodie.write.executor.type")
|
163 |
| - .defaultValue(BOUNDED_IN_MEMORY.name()) |
164 |
| - .withValidValues(BOUNDED_IN_MEMORY.name(), DISRUPTOR.name()) |
| 164 | + .defaultValue(SIMPLE.name()) |
| 165 | + .withValidValues(Arrays.stream(ExecutorType.values()).map(Enum::name).toArray(String[]::new)) |
165 | 166 | .sinceVersion("0.13.0")
|
166 | 167 | .withDocumentation("Set executor which orchestrates concurrent producers and consumers communicating through a message queue."
|
167 | 168 | + "BOUNDED_IN_MEMORY(default): Use LinkedBlockingQueue as a bounded in-memory queue, this queue will use extra lock to balance producers and consumer"
|
@@ -271,15 +272,15 @@ public class HoodieWriteConfig extends HoodieConfig {
|
271 | 272 | .defaultValue(String.valueOf(4 * 1024 * 1024))
|
272 | 273 | .withDocumentation("Size of in-memory buffer used for parallelizing network reads and lake storage writes.");
|
273 | 274 |
|
274 |
| - public static final ConfigProperty<String> WRITE_DISRUPTOR_BUFFER_SIZE = ConfigProperty |
| 275 | + public static final ConfigProperty<String> WRITE_EXECUTOR_DISRUPTOR_BUFFER_SIZE = ConfigProperty |
275 | 276 | .key("hoodie.write.executor.disruptor.buffer.size")
|
276 | 277 | .defaultValue(String.valueOf(1024))
|
277 | 278 | .sinceVersion("0.13.0")
|
278 | 279 | .withDocumentation("The size of the Disruptor Executor ring buffer, must be power of 2");
|
279 | 280 |
|
280 |
| - public static final ConfigProperty<String> WRITE_WAIT_STRATEGY = ConfigProperty |
| 281 | + public static final ConfigProperty<String> WRITE_EXECUTOR_DISRUPTOR_WAIT_STRATEGY = ConfigProperty |
281 | 282 | .key("hoodie.write.executor.disruptor.wait.strategy")
|
282 |
| - .defaultValue("BLOCKING_WAIT") |
| 283 | + .defaultValue(DisruptorWaitStrategyType.BLOCKING_WAIT.name()) |
283 | 284 | .sinceVersion("0.13.0")
|
284 | 285 | .withDocumentation("Strategy employed for making Disruptor Executor wait on a cursor. Other options are "
|
285 | 286 | + "SLEEPING_WAIT, it attempts to be conservative with CPU usage by using a simple busy wait loop"
|
@@ -1107,7 +1108,7 @@ public String getKeyGeneratorClass() {
|
1107 | 1108 | }
|
1108 | 1109 |
|
1109 | 1110 | public ExecutorType getExecutorType() {
|
1110 |
| - return ExecutorType.valueOf(getStringOrDefault(EXECUTOR_TYPE).toUpperCase(Locale.ROOT)); |
| 1111 | + return ExecutorType.valueOf(getStringOrDefault(WRITE_EXECUTOR_TYPE).toUpperCase(Locale.ROOT)); |
1111 | 1112 | }
|
1112 | 1113 |
|
1113 | 1114 | public boolean isCDCEnabled() {
|
@@ -1175,12 +1176,12 @@ public int getWriteBufferLimitBytes() {
|
1175 | 1176 | return Integer.parseInt(getStringOrDefault(WRITE_BUFFER_LIMIT_BYTES_VALUE));
|
1176 | 1177 | }
|
1177 | 1178 |
|
1178 |
| - public Option<String> getWriteExecutorWaitStrategy() { |
1179 |
| - return Option.of(getString(WRITE_WAIT_STRATEGY)); |
| 1179 | + public String getWriteExecutorDisruptorWaitStrategy() { |
| 1180 | + return getStringOrDefault(WRITE_EXECUTOR_DISRUPTOR_WAIT_STRATEGY); |
1180 | 1181 | }
|
1181 | 1182 |
|
1182 |
| - public Option<Integer> getDisruptorWriteBufferSize() { |
1183 |
| - return Option.of(Integer.parseInt(getStringOrDefault(WRITE_DISRUPTOR_BUFFER_SIZE))); |
| 1183 | + public Integer getWriteExecutorDisruptorWriteBufferSize() { |
| 1184 | + return Integer.parseInt(getStringOrDefault(WRITE_EXECUTOR_DISRUPTOR_BUFFER_SIZE)); |
1184 | 1185 | }
|
1185 | 1186 |
|
1186 | 1187 | public boolean shouldCombineBeforeInsert() {
|
@@ -1987,7 +1988,7 @@ public ApiSite getDatadogApiSite() {
|
1987 | 1988 | public String getDatadogApiKey() {
|
1988 | 1989 | if (props.containsKey(HoodieMetricsDatadogConfig.API_KEY.key())) {
|
1989 | 1990 | return getString(HoodieMetricsDatadogConfig.API_KEY);
|
1990 |
| - |
| 1991 | + |
1991 | 1992 | } else {
|
1992 | 1993 | Supplier<String> apiKeySupplier = ReflectionUtils.loadClass(
|
1993 | 1994 | getString(HoodieMetricsDatadogConfig.API_KEY_SUPPLIER));
|
@@ -2481,7 +2482,7 @@ public Builder withKeyGenerator(String keyGeneratorClass) {
|
2481 | 2482 | }
|
2482 | 2483 |
|
2483 | 2484 | public Builder withExecutorType(String executorClass) {
|
2484 |
| - writeConfig.setValue(EXECUTOR_TYPE, executorClass); |
| 2485 | + writeConfig.setValue(WRITE_EXECUTOR_TYPE, executorClass); |
2485 | 2486 | return this;
|
2486 | 2487 | }
|
2487 | 2488 |
|
@@ -2536,13 +2537,13 @@ public Builder withWriteBufferLimitBytes(int writeBufferLimit) {
|
2536 | 2537 | return this;
|
2537 | 2538 | }
|
2538 | 2539 |
|
2539 |
| - public Builder withWriteWaitStrategy(String waitStrategy) { |
2540 |
| - writeConfig.setValue(WRITE_WAIT_STRATEGY, String.valueOf(waitStrategy)); |
| 2540 | + public Builder withWriteExecutorDisruptorWaitStrategy(String waitStrategy) { |
| 2541 | + writeConfig.setValue(WRITE_EXECUTOR_DISRUPTOR_WAIT_STRATEGY, String.valueOf(waitStrategy)); |
2541 | 2542 | return this;
|
2542 | 2543 | }
|
2543 | 2544 |
|
2544 |
| - public Builder withWriteBufferSize(int size) { |
2545 |
| - writeConfig.setValue(WRITE_DISRUPTOR_BUFFER_SIZE, String.valueOf(size)); |
| 2545 | + public Builder withWriteExecutorDisruptorWriteBufferSize(long size) { |
| 2546 | + writeConfig.setValue(WRITE_EXECUTOR_DISRUPTOR_BUFFER_SIZE, String.valueOf(size)); |
2546 | 2547 | return this;
|
2547 | 2548 | }
|
2548 | 2549 |
|
@@ -2970,8 +2971,15 @@ private void validate() {
|
2970 | 2971 | }
|
2971 | 2972 |
|
2972 | 2973 | public HoodieWriteConfig build() {
|
| 2974 | + return build(true); |
| 2975 | + } |
| 2976 | + |
| 2977 | + @VisibleForTesting |
| 2978 | + public HoodieWriteConfig build(boolean shouldValidate) { |
2973 | 2979 | setDefaults();
|
2974 |
| - validate(); |
| 2980 | + if (shouldValidate) { |
| 2981 | + validate(); |
| 2982 | + } |
2975 | 2983 | // Build WriteConfig at the end
|
2976 | 2984 | return new HoodieWriteConfig(engineType, writeConfig.getProps());
|
2977 | 2985 | }
|
|
0 commit comments