1
0

[HUDI-1757] Assigns the buckets by record key for Flink writer (#2757)

Currently we assign the buckets by record partition path which could
cause hotspot if the partition field is datetime type. Changes to assign
buckets by grouping the record whth their key first, the assignment is
valid if only there is no conflict(two task write to the same bucket).

This patch also changes the coordinator execution to be asynchronous.
This commit is contained in:
Danny Chan
2021-04-06 19:06:41 +08:00
committed by GitHub
parent 920537cac8
commit 9c369c607d
25 changed files with 638 additions and 400 deletions

View File

@@ -120,8 +120,8 @@ public class StreamWriteITCase extends TestLogger {
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)))
.setParallelism(4)
.map(new RowDataToHoodieFunction<>(rowType, conf), TypeInformation.of(HoodieRecord.class))
// Key-by partition path, to avoid multiple subtasks write to a partition at the same time
.keyBy(HoodieRecord::getPartitionPath)
// Key-by record key, to avoid multiple subtasks write to a bucket at the same time
.keyBy(HoodieRecord::getRecordKey)
.transform(
"bucket_assigner",
TypeInformation.of(HoodieRecord.class),
@@ -179,8 +179,8 @@ public class StreamWriteITCase extends TestLogger {
.name("instant_generator")
.uid("instant_generator_id")
// Keyby partition path, to avoid multiple subtasks writing to a partition at the same time
.keyBy(HoodieRecord::getPartitionPath)
// Key-by record key, to avoid multiple subtasks write to a bucket at the same time
.keyBy(HoodieRecord::getRecordKey)
// use the bucket assigner to generate bucket IDs
.transform(
"bucket_assigner",
@@ -249,8 +249,8 @@ public class StreamWriteITCase extends TestLogger {
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)))
.setParallelism(4)
.map(new RowDataToHoodieFunction<>(rowType, conf), TypeInformation.of(HoodieRecord.class))
// Key-by partition path, to avoid multiple subtasks write to a partition at the same time
.keyBy(HoodieRecord::getPartitionPath)
// Key-by record key, to avoid multiple subtasks write to a bucket at the same time
.keyBy(HoodieRecord::getRecordKey)
.transform(
"bucket_assigner",
TypeInformation.of(HoodieRecord.class),

View File

@@ -23,12 +23,15 @@ import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.sink.event.BatchWriteSuccessEvent;
import org.apache.hudi.sink.utils.MockCoordinatorExecutor;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.utils.TestConfigurations;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -42,11 +45,12 @@ import java.io.IOException;
import java.util.Collections;
import java.util.concurrent.CompletableFuture;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
@@ -60,9 +64,11 @@ public class TestStreamWriteOperatorCoordinator {
@BeforeEach
public void before() throws Exception {
OperatorCoordinator.Context context = new MockOperatorCoordinatorContext(new OperatorID(), 2);
coordinator = new StreamWriteOperatorCoordinator(
TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()), 2);
TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()), context);
coordinator.start();
coordinator.setExecutor(new MockCoordinatorExecutor(context));
}
@AfterEach
@@ -99,8 +105,8 @@ public class TestStreamWriteOperatorCoordinator {
coordinator.notifyCheckpointComplete(1);
String inflight = coordinator.getWriteClient()
.getInflightAndRequestedInstant("COPY_ON_WRITE");
String lastCompleted = coordinator.getWriteClient().getLastCompletedInstant("COPY_ON_WRITE");
.getInflightAndRequestedInstant(FlinkOptions.TABLE_TYPE_COPY_ON_WRITE);
String lastCompleted = coordinator.getWriteClient().getLastCompletedInstant(FlinkOptions.TABLE_TYPE_COPY_ON_WRITE);
assertThat("Instant should be complete", lastCompleted, is(instant));
assertNotEquals("", inflight, "Should start a new instant");
assertNotEquals(instant, inflight, "Should start a new instant");
@@ -131,27 +137,43 @@ public class TestStreamWriteOperatorCoordinator {
.instantTime("abc")
.writeStatus(Collections.emptyList())
.build();
assertThrows(IllegalStateException.class,
() -> coordinator.handleEventFromOperator(0, event),
assertError(() -> coordinator.handleEventFromOperator(0, event),
"Receive an unexpected event for instant abc from task 0");
}
@Test
public void testCheckpointCompleteWithException() {
public void testCheckpointCompleteWithPartialEvents() {
final CompletableFuture<byte[]> future = new CompletableFuture<>();
coordinator.checkpointCoordinator(1, future);
String inflightInstant = coordinator.getInstant();
String instant = coordinator.getInstant();
OperatorEvent event = BatchWriteSuccessEvent.builder()
.taskID(0)
.instantTime(inflightInstant)
.instantTime(instant)
.writeStatus(Collections.emptyList())
.build();
coordinator.handleEventFromOperator(0, event);
assertThrows(HoodieException.class,
() -> coordinator.notifyCheckpointComplete(1),
"org.apache.hudi.exception.HoodieException: Instant [20210330153432] has a complete checkpoint [1],\n"
+ "but the coordinator has not received full write success events,\n"
+ "rolls back the instant and rethrow");
assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(1),
"Returns early for empty write results");
String lastCompleted = coordinator.getWriteClient().getLastCompletedInstant(FlinkOptions.TABLE_TYPE_COPY_ON_WRITE);
assertNull(lastCompleted, "Returns early for empty write results");
assertNull(coordinator.getEventBuffer()[0]);
WriteStatus writeStatus1 = new WriteStatus(false, 0.2D);
writeStatus1.setPartitionPath("par2");
writeStatus1.setStat(new HoodieWriteStat());
OperatorEvent event1 = BatchWriteSuccessEvent.builder()
.taskID(1)
.instantTime(instant)
.writeStatus(Collections.singletonList(writeStatus1))
.isLastBatch(true)
.build();
coordinator.handleEventFromOperator(1, event1);
assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(2),
"Commits the instant with partial events anyway");
lastCompleted = coordinator.getWriteClient().getLastCompletedInstant(FlinkOptions.TABLE_TYPE_COPY_ON_WRITE);
assertThat("Commits the instant with partial events anyway", lastCompleted, is(instant));
}
@Test
@@ -159,8 +181,10 @@ public class TestStreamWriteOperatorCoordinator {
// override the default configuration
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setBoolean(FlinkOptions.HIVE_SYNC_ENABLED, true);
coordinator = new StreamWriteOperatorCoordinator(conf, 1);
OperatorCoordinator.Context context = new MockOperatorCoordinatorContext(new OperatorID(), 1);
coordinator = new StreamWriteOperatorCoordinator(conf, context);
coordinator.start();
coordinator.setExecutor(new MockCoordinatorExecutor(context));
String instant = coordinator.getInstant();
assertNotEquals("", instant);
@@ -180,4 +204,16 @@ public class TestStreamWriteOperatorCoordinator {
// never throw for hive synchronization now
assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(1));
}
// -------------------------------------------------------------------------
// Utilities
// -------------------------------------------------------------------------
private void assertError(Runnable runnable, String message) {
runnable.run();
// wait a little while for the task to finish
assertThat(coordinator.getContext(), instanceOf(MockOperatorCoordinatorContext.class));
MockOperatorCoordinatorContext context = (MockOperatorCoordinatorContext) coordinator.getContext();
assertTrue(context.isJobFailed(), message);
}
}

View File

@@ -25,7 +25,6 @@ import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.sink.event.BatchWriteSuccessEvent;
import org.apache.hudi.sink.utils.StreamWriteFunctionWrapper;
import org.apache.hudi.utils.TestConfigurations;
@@ -53,11 +52,10 @@ import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Test cases for StreamingSinkFunction.
* Test cases for stream write.
*/
public class TestWriteCopyOnWrite {
@@ -193,19 +191,19 @@ public class TestWriteCopyOnWrite {
assertThat(writeStatuses.size(), is(0)); // no data write
// fails the checkpoint
assertThrows(HoodieException.class,
() -> funcWrapper.checkpointFails(1),
"The last checkpoint was aborted, roll back the last write and throw");
funcWrapper.checkpointFails(1);
assertFalse(funcWrapper.getCoordinatorContext().isJobFailed(),
"The last checkpoint was aborted, ignore the events");
// the instant metadata should be cleared
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, null);
// the instant metadata should be reused
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, instant);
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, null);
for (RowData rowData : TestData.DATA_SET_INSERT) {
funcWrapper.invoke(rowData);
}
// this returns early cause there is no inflight instant
// this returns early because there is no inflight instant
funcWrapper.checkpointFunction(2);
// do not sent the write event and fails the checkpoint,
// behaves like the last checkpoint is successful.
@@ -501,16 +499,11 @@ public class TestWriteCopyOnWrite {
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, instant);
assertFalse(funcWrapper.isAllPartitionsLoaded(),
"All partitions assume to be loaded into the index state");
funcWrapper.checkpointComplete(2);
// the coordinator checkpoint commits the inflight instant.
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
checkWrittenData(tempFile, EXPECTED2);
// next element triggers all partitions load check
funcWrapper.invoke(TestData.DATA_SET_INSERT.get(0));
assertTrue(funcWrapper.isAllPartitionsLoaded(),
"All partitions assume to be loaded into the index state");
}
// -------------------------------------------------------------------------

View File

@@ -146,6 +146,55 @@ public class TestBucketAssigner {
assertBucketEquals(bucketInfo, "par3", BucketType.INSERT);
}
/**
* Test that only partial small files are assigned to the task.
*/
@Test
public void testInsertWithPartialSmallFiles() {
SmallFile f0 = new SmallFile();
f0.location = new HoodieRecordLocation("t0", "f0");
f0.sizeBytes = 12;
SmallFile f1 = new SmallFile();
f1.location = new HoodieRecordLocation("t0", "f1");
f1.sizeBytes = 122879; // no left space to append new records to this bucket
SmallFile f2 = new SmallFile();
f2.location = new HoodieRecordLocation("t0", "f2");
f2.sizeBytes = 56;
Map<String, List<SmallFile>> smallFilesMap = new HashMap<>();
smallFilesMap.put("par1", Arrays.asList(f0, f1, f2));
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(0, 2, context, writeConfig, smallFilesMap);
BucketInfo bucketInfo = mockBucketAssigner.addInsert("par1");
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
mockBucketAssigner.addInsert("par1");
bucketInfo = mockBucketAssigner.addInsert("par1");
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
bucketInfo = mockBucketAssigner.addInsert("par3");
assertBucketEquals(bucketInfo, "par3", BucketType.INSERT);
bucketInfo = mockBucketAssigner.addInsert("par3");
assertBucketEquals(bucketInfo, "par3", BucketType.INSERT);
MockBucketAssigner mockBucketAssigner2 = new MockBucketAssigner(1, 2, context, writeConfig, smallFilesMap);
BucketInfo bucketInfo2 = mockBucketAssigner2.addInsert("par1");
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f1");
mockBucketAssigner2.addInsert("par1");
bucketInfo2 = mockBucketAssigner2.addInsert("par1");
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f1");
bucketInfo2 = mockBucketAssigner2.addInsert("par3");
assertBucketEquals(bucketInfo2, "par3", BucketType.INSERT);
bucketInfo2 = mockBucketAssigner2.addInsert("par3");
assertBucketEquals(bucketInfo2, "par3", BucketType.INSERT);
}
@Test
public void testUpdateAndInsertWithSmallFiles() {
SmallFile f0 = new SmallFile();
@@ -187,6 +236,60 @@ public class TestBucketAssigner {
assertBucketEquals(bucketInfo, "par2", BucketType.UPDATE, "f2");
}
/**
* Test that only partial small files are assigned to the task.
*/
@Test
public void testUpdateAndInsertWithPartialSmallFiles() {
SmallFile f0 = new SmallFile();
f0.location = new HoodieRecordLocation("t0", "f0");
f0.sizeBytes = 12;
SmallFile f1 = new SmallFile();
f1.location = new HoodieRecordLocation("t0", "f1");
f1.sizeBytes = 122879; // no left space to append new records to this bucket
SmallFile f2 = new SmallFile();
f2.location = new HoodieRecordLocation("t0", "f2");
f2.sizeBytes = 56;
Map<String, List<SmallFile>> smallFilesMap = new HashMap<>();
smallFilesMap.put("par1", Arrays.asList(f0, f1, f2));
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(0, 2, context, writeConfig, smallFilesMap);
mockBucketAssigner.addUpdate("par1", "f0");
BucketInfo bucketInfo = mockBucketAssigner.addInsert("par1");
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
mockBucketAssigner.addInsert("par1");
bucketInfo = mockBucketAssigner.addInsert("par1");
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
mockBucketAssigner.addUpdate("par1", "f2");
mockBucketAssigner.addInsert("par1");
bucketInfo = mockBucketAssigner.addInsert("par1");
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
MockBucketAssigner mockBucketAssigner2 = new MockBucketAssigner(1, 2, context, writeConfig, smallFilesMap);
mockBucketAssigner2.addUpdate("par1", "f0");
BucketInfo bucketInfo2 = mockBucketAssigner2.addInsert("par1");
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f1");
mockBucketAssigner2.addInsert("par1");
bucketInfo2 = mockBucketAssigner2.addInsert("par1");
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f1");
mockBucketAssigner2.addUpdate("par1", "f2");
mockBucketAssigner2.addInsert("par1");
bucketInfo2 = mockBucketAssigner2.addInsert("par1");
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f1");
}
private void assertBucketEquals(
BucketInfo bucketInfo,
String partition,
@@ -220,7 +323,16 @@ public class TestBucketAssigner {
HoodieFlinkEngineContext context,
HoodieWriteConfig config,
Map<String, List<SmallFile>> smallFilesMap) {
super(context, config);
this(0, 1, context, config, smallFilesMap);
}
MockBucketAssigner(
int taskID,
int numTasks,
HoodieFlinkEngineContext context,
HoodieWriteConfig config,
Map<String, List<SmallFile>> smallFilesMap) {
super(taskID, numTasks, context, config);
this.smallFilesMap = smallFilesMap;
}

View File

@@ -28,7 +28,9 @@ import org.apache.hudi.sink.compact.CompactionPlanOperator;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext;
import org.apache.flink.runtime.operators.testutils.MockEnvironment;
import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder;
import org.apache.flink.streaming.api.operators.Output;
@@ -78,6 +80,9 @@ public class CompactFunctionWrapper {
compactFunction = new CompactFunction(conf);
compactFunction.setRuntimeContext(runtimeContext);
compactFunction.open(conf);
final NonThrownExecutor syncExecutor = new MockCoordinatorExecutor(
new MockOperatorCoordinatorContext(new OperatorID(), 1));
compactFunction.setExecutor(syncExecutor);
commitSink = new CompactionCommitSink(conf);
commitSink.setRuntimeContext(runtimeContext);

View File

@@ -0,0 +1,51 @@
/*
* 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.utils;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.function.ThrowingRunnable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A mock {@link CoordinatorExecutor} that executes the actions synchronously.
*/
public class MockCoordinatorExecutor extends CoordinatorExecutor {
private static final Logger LOG = LoggerFactory.getLogger(MockCoordinatorExecutor.class);
public MockCoordinatorExecutor(OperatorCoordinator.Context context) {
super(context, LOG);
}
@Override
public void execute(ThrowingRunnable<Throwable> action, String actionName, Object... actionParams) {
final String actionString = String.format(actionName, actionParams);
try {
action.run();
LOG.info("Executor executes action [{}] success!", actionString);
} catch (Throwable t) {
// if we have a JVM critical error, promote it immediately, there is a good
// chance the
// logging or job failing will not succeed any more
ExceptionUtils.rethrowIfFatalErrorOrOOM(t);
exceptionHook(actionString, t);
}
}
}

View File

@@ -33,7 +33,9 @@ import org.apache.hudi.utils.TestConfigurations;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext;
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
import org.apache.flink.runtime.operators.testutils.MockEnvironment;
import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder;
@@ -57,6 +59,7 @@ public class StreamWriteFunctionWrapper<I> {
private final IOManager ioManager;
private final StreamingRuntimeContext runtimeContext;
private final MockOperatorEventGateway gateway;
private final MockOperatorCoordinatorContext coordinatorContext;
private final StreamWriteOperatorCoordinator coordinator;
private final MockFunctionInitializationContext functionInitializationContext;
@@ -84,13 +87,15 @@ public class StreamWriteFunctionWrapper<I> {
this.gateway = new MockOperatorEventGateway();
this.conf = conf;
// one function
this.coordinator = new StreamWriteOperatorCoordinator(conf, 1);
this.coordinatorContext = new MockOperatorCoordinatorContext(new OperatorID(), 1);
this.coordinator = new StreamWriteOperatorCoordinator(conf, this.coordinatorContext);
this.functionInitializationContext = new MockFunctionInitializationContext();
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
}
public void openFunction() throws Exception {
this.coordinator.start();
this.coordinator.setExecutor(new MockCoordinatorExecutor(coordinatorContext));
toHoodieFunction = new RowDataToHoodieFunction<>(TestConfigurations.ROW_TYPE, conf);
toHoodieFunction.setRuntimeContext(runtimeContext);
toHoodieFunction.open(conf);
@@ -181,6 +186,10 @@ public class StreamWriteFunctionWrapper<I> {
return coordinator;
}
public MockOperatorCoordinatorContext getCoordinatorContext() {
return coordinatorContext;
}
public void clearIndexState() {
this.bucketAssignerFunction.clearIndexState();
}
@@ -188,8 +197,4 @@ public class StreamWriteFunctionWrapper<I> {
public boolean isKeyInState(HoodieKey hoodieKey) {
return this.bucketAssignerFunction.isKeyInState(hoodieKey);
}
public boolean isAllPartitionsLoaded() {
return this.bucketAssignerFunction.isAllPartitionsLoaded();
}
}