1
0

[HUDI-2258] Metadata table for flink (#3381)

This commit is contained in:
Danny Chan
2021-08-04 10:54:55 +08:00
committed by GitHub
parent b4c14eaa29
commit 02331fc223
19 changed files with 645 additions and 119 deletions

View File

@@ -23,7 +23,9 @@ import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.metadata.HoodieTableMetadata;
import org.apache.hudi.sink.event.WriteMetadataEvent;
import org.apache.hudi.sink.utils.MockCoordinatorExecutor;
import org.apache.hudi.util.StreamerUtil;
@@ -34,6 +36,7 @@ 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.flink.util.FileUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.junit.jupiter.api.AfterEach;
@@ -71,22 +74,8 @@ public class TestStreamWriteOperatorCoordinator {
coordinator.start();
coordinator.setExecutor(new MockCoordinatorExecutor(context));
final WriteMetadataEvent event0 = WriteMetadataEvent.builder()
.taskID(0)
.instantTime("")
.writeStatus(Collections.emptyList())
.bootstrap(true)
.build();
final WriteMetadataEvent event1 = WriteMetadataEvent.builder()
.taskID(1)
.instantTime("")
.writeStatus(Collections.emptyList())
.bootstrap(true)
.build();
coordinator.handleEventFromOperator(0, event0);
coordinator.handleEventFromOperator(1, event1);
coordinator.handleEventFromOperator(0, WriteMetadataEvent.emptyBootstrap(0));
coordinator.handleEventFromOperator(1, WriteMetadataEvent.emptyBootstrap(1));
}
@AfterEach
@@ -99,25 +88,8 @@ public class TestStreamWriteOperatorCoordinator {
String instant = coordinator.getInstant();
assertNotEquals("", instant);
WriteStatus writeStatus = new WriteStatus(true, 0.1D);
writeStatus.setPartitionPath("par1");
writeStatus.setStat(new HoodieWriteStat());
OperatorEvent event0 = WriteMetadataEvent.builder()
.taskID(0)
.instantTime(instant)
.writeStatus(Collections.singletonList(writeStatus))
.lastBatch(true)
.build();
WriteStatus writeStatus1 = new WriteStatus(false, 0.2D);
writeStatus1.setPartitionPath("par2");
writeStatus1.setStat(new HoodieWriteStat());
OperatorEvent event1 = WriteMetadataEvent.builder()
.taskID(1)
.instantTime(instant)
.writeStatus(Collections.singletonList(writeStatus1))
.lastBatch(true)
.build();
OperatorEvent event0 = createOperatorEvent(0, instant, "par1", true, 0.1);
OperatorEvent event1 = createOperatorEvent(1, instant, "par2", false, 0.2);
coordinator.handleEventFromOperator(0, event0);
coordinator.handleEventFromOperator(1, event1);
@@ -177,15 +149,7 @@ public class TestStreamWriteOperatorCoordinator {
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 = WriteMetadataEvent.builder()
.taskID(1)
.instantTime(instant)
.writeStatus(Collections.singletonList(writeStatus1))
.lastBatch(true)
.build();
OperatorEvent event1 = createOperatorEvent(1, instant, "par2", false, 0.2);
coordinator.handleEventFromOperator(1, event1);
assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(2),
"Commits the instant with partial events anyway");
@@ -195,6 +159,8 @@ public class TestStreamWriteOperatorCoordinator {
@Test
public void testHiveSyncInvoked() throws Exception {
// reset
reset();
// override the default configuration
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setBoolean(FlinkOptions.HIVE_SYNC_ENABLED, true);
@@ -203,39 +169,97 @@ public class TestStreamWriteOperatorCoordinator {
coordinator.start();
coordinator.setExecutor(new MockCoordinatorExecutor(context));
final WriteMetadataEvent event0 = WriteMetadataEvent.builder()
.taskID(0)
.instantTime("")
.writeStatus(Collections.emptyList())
.bootstrap(true)
.build();
final WriteMetadataEvent event0 = WriteMetadataEvent.emptyBootstrap(0);
coordinator.handleEventFromOperator(0, event0);
String instant = mockWriteWithMetadata();
assertNotEquals("", instant);
// never throw for hive synchronization now
assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(1));
}
@Test
void testSyncMetadataTable() throws Exception {
// reset
reset();
// override the default configuration
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setBoolean(FlinkOptions.METADATA_ENABLED, true);
conf.setInteger(FlinkOptions.METADATA_COMPACTION_DELTA_COMMITS, 5);
OperatorCoordinator.Context context = new MockOperatorCoordinatorContext(new OperatorID(), 1);
coordinator = new StreamWriteOperatorCoordinator(conf, context);
coordinator.start();
coordinator.setExecutor(new MockCoordinatorExecutor(context));
coordinator.setMetadataSyncExecutor(new MockCoordinatorExecutor(context));
final WriteMetadataEvent event0 = WriteMetadataEvent.emptyBootstrap(0);
coordinator.handleEventFromOperator(0, event0);
String instant = coordinator.getInstant();
assertNotEquals("", instant);
WriteStatus writeStatus = new WriteStatus(true, 0.1D);
writeStatus.setPartitionPath("par1");
writeStatus.setStat(new HoodieWriteStat());
final String metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(tempFile.getAbsolutePath());
HoodieTableMetaClient metadataTableMetaClient = StreamerUtil.createMetaClient(metadataTableBasePath);
HoodieTimeline completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants();
assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(1L));
assertThat(completedTimeline.lastInstant().get().getTimestamp(), is("0000000000000"));
OperatorEvent event1 = WriteMetadataEvent.builder()
.taskID(0)
.instantTime(instant)
.writeStatus(Collections.singletonList(writeStatus))
.lastBatch(true)
.build();
coordinator.handleEventFromOperator(0, event1);
// never throw for hive synchronization now
assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(1));
// test metadata table compaction
// write another 4 commits
for (int i = 1; i < 4; i++) {
instant = mockWriteWithMetadata();
metadataTableMetaClient.reloadActiveTimeline();
completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants();
assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(i + 1L));
assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant));
}
// the 5th commit triggers the compaction
instant = mockWriteWithMetadata();
metadataTableMetaClient.reloadActiveTimeline();
completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(6L));
assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant + "001"));
assertThat(completedTimeline.lastInstant().get().getAction(), is(HoodieTimeline.COMMIT_ACTION));
}
// -------------------------------------------------------------------------
// Utilities
// -------------------------------------------------------------------------
private String mockWriteWithMetadata() {
final String instant = coordinator.getInstant();
OperatorEvent event = createOperatorEvent(0, instant, "par1", true, 0.1);
coordinator.handleEventFromOperator(0, event);
coordinator.notifyCheckpointComplete(0);
return instant;
}
private static WriteMetadataEvent createOperatorEvent(
int taskId,
String instant,
String partitionPath,
boolean trackSuccessRecords,
double failureFraction) {
final WriteStatus writeStatus = new WriteStatus(trackSuccessRecords, failureFraction);
writeStatus.setPartitionPath(partitionPath);
writeStatus.setStat(new HoodieWriteStat());
return WriteMetadataEvent.builder()
.taskID(taskId)
.instantTime(instant)
.writeStatus(Collections.singletonList(writeStatus))
.lastBatch(true)
.build();
}
private void reset() throws Exception {
FileUtils.cleanDirectory(tempFile);
}
private void assertError(Runnable runnable, String message) {
runnable.run();
// wait a little while for the task to finish