1
0

[HUDI-2376] Add pipeline for Append mode (#3573)

Co-authored-by: 喻兆靖 <yuzhaojing@bilibili.com>
This commit is contained in:
yuzhaojing
2021-09-02 16:32:40 +08:00
committed by GitHub
parent 21fd6edfe7
commit 7a1bd225ca
42 changed files with 946 additions and 443 deletions

View File

@@ -245,8 +245,6 @@ public class StreamWriteITCase extends TestLogger {
RowType rowType =
(RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf))
.getLogicalType();
StreamWriteOperatorFactory<HoodieRecord> operatorFactory =
new StreamWriteOperatorFactory<>(conf);
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
rowType,
@@ -302,8 +300,6 @@ public class StreamWriteITCase extends TestLogger {
RowType rowType =
(RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf))
.getLogicalType();
StreamWriteOperatorFactory<HoodieRecord> operatorFactory =
new StreamWriteOperatorFactory<>(conf);
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
rowType,

View File

@@ -23,13 +23,13 @@ import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.sink.event.WriteMetadataEvent;
import org.apache.hudi.sink.utils.InsertFunctionWrapper;
import org.apache.hudi.sink.utils.StreamWriteFunctionWrapper;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.utils.TestConfigurations;
@@ -58,6 +58,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
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.assertNull;
import static org.junit.jupiter.api.Assertions.assertSame;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -532,11 +533,7 @@ public class TestWriteCopyOnWrite {
@Test
public void testInsertAllowsDuplication() throws Exception {
// reset the config option
conf.setDouble(FlinkOptions.WRITE_BATCH_SIZE, 0.0006); // 630 bytes batch size
conf.setString(FlinkOptions.OPERATION, WriteOperationType.INSERT.value());
conf.setBoolean(FlinkOptions.INSERT_DEDUP, false);
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
InsertFunctionWrapper<RowData> funcWrapper = new InsertFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
// open the function and ingest data
funcWrapper.openFunction();
@@ -547,19 +544,16 @@ public class TestWriteCopyOnWrite {
// this triggers the data write and event send
funcWrapper.checkpointFunction(1);
Map<String, List<HoodieRecord>> dataBuffer = funcWrapper.getDataBuffer();
assertThat("All data should be flushed out", dataBuffer.size(), is(0));
assertNull(funcWrapper.getWriterHelper());
final OperatorEvent event1 = funcWrapper.getNextEvent(); // remove the first event first
final OperatorEvent event2 = funcWrapper.getNextEvent();
assertThat("The operator expect to send an event", event2, instanceOf(WriteMetadataEvent.class));
assertThat("The operator expect to send an event", event1, instanceOf(WriteMetadataEvent.class));
funcWrapper.getCoordinator().handleEventFromOperator(0, event1);
funcWrapper.getCoordinator().handleEventFromOperator(0, event2);
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
String instant = funcWrapper.getWriteClient()
.getLastPendingInstant(getTableType());
.getLastPendingInstant(getTableType());
funcWrapper.checkpointComplete(1);
@@ -585,10 +579,8 @@ public class TestWriteCopyOnWrite {
funcWrapper.checkpointFunction(2);
final OperatorEvent event3 = funcWrapper.getNextEvent(); // remove the first event first
final OperatorEvent event4 = funcWrapper.getNextEvent();
funcWrapper.getCoordinator().handleEventFromOperator(0, event3);
funcWrapper.getCoordinator().handleEventFromOperator(0, event4);
final OperatorEvent event2 = funcWrapper.getNextEvent(); // remove the first event first
funcWrapper.getCoordinator().handleEventFromOperator(0, event2);
funcWrapper.checkpointComplete(2);
// same with the original base file content.

View File

@@ -77,7 +77,7 @@ public class TestRowDataKeyGen {
assertThat(keyGen1.getPartitionPath(rowData1), is("par1/1970-01-01T00:00:00.001"));
// null record key and partition path
final RowData rowData2 = insertRow(TestConfigurations.ROW_TYPE,null, null, 23, null, null);
final RowData rowData2 = insertRow(TestConfigurations.ROW_TYPE, null, null, 23, null, null);
assertThrows(HoodieKeyException.class, () -> keyGen1.getRecordKey(rowData2));
assertThat(keyGen1.getPartitionPath(rowData2), is("default/default"));
// empty record key and partition path

View File

@@ -53,11 +53,17 @@ public class CompactFunctionWrapper {
private final IOManager ioManager;
private final StreamingRuntimeContext runtimeContext;
/** Function that generates the {@link HoodieCompactionPlan}. */
/**
* Function that generates the {@link HoodieCompactionPlan}.
*/
private CompactionPlanOperator compactionPlanOperator;
/** Function that executes the compaction task. */
/**
* Function that executes the compaction task.
*/
private CompactFunction compactFunction;
/** Stream sink to handle compaction commits. */
/**
* Stream sink to handle compaction commits.
*/
private CompactionCommitSink commitSink;
public CompactFunctionWrapper(Configuration conf) throws Exception {
@@ -120,7 +126,7 @@ public class CompactFunctionWrapper {
compactionPlanOperator.notifyCheckpointComplete(checkpointID);
// collect the CompactCommitEvents
List<CompactionCommitEvent> compactCommitEvents = new ArrayList<>();
for (CompactionPlanEvent event: events) {
for (CompactionPlanEvent event : events) {
compactFunction.processElement(event, null, new Collector<CompactionCommitEvent>() {
@Override
public void collect(CompactionCommitEvent event) {

View File

@@ -0,0 +1,141 @@
/*
* 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.hudi.client.HoodieFlinkWriteClient;
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
import org.apache.hudi.sink.append.AppendWriteFunction;
import org.apache.hudi.sink.bulk.BulkInsertWriterHelper;
import org.apache.hudi.sink.event.WriteMetadataEvent;
import org.apache.hudi.util.AvroSchemaConverter;
import org.apache.hudi.util.StreamerUtil;
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;
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import org.apache.flink.streaming.api.operators.collect.utils.MockOperatorEventGateway;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.logical.RowType;
import java.util.concurrent.CompletableFuture;
/**
* A wrapper class to manipulate the {@link AppendWriteFunction} instance for testing.
*
* @param <I> Input type
*/
public class InsertFunctionWrapper<I> {
private final Configuration conf;
private final RowType rowType;
private final StreamingRuntimeContext runtimeContext;
private final MockOperatorEventGateway gateway;
private final MockOperatorCoordinatorContext coordinatorContext;
private final StreamWriteOperatorCoordinator coordinator;
private final MockStateInitializationContext stateInitializationContext;
/**
* Append write function.
*/
private AppendWriteFunction<RowData> writeFunction;
public InsertFunctionWrapper(String tablePath, Configuration conf) {
IOManager ioManager = new IOManagerAsync();
MockEnvironment environment = new MockEnvironmentBuilder()
.setTaskName("mockTask")
.setManagedMemorySize(4 * MemoryManager.DEFAULT_PAGE_SIZE)
.setIOManager(ioManager)
.build();
this.runtimeContext = new MockStreamingRuntimeContext(false, 1, 0, environment);
this.gateway = new MockOperatorEventGateway();
this.conf = conf;
this.rowType = (RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf)).getLogicalType();
// one function
this.coordinatorContext = new MockOperatorCoordinatorContext(new OperatorID(), 1);
this.coordinator = new StreamWriteOperatorCoordinator(conf, this.coordinatorContext);
this.stateInitializationContext = new MockStateInitializationContext();
}
public void openFunction() throws Exception {
this.coordinator.start();
this.coordinator.setExecutor(new MockCoordinatorExecutor(coordinatorContext));
setupWriteFunction();
}
public void invoke(I record) throws Exception {
writeFunction.processElement((RowData) record, null, null);
}
public WriteMetadataEvent[] getEventBuffer() {
return this.coordinator.getEventBuffer();
}
public OperatorEvent getNextEvent() {
return this.gateway.getNextEvent();
}
@SuppressWarnings("rawtypes")
public HoodieFlinkWriteClient getWriteClient() {
return this.writeFunction.getWriteClient();
}
public void checkpointFunction(long checkpointId) throws Exception {
// checkpoint the coordinator first
this.coordinator.checkpointCoordinator(checkpointId, new CompletableFuture<>());
writeFunction.snapshotState(null);
stateInitializationContext.getOperatorStateStore().checkpointBegin(checkpointId);
}
public void checkpointComplete(long checkpointId) {
stateInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId);
coordinator.notifyCheckpointComplete(checkpointId);
}
public StreamWriteOperatorCoordinator getCoordinator() {
return coordinator;
}
public BulkInsertWriterHelper getWriterHelper() {
return this.writeFunction.getWriterHelper();
}
// -------------------------------------------------------------------------
// Utilities
// -------------------------------------------------------------------------
private void setupWriteFunction() throws Exception {
writeFunction = new AppendWriteFunction<>(conf, rowType);
writeFunction.setRuntimeContext(runtimeContext);
writeFunction.setOperatorEventGateway(gateway);
writeFunction.initializeState(this.stateInitializationContext);
writeFunction.open(conf);
// handle the bootstrap event
coordinator.handleEventFromOperator(0, getNextEvent());
}
}

View File

@@ -95,7 +95,7 @@ public class StreamWriteFunctionWrapper<I> {
/**
* Stream write function.
*/
private StreamWriteFunction<Object, HoodieRecord<?>, Object> writeFunction;
private StreamWriteFunction<HoodieRecord<?>> writeFunction;
private CompactFunctionWrapper compactFunctionWrapper;

View File

@@ -68,6 +68,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
*/
public class TestStreamReadOperator {
private static final Map<String, String> EXPECTED = new HashMap<>();
static {
EXPECTED.put("par1", "+I[id1, Danny, 23, 1970-01-01T00:00:00.001, par1], +I[id2, Stephen, 33, 1970-01-01T00:00:00.002, par1]");
EXPECTED.put("par2", "+I[id3, Julian, 53, 1970-01-01T00:00:00.003, par2], +I[id4, Fabian, 31, 1970-01-01T00:00:00.004, par2]");

View File

@@ -796,6 +796,35 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
+ "+I[id1, Sophia, 18, 1970-01-01T00:00:05, par5]]", 3);
}
@Test
void testAppendWrite() {
TableEnvironment tableEnv = batchTableEnv;
// csv source
String csvSourceDDL = TestConfigurations.getCsvSourceDDL("csv_source", "test_source_5.data");
tableEnv.executeSql(csvSourceDDL);
String hoodieTableDDL = sql("hoodie_sink")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.OPERATION, "insert")
.option(FlinkOptions.INSERT_DEDUP, false)
.end();
tableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into hoodie_sink select * from csv_source";
execInsertSql(tableEnv, insertInto);
List<Row> result1 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from hoodie_sink").execute().collect());
assertRowsEquals(result1, TestData.DATA_SET_SOURCE_INSERT);
// apply filters
List<Row> result2 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from hoodie_sink where uuid > 'id5'").execute().collect());
assertRowsEquals(result2, "["
+ "+I[id6, Emma, 20, 1970-01-01T00:00:06, par3], "
+ "+I[id7, Bob, 44, 1970-01-01T00:00:07, par4], "
+ "+I[id8, Han, 56, 1970-01-01T00:00:08, par4]]");
}
// -------------------------------------------------------------------------
// Utilities
// -------------------------------------------------------------------------
@@ -874,7 +903,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
}
private List<Row> execSelectSql(TableEnvironment tEnv, String select, String sinkDDL, long timeout)
throws InterruptedException {
throws InterruptedException {
tEnv.executeSql("DROP TABLE IF EXISTS sink");
tEnv.executeSql(sinkDDL);
TableResult tableResult = tEnv.executeSql("insert into sink " + select);
@@ -883,7 +912,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
tableResult.getJobClient().ifPresent(JobClient::cancel);
tEnv.executeSql("DROP TABLE IF EXISTS sink");
return CollectSinkTableFactory.RESULT.values().stream()
.flatMap(Collection::stream)
.collect(Collectors.toList());
.flatMap(Collection::stream)
.collect(Collectors.toList());
}
}

View File

@@ -128,18 +128,18 @@ public class TestConfigurations {
DataType[] fieldTypes = tableSchema.getFieldDataTypes();
for (int i = 0; i < fieldNames.length; i++) {
builder.append(" `")
.append(fieldNames[i])
.append("` ")
.append(fieldTypes[i].toString());
.append(fieldNames[i])
.append("` ")
.append(fieldTypes[i].toString());
if (i != fieldNames.length - 1) {
builder.append(",");
}
builder.append("\n");
}
final String withProps = ""
+ ") with (\n"
+ " 'connector' = '" + CollectSinkTableFactory.FACTORY_ID + "'\n"
+ ")";
+ ") with (\n"
+ " 'connector' = '" + CollectSinkTableFactory.FACTORY_ID + "'\n"
+ ")";
builder.append(withProps);
return builder.toString();
}

View File

@@ -22,7 +22,8 @@ package org.apache.hudi.utils;
* Test sql statements.
*/
public class TestSQL {
private TestSQL() {}
private TestSQL() {
}
public static final String INSERT_T1 = "insert into t1 values\n"
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:01','par1'),\n"

View File

@@ -57,12 +57,12 @@ public class TestStreamerUtil {
// Validate the partition fields & preCombineField in hoodie.properties.
HoodieTableMetaClient metaClient1 = HoodieTableMetaClient.builder()
.setBasePath(tempFile.getAbsolutePath())
.setConf(new org.apache.hadoop.conf.Configuration())
.build();
.setBasePath(tempFile.getAbsolutePath())
.setConf(new org.apache.hadoop.conf.Configuration())
.build();
assertTrue(metaClient1.getTableConfig().getPartitionFields().isPresent(),
"Missing partition columns in the hoodie.properties.");
assertArrayEquals(metaClient1.getTableConfig().getPartitionFields().get(), new String[] { "p0", "p1" });
"Missing partition columns in the hoodie.properties.");
assertArrayEquals(metaClient1.getTableConfig().getPartitionFields().get(), new String[] {"p0", "p1"});
assertEquals(metaClient1.getTableConfig().getPreCombineField(), "ts");
// Test for non-partitioned table.
@@ -70,9 +70,9 @@ public class TestStreamerUtil {
FileIOUtils.deleteDirectory(tempFile);
StreamerUtil.initTableIfNotExists(conf);
HoodieTableMetaClient metaClient2 = HoodieTableMetaClient.builder()
.setBasePath(tempFile.getAbsolutePath())
.setConf(new org.apache.hadoop.conf.Configuration())
.build();
.setBasePath(tempFile.getAbsolutePath())
.setConf(new org.apache.hadoop.conf.Configuration())
.build();
assertFalse(metaClient2.getTableConfig().getPartitionFields().isPresent());
}