1
0

[HUDI-1878] Add max memory option for flink writer task (#2920)

Also removes the rate limiter because it has the similar functionality,
modify the create and merge handle cleans the retry files automatically.
This commit is contained in:
Danny Chan
2021-05-08 14:27:56 +08:00
committed by GitHub
parent 2c5a661a64
commit bfbf993cbe
11 changed files with 298 additions and 224 deletions

View File

@@ -436,6 +436,68 @@ public class TestWriteCopyOnWrite {
checkWrittenData(tempFile, expected, 1);
}
@Test
public void testInsertWithSmallBufferSize() throws Exception {
// reset the config option
conf.setDouble(FlinkOptions.WRITE_TASK_MAX_SIZE, 200.001); // 1Kb buffer size
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
// open the function and ingest data
funcWrapper.openFunction();
// each record is 424 bytes. so 3 records expect to trigger buffer flush:
// flush the max size bucket once at a time.
for (RowData rowData : TestData.DATA_SET_INSERT_DUPLICATES) {
funcWrapper.invoke(rowData);
}
Map<String, List<HoodieRecord>> dataBuffer = funcWrapper.getDataBuffer();
assertThat("Should have 1 data bucket", dataBuffer.size(), is(1));
assertThat("2 records expect to flush out as a mini-batch",
dataBuffer.values().stream().findFirst().map(List::size).orElse(-1),
is(3));
// this triggers the data write and event send
funcWrapper.checkpointFunction(1);
dataBuffer = funcWrapper.getDataBuffer();
assertThat("All data should be flushed out", dataBuffer.size(), is(0));
for (int i = 0; i < 2; i++) {
final OperatorEvent event = funcWrapper.getNextEvent(); // remove the first event first
assertThat("The operator expect to send an event", event, instanceOf(BatchWriteSuccessEvent.class));
funcWrapper.getCoordinator().handleEventFromOperator(0, event);
}
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
String instant = funcWrapper.getWriteClient()
.getLastPendingInstant(getTableType());
funcWrapper.checkpointComplete(1);
Map<String, String> expected = getMiniBatchExpected();
checkWrittenData(tempFile, expected, 1);
// started a new instant already
checkInflightInstant(funcWrapper.getWriteClient());
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
// insert duplicates again
for (RowData rowData : TestData.DATA_SET_INSERT_DUPLICATES) {
funcWrapper.invoke(rowData);
}
funcWrapper.checkpointFunction(2);
for (int i = 0; i < 2; i++) {
final OperatorEvent event = funcWrapper.getNextEvent(); // remove the first event first
funcWrapper.getCoordinator().handleEventFromOperator(0, event);
}
funcWrapper.checkpointComplete(2);
// Same the original base file content.
checkWrittenData(tempFile, expected, 1);
}
Map<String, String> getMiniBatchExpected() {
Map<String, String> expected = new HashMap<>();
expected.put("par1", "[id1,par1,id1,Danny,23,1,par1, "

View File

@@ -1,83 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.sink.transform;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.sink.utils.MockStreamingRuntimeContext;
import org.apache.hudi.utils.TestConfigurations;
import org.apache.hudi.utils.TestData;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.table.data.RowData;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import java.io.File;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Test cases for {@link RowDataToHoodieFunction}.
*/
public class TestRowDataToHoodieFunction {
@TempDir
File tempFile;
private Configuration conf;
@BeforeEach
public void before() {
final String basePath = tempFile.getAbsolutePath();
conf = TestConfigurations.getDefaultConf(basePath);
}
@Test
void testRateLimit() throws Exception {
// at most 100 record per second
RowDataToHoodieFunction<RowData, ?> func1 = getFunc(100);
long instant1 = System.currentTimeMillis();
for (RowData rowData : TestData.DATA_SET_INSERT_DUPLICATES) {
func1.map(rowData);
}
long instant2 = System.currentTimeMillis();
long processTime1 = instant2 - instant1;
// at most 1 record per second
RowDataToHoodieFunction<RowData, ?> func2 = getFunc(1);
long instant3 = System.currentTimeMillis();
for (RowData rowData : TestData.DATA_SET_INSERT_DUPLICATES) {
func2.map(rowData);
}
long instant4 = System.currentTimeMillis();
long processTime2 = instant4 - instant3;
assertTrue(processTime2 > processTime1, "lower rate should have longer process time");
assertTrue(processTime2 > 5000, "should process at least 5 seconds");
}
private RowDataToHoodieFunction<RowData, ?> getFunc(long rate) throws Exception {
conf.setLong(FlinkOptions.WRITE_RATE_LIMIT, rate);
RowDataToHoodieFunction<RowData, ?> func =
new RowDataToHoodieFunction<>(TestConfigurations.ROW_TYPE, conf);
func.setRuntimeContext(new MockStreamingRuntimeContext(false, 1, 1));
func.open(conf);
return func;
}
}

View File

@@ -163,6 +163,7 @@ public class StreamWriteFunctionWrapper<I> {
functionInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId);
coordinator.notifyCheckpointComplete(checkpointId);
this.bucketAssignerFunction.notifyCheckpointComplete(checkpointId);
this.writeFunction.notifyCheckpointComplete(checkpointId);
if (conf.getBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED)) {
try {
compactFunctionWrapper.compact(checkpointId);