1
0

[HUDI-1729] Asynchronous Hive sync and commits cleaning for Flink writer (#2732)

This commit is contained in:
Danny Chan
2021-03-29 10:47:29 +08:00
committed by GitHub
parent ecbd389a3f
commit d415d45416
23 changed files with 704 additions and 41 deletions

View File

@@ -243,7 +243,7 @@ public class StreamWriteITCase extends TestLogger {
TypeInformation<String> typeInfo = BasicTypeInfo.STRING_TYPE_INFO;
format.setCharsetName("UTF-8");
execEnv
DataStream<Object> pipeline = execEnv
// use PROCESS_CONTINUOUSLY mode to trigger checkpoint
.readFile(format, sourcePath, FileProcessingMode.PROCESS_CONTINUOUSLY, 1000, typeInfo)
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)))
@@ -259,10 +259,15 @@ public class StreamWriteITCase extends TestLogger {
// shuffle by fileId(bucket id)
.keyBy(record -> record.getCurrentLocation().getFileId())
.transform("hoodie_stream_write", TypeInformation.of(Object.class), operatorFactory)
.uid("uid_hoodie_stream_write")
.transform("compact_plan_generate",
TypeInformation.of(CompactionPlanEvent.class),
new CompactionPlanOperator(conf))
.uid("uid_hoodie_stream_write");
pipeline.addSink(new CleanFunction<>(conf))
.setParallelism(1)
.name("clean_commits").uid("uid_clean_commits");
pipeline.transform("compact_plan_generate",
TypeInformation.of(CompactionPlanEvent.class),
new CompactionPlanOperator(conf))
.uid("uid_compact_plan_generate")
.setParallelism(1) // plan generate must be singleton
.keyBy(event -> event.getOperation().hashCode())

View File

@@ -22,11 +22,13 @@ import org.apache.hudi.client.WriteStatus;
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.util.StreamerUtil;
import org.apache.hudi.utils.TestConfigurations;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -42,6 +44,7 @@ import java.util.concurrent.CompletableFuture;
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.assertTrue;
@@ -63,7 +66,7 @@ public class TestStreamWriteOperatorCoordinator {
}
@AfterEach
public void after() {
public void after() throws Exception {
coordinator.close();
}
@@ -148,4 +151,31 @@ public class TestStreamWriteOperatorCoordinator {
() -> coordinator.notifyCheckpointComplete(1),
"Try 3 to commit instant");
}
@Test
public void testHiveSyncInvoked() throws Exception {
// override the default configuration
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setBoolean(FlinkOptions.HIVE_SYNC_ENABLED, true);
coordinator = new StreamWriteOperatorCoordinator(conf, 1);
coordinator.start();
String instant = coordinator.getInstant();
assertNotEquals("", instant);
WriteStatus writeStatus = new WriteStatus(true, 0.1D);
writeStatus.setPartitionPath("par1");
writeStatus.setStat(new HoodieWriteStat());
OperatorEvent event0 = BatchWriteSuccessEvent.builder()
.taskID(0)
.instantTime(instant)
.writeStatus(Collections.singletonList(writeStatus))
.isLastBatch(true)
.build();
coordinator.handleEventFromOperator(0, event0);
// never throw for hive synchronization now
assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(1));
}
}

View File

@@ -491,7 +491,7 @@ public class TestWriteCopyOnWrite {
funcWrapper.checkpointFunction(2);
String instant = funcWrapper.getWriteClient()
.getInflightAndRequestedInstant("COPY_ON_WRITE");
.getInflightAndRequestedInstant(getTableType());
nextEvent = funcWrapper.getNextEvent();
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
@@ -507,6 +507,8 @@ public class TestWriteCopyOnWrite {
// 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

@@ -18,12 +18,15 @@
package org.apache.hudi.table;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.utils.TestConfigurations;
import org.apache.hudi.utils.TestData;
import org.apache.hudi.utils.TestUtils;
import org.apache.hudi.utils.factory.CollectSinkTableFactory;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.TableEnvironment;
@@ -49,6 +52,7 @@ import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import static org.apache.hudi.utils.TestData.assertRowsEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* IT cases for Hoodie table source and sink.
@@ -105,7 +109,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
void testStreamReadAppendData() throws Exception {
// create filesystem table named source
String createSource = TestConfigurations.getFileSourceDDL("source");
String createSource2 = TestConfigurations.getFileSourceDDL("source2", "test_source2.data");
String createSource2 = TestConfigurations.getFileSourceDDL("source2", "test_source_2.data");
streamTableEnv.executeSql(createSource);
streamTableEnv.executeSql(createSource2);
@@ -175,6 +179,35 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT);
}
@Test
void testStreamWriteWithCleaning() throws InterruptedException {
// create filesystem table named source
// the source generates 4 commits but the cleaning task
// would always try to keep the remaining commits number as 1
String createSource = TestConfigurations.getFileSourceDDL(
"source", "test_source_3.data", 4);
streamTableEnv.executeSql(createSource);
Map<String, String> options = new HashMap<>();
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
options.put(FlinkOptions.CLEAN_RETAIN_COMMITS.key(), "1"); // only keep 1 commits
String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options);
streamTableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into t1 select * from source";
execInsertSql(streamTableEnv, insertInto);
Configuration defaultConf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
Map<String, String> options1 = new HashMap<>(defaultConf.toMap());
options1.put(FlinkOptions.TABLE_NAME.key(), "t1");
Configuration conf = Configuration.fromMap(options1);
HoodieTimeline timeline = StreamerUtil.createWriteClient(conf, null)
.getHoodieTable().getActiveTimeline();
assertTrue(timeline.filterCompletedInstants()
.getInstants().anyMatch(instant -> instant.getAction().equals("clean")),
"some commits should be cleaned");
}
@ParameterizedTest
@EnumSource(value = ExecMode.class)
void testWriteAndRead(ExecMode execMode) {

View File

@@ -91,7 +91,15 @@ public class TestConfigurations {
return getFileSourceDDL(tableName, "test_source.data");
}
public static String getFileSourceDDL(String tableName, int checkpoints) {
return getFileSourceDDL(tableName, "test_source.data", checkpoints);
}
public static String getFileSourceDDL(String tableName, String fileName) {
return getFileSourceDDL(tableName, fileName, 2);
}
public static String getFileSourceDDL(String tableName, String fileName, int checkpoints) {
String sourcePath = Objects.requireNonNull(Thread.currentThread()
.getContextClassLoader().getResource(fileName)).toString();
return "create table " + tableName + "(\n"
@@ -102,7 +110,8 @@ public class TestConfigurations {
+ " `partition` varchar(20)\n"
+ ") with (\n"
+ " 'connector' = '" + ContinuousFileSourceFactory.FACTORY_ID + "',\n"
+ " 'path' = '" + sourcePath + "'\n"
+ " 'path' = '" + sourcePath + "',\n"
+ " 'checkpoints' = '" + checkpoints + "'\n"
+ ")";
}

View File

@@ -141,7 +141,7 @@ public class TestData {
TimestampData.fromEpochMillis(8000), StringData.fromString("par4"))
);
// merged data set of test_source.data and test_source2.data
// merged data set of test_source.data and test_source_2.data
public static List<RowData> DATA_SET_SOURCE_MERGED = Arrays.asList(
insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 24,
TimestampData.fromEpochMillis(1000), StringData.fromString("par1")),

View File

@@ -22,6 +22,7 @@ import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.utils.source.ContinuousFileSource;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
import org.apache.flink.table.api.ValidationException;
@@ -38,6 +39,12 @@ import java.util.Set;
public class ContinuousFileSourceFactory implements DynamicTableSourceFactory {
public static final String FACTORY_ID = "continuous-file-source";
public static final ConfigOption<Integer> CHECKPOINTS = ConfigOptions
.key("checkpoints")
.intType()
.defaultValue(2)
.withDescription("Number of checkpoints to write the data set as, default 2");
@Override
public DynamicTableSource createDynamicTableSource(Context context) {
FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
@@ -56,11 +63,11 @@ public class ContinuousFileSourceFactory implements DynamicTableSourceFactory {
@Override
public Set<ConfigOption<?>> requiredOptions() {
return Collections.emptySet();
return Collections.singleton(FlinkOptions.PATH);
}
@Override
public Set<ConfigOption<?>> optionalOptions() {
return Collections.singleton(FlinkOptions.PATH);
return Collections.singleton(CHECKPOINTS);
}
}

View File

@@ -42,6 +42,8 @@ import java.nio.file.Paths;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.hudi.utils.factory.ContinuousFileSourceFactory.CHECKPOINTS;
/**
* A continuous file source that can trigger checkpoints continuously.
*
@@ -89,7 +91,7 @@ public class ContinuousFileSource implements ScanTableSource {
true,
TimestampFormat.ISO_8601);
return execEnv.addSource(new BoundedSourceFunction(path, 2))
return execEnv.addSource(new BoundedSourceFunction(path, conf.getInteger(CHECKPOINTS)))
.name("continuous_file_source")
.setParallelism(1)
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)),

View File

@@ -0,0 +1,8 @@
{"uuid": "id1", "name": "Danny", "age": 24, "ts": "1970-01-01T00:00:01", "partition": "par1"}
{"uuid": "id2", "name": "Stephen", "age": 34, "ts": "1970-01-01T00:00:02", "partition": "par1"}
{"uuid": "id1", "name": "Danny", "age": 24, "ts": "1970-01-01T00:00:01", "partition": "par1"}
{"uuid": "id2", "name": "Stephen", "age": 34, "ts": "1970-01-01T00:00:02", "partition": "par1"}
{"uuid": "id1", "name": "Danny", "age": 24, "ts": "1970-01-01T00:00:01", "partition": "par1"}
{"uuid": "id2", "name": "Stephen", "age": 34, "ts": "1970-01-01T00:00:02", "partition": "par1"}
{"uuid": "id1", "name": "Danny", "age": 24, "ts": "1970-01-01T00:00:01", "partition": "par1"}
{"uuid": "id2", "name": "Stephen", "age": 34, "ts": "1970-01-01T00:00:02", "partition": "par1"}