1
0

[HUDI-1632] Supports merge on read write mode for Flink writer (#2593)

Also supports async compaction with pluggable strategies.
This commit is contained in:
Danny Chan
2021-03-01 12:29:41 +08:00
committed by GitHub
parent be257b58c6
commit 7a11de1276
38 changed files with 2296 additions and 152 deletions

View File

@@ -20,6 +20,12 @@ package org.apache.hudi.operator;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.operator.compact.CompactFunction;
import org.apache.hudi.operator.compact.CompactionCommitEvent;
import org.apache.hudi.operator.compact.CompactionCommitSink;
import org.apache.hudi.operator.compact.CompactionPlanEvent;
import org.apache.hudi.operator.compact.CompactionPlanOperator;
import org.apache.hudi.operator.partitioner.BucketAssignFunction;
import org.apache.hudi.operator.transform.RowDataToHoodieFunction;
import org.apache.hudi.operator.utils.TestConfigurations;
@@ -219,4 +225,81 @@ public class StreamWriteITCase extends TestLogger {
TestData.checkWrittenFullData(tempFile, EXPECTED);
}
@Test
public void testMergeOnReadWriteWithCompaction() throws Exception {
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, 1);
conf.setString(FlinkOptions.TABLE_TYPE, HoodieTableType.MERGE_ON_READ.name());
StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
execEnv.getConfig().disableObjectReuse();
execEnv.setParallelism(4);
// set up checkpoint interval
execEnv.enableCheckpointing(4000, CheckpointingMode.EXACTLY_ONCE);
execEnv.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
// Read from file source
RowType rowType =
(RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf))
.getLogicalType();
StreamWriteOperatorFactory<HoodieRecord> operatorFactory =
new StreamWriteOperatorFactory<>(conf);
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
rowType,
new RowDataTypeInfo(rowType),
false,
true,
TimestampFormat.ISO_8601
);
String sourcePath = Objects.requireNonNull(Thread.currentThread()
.getContextClassLoader().getResource("test_source.data")).toString();
TextInputFormat format = new TextInputFormat(new Path(sourcePath));
format.setFilesFilter(FilePathFilter.createDefaultFilter());
TypeInformation<String> typeInfo = BasicTypeInfo.STRING_TYPE_INFO;
format.setCharsetName("UTF-8");
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)))
.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)
.transform(
"bucket_assigner",
TypeInformation.of(HoodieRecord.class),
new KeyedProcessOperator<>(new BucketAssignFunction<>(conf)))
.uid("uid_bucket_assigner")
// 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_compact_plan_generate")
.setParallelism(1) // plan generate must be singleton
.keyBy(event -> event.getOperation().hashCode())
.transform("compact_task",
TypeInformation.of(CompactionCommitEvent.class),
new KeyedProcessOperator<>(new CompactFunction(conf)))
.addSink(new CompactionCommitSink(conf))
.name("compact_commit")
.setParallelism(1);
JobClient client = execEnv.executeAsync(execEnv.getStreamGraph(conf.getString(FlinkOptions.TABLE_NAME)));
if (client.getJobStatus().get() != JobStatus.FAILED) {
try {
TimeUnit.SECONDS.sleep(20); // wait long enough for the compaction to finish
client.cancel();
} catch (Throwable var1) {
// ignored
}
}
TestData.checkWrittenFullData(tempFile, EXPECTED);
}
}

View File

@@ -22,6 +22,7 @@ import org.apache.hudi.client.HoodieFlinkWriteClient;
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.table.timeline.HoodieInstant;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
@@ -45,7 +46,6 @@ import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import static org.apache.hudi.operator.utils.TestData.checkWrittenData;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
@@ -58,13 +58,13 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Test cases for StreamingSinkFunction.
*/
public class StreamWriteFunctionTest {
public class TestWriteCopyOnWrite {
private static final Map<String, String> EXPECTED1 = new HashMap<>();
protected static final Map<String, String> EXPECTED1 = new HashMap<>();
private static final Map<String, String> EXPECTED2 = new HashMap<>();
protected static final Map<String, String> EXPECTED2 = new HashMap<>();
private static final Map<String, String> EXPECTED3 = new HashMap<>();
protected static final Map<String, String> EXPECTED3 = new HashMap<>();
static {
EXPECTED1.put("par1", "[id1,par1,id1,Danny,23,1,par1, id2,par1,id2,Stephen,33,2,par1]");
@@ -82,14 +82,27 @@ public class StreamWriteFunctionTest {
EXPECTED3.put("par1", "[id1,par1,id1,Danny,23,1,par1]");
}
private StreamWriteFunctionWrapper<RowData> funcWrapper;
protected Configuration conf;
protected StreamWriteFunctionWrapper<RowData> funcWrapper;
@TempDir
File tempFile;
@BeforeEach
public void before() throws Exception {
this.funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath());
final String basePath = tempFile.getAbsolutePath();
conf = TestConfigurations.getDefaultConf(basePath);
conf.setString(FlinkOptions.TABLE_TYPE, getTableType());
setUp(conf);
this.funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
}
/**
* Override to have custom configuration.
*/
protected void setUp(Configuration conf) {
// for sub-class extension
}
@AfterEach
@@ -114,7 +127,7 @@ public class StreamWriteFunctionTest {
funcWrapper.checkpointFunction(1);
String instant = funcWrapper.getWriteClient()
.getInflightAndRequestedInstant("COPY_ON_WRITE");
.getInflightAndRequestedInstant(getTableType());
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
MatcherAssert.assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
@@ -140,7 +153,7 @@ public class StreamWriteFunctionTest {
funcWrapper.checkpointFunction(2);
String instant2 = funcWrapper.getWriteClient()
.getInflightAndRequestedInstant("COPY_ON_WRITE");
.getInflightAndRequestedInstant(getTableType());
assertNotEquals(instant, instant2);
final OperatorEvent nextEvent2 = funcWrapper.getNextEvent();
@@ -169,7 +182,7 @@ public class StreamWriteFunctionTest {
funcWrapper.checkpointFunction(1);
String instant = funcWrapper.getWriteClient()
.getInflightAndRequestedInstant("COPY_ON_WRITE");
.getInflightAndRequestedInstant(getTableType());
assertNotNull(instant);
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
@@ -191,10 +204,8 @@ public class StreamWriteFunctionTest {
funcWrapper.invoke(rowData);
}
// this triggers NPE cause there is no inflight instant
assertThrows(NullPointerException.class,
() -> funcWrapper.checkpointFunction(2),
"No inflight instant when flushing data");
// this returns early cause 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.
funcWrapper.checkpointFails(2);
@@ -213,7 +224,7 @@ public class StreamWriteFunctionTest {
funcWrapper.checkpointFunction(1);
String instant = funcWrapper.getWriteClient()
.getInflightAndRequestedInstant("COPY_ON_WRITE");
.getInflightAndRequestedInstant(getTableType());
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
@@ -232,7 +243,6 @@ public class StreamWriteFunctionTest {
@Test
public void testInsertDuplicates() throws Exception {
// reset the config option
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setBoolean(FlinkOptions.INSERT_DROP_DUPS, true);
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
@@ -300,11 +310,10 @@ public class StreamWriteFunctionTest {
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));
checkWrittenData(tempFile, EXPECTED2);
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
@@ -319,7 +328,6 @@ public class StreamWriteFunctionTest {
@Test
public void testInsertWithMiniBatches() throws Exception {
// reset the config option
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setDouble(FlinkOptions.WRITE_BATCH_SIZE, 0.001); // 1Kb batch size
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
@@ -349,16 +357,11 @@ public class StreamWriteFunctionTest {
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
String instant = funcWrapper.getWriteClient()
.getInflightAndRequestedInstant("COPY_ON_WRITE");
.getInflightAndRequestedInstant(getTableType());
funcWrapper.checkpointComplete(1);
Map<String, String> expected = new HashMap<>();
expected.put("par1", "[id1,par1,id1,Danny,23,1,par1, "
+ "id1,par1,id1,Danny,23,1,par1, "
+ "id1,par1,id1,Danny,23,1,par1, "
+ "id1,par1,id1,Danny,23,1,par1, "
+ "id1,par1,id1,Danny,23,1,par1]");
Map<String, String> expected = getMiniBatchExpected();
checkWrittenData(tempFile, expected, 1);
// started a new instant already
@@ -384,6 +387,16 @@ public class StreamWriteFunctionTest {
checkWrittenData(tempFile, expected, 1);
}
Map<String, String> getMiniBatchExpected() {
Map<String, String> expected = new HashMap<>();
expected.put("par1", "[id1,par1,id1,Danny,23,1,par1, "
+ "id1,par1,id1,Danny,23,1,par1, "
+ "id1,par1,id1,Danny,23,1,par1, "
+ "id1,par1,id1,Danny,23,1,par1, "
+ "id1,par1,id1,Danny,23,1,par1]");
return expected;
}
@Test
public void testIndexStateBootstrap() throws Exception {
// open the function and ingest data
@@ -452,7 +465,7 @@ public class StreamWriteFunctionTest {
@SuppressWarnings("rawtypes")
private void checkInflightInstant(HoodieFlinkWriteClient writeClient) {
final String instant = writeClient.getInflightAndRequestedInstant("COPY_ON_WRITE");
final String instant = writeClient.getInflightAndRequestedInstant(getTableType());
assertNotNull(instant);
}
@@ -464,10 +477,10 @@ public class StreamWriteFunctionTest {
final String instant;
switch (state) {
case REQUESTED:
instant = writeClient.getInflightAndRequestedInstant("COPY_ON_WRITE");
instant = writeClient.getInflightAndRequestedInstant(getTableType());
break;
case COMPLETED:
instant = writeClient.getLastCompletedInstant("COPY_ON_WRITE");
instant = writeClient.getLastCompletedInstant(getTableType());
break;
default:
throw new AssertionError("Unexpected state");
@@ -475,10 +488,22 @@ public class StreamWriteFunctionTest {
assertThat(instant, is(instantStr));
}
protected String getTableType() {
return HoodieTableType.COPY_ON_WRITE.name();
}
protected void checkWrittenData(File baseFile, Map<String, String> expected) throws Exception {
checkWrittenData(baseFile, expected, 4);
}
protected void checkWrittenData(File baseFile, Map<String, String> expected, int partitions) throws Exception {
TestData.checkWrittenData(baseFile, expected, partitions);
}
/**
* Asserts the data files are empty.
*/
private void assertEmptyDataFiles() {
protected void assertEmptyDataFiles() {
File[] dataFiles = tempFile.listFiles(file -> !file.getName().startsWith("."));
assertNotNull(dataFiles);
assertThat(dataFiles.length, is(0));

View File

@@ -0,0 +1,96 @@
/*
* 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.operator;
import org.apache.hudi.client.FlinkTaskContextSupplier;
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.operator.utils.TestData;
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.util.StreamerUtil;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import java.io.File;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Map;
import java.util.stream.Collectors;
/**
* Test cases for delta stream write.
*/
public class TestWriteMergeOnRead extends TestWriteCopyOnWrite {
private FileSystem fs;
private HoodieWriteConfig writeConfig;
private HoodieFlinkEngineContext context;
@BeforeEach
public void before() throws Exception {
super.before();
fs = FSUtils.getFs(tempFile.getAbsolutePath(), new org.apache.hadoop.conf.Configuration());
writeConfig = StreamerUtil.getHoodieClientConfig(conf);
context = new HoodieFlinkEngineContext(
new SerializableConfiguration(StreamerUtil.getHadoopConf()),
new FlinkTaskContextSupplier(null));
}
@Override
protected void checkWrittenData(File baseFile, Map<String, String> expected, int partitions) throws Exception {
HoodieTableMetaClient metaClient = HoodieFlinkTable.create(writeConfig, context).getMetaClient();
Schema schema = new TableSchemaResolver(metaClient).getTableAvroSchema();
String latestInstant = metaClient.getCommitsTimeline().filterCompletedInstants()
.getInstants()
.filter(x -> x.getAction().equals(HoodieActiveTimeline.DELTA_COMMIT_ACTION))
.map(HoodieInstant::getTimestamp)
.collect(Collectors.toList()).stream()
.max(Comparator.naturalOrder())
.orElse(null);
TestData.checkWrittenDataMOR(fs, latestInstant, baseFile, expected, partitions, schema);
}
@Disabled
@Test
public void testIndexStateBootstrap() {
// Ignore the index bootstrap because we only support parquet load now.
}
Map<String, String> getMiniBatchExpected() {
Map<String, String> expected = new HashMap<>();
// MOR mode merges the messages with the same key.
expected.put("par1", "[id1,par1,id1,Danny,23,1,par1]");
return expected;
}
@Override
protected String getTableType() {
return HoodieTableType.MERGE_ON_READ.name();
}
}

View File

@@ -0,0 +1,58 @@
/*
* 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.operator;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.flink.configuration.Configuration;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import java.util.HashMap;
import java.util.Map;
/**
* Test cases for delta stream write with compaction.
*/
public class TestWriteMergeOnReadWithCompact extends TestWriteCopyOnWrite {
@Override
protected void setUp(Configuration conf) {
// trigger the compaction for every finished checkpoint
conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, 1);
}
@Disabled
@Test
public void testIndexStateBootstrap() {
// Ignore the index bootstrap because we only support parquet load now.
}
Map<String, String> getMiniBatchExpected() {
Map<String, String> expected = new HashMap<>();
// MOR mode merges the messages with the same key.
expected.put("par1", "[id1,par1,id1,Danny,23,1,par1]");
return expected;
}
@Override
protected String getTableType() {
return HoodieTableType.MERGE_ON_READ.name();
}
}

View File

@@ -0,0 +1,142 @@
/*
* 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.operator.utils;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.operator.compact.CompactFunction;
import org.apache.hudi.operator.compact.CompactionCommitEvent;
import org.apache.hudi.operator.compact.CompactionCommitSink;
import org.apache.hudi.operator.compact.CompactionPlanEvent;
import org.apache.hudi.operator.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.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.MockEnvironment;
import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.util.Collector;
import org.apache.flink.util.OutputTag;
import java.util.ArrayList;
import java.util.List;
/**
* A wrapper class to manipulate the {@link org.apache.hudi.operator.compact.CompactFunction} instance for testing.
*/
public class CompactFunctionWrapper {
private final Configuration conf;
private final IOManager ioManager;
private final StreamingRuntimeContext runtimeContext;
private final MockFunctionInitializationContext functionInitializationContext;
/** Function that generates the {@link HoodieCompactionPlan}. */
private CompactionPlanOperator compactionPlanFunction;
/** Function that executes the compaction task. */
private CompactFunction compactFunction;
/** Stream sink to handle compaction commits. */
private CompactionCommitSink commitSink;
public CompactFunctionWrapper(Configuration conf) throws Exception {
this.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.conf = conf;
this.functionInitializationContext = new MockFunctionInitializationContext();
}
public void openFunction() throws Exception {
compactionPlanFunction = new CompactionPlanOperator(conf);
compactionPlanFunction.open();
compactFunction = new CompactFunction(conf);
compactFunction.setRuntimeContext(runtimeContext);
compactFunction.open(conf);
commitSink = new CompactionCommitSink(conf);
commitSink.setRuntimeContext(runtimeContext);
commitSink.open(conf);
}
public void compact(long checkpointID) throws Exception {
List<CompactionPlanEvent> events = new ArrayList<>();
// collect the CompactEvents.
Output<StreamRecord<CompactionPlanEvent>> output = new Output<StreamRecord<CompactionPlanEvent>>() {
@Override
public void emitWatermark(Watermark watermark) {
}
@Override
public <X> void collect(OutputTag<X> outputTag, StreamRecord<X> streamRecord) {
}
@Override
public void emitLatencyMarker(LatencyMarker latencyMarker) {
}
@Override
public void collect(StreamRecord<CompactionPlanEvent> record) {
events.add(record.getValue());
}
@Override
public void close() {
}
};
compactionPlanFunction.setOutput(output);
compactionPlanFunction.notifyCheckpointComplete(checkpointID);
// collect the CompactCommitEvents
List<CompactionCommitEvent> compactCommitEvents = new ArrayList<>();
for (CompactionPlanEvent event: events) {
compactFunction.processElement(event, null, new Collector<CompactionCommitEvent>() {
@Override
public void collect(CompactionCommitEvent event) {
compactCommitEvents.add(event);
}
@Override
public void close() {
}
});
}
// handle and commit the compaction
for (CompactionCommitEvent event : compactCommitEvents) {
commitSink.invoke(event, null);
}
}
public void close() throws Exception {
ioManager.close();
}
}

View File

@@ -21,6 +21,8 @@ package org.apache.hudi.operator.utils;
import org.apache.hudi.client.HoodieFlinkWriteClient;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.operator.FlinkOptions;
import org.apache.hudi.operator.StreamWriteFunction;
import org.apache.hudi.operator.StreamWriteOperatorCoordinator;
import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
@@ -64,6 +66,8 @@ public class StreamWriteFunctionWrapper<I> {
/** Stream write function. */
private StreamWriteFunction<Object, HoodieRecord<?>, Object> writeFunction;
private CompactFunctionWrapper compactFunctionWrapper;
public StreamWriteFunctionWrapper(String tablePath) throws Exception {
this(tablePath, TestConfigurations.getDefaultConf(tablePath));
}
@@ -81,6 +85,7 @@ public class StreamWriteFunctionWrapper<I> {
// one function
this.coordinator = new StreamWriteOperatorCoordinator(conf, 1);
this.functionInitializationContext = new MockFunctionInitializationContext();
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
}
public void openFunction() throws Exception {
@@ -98,6 +103,10 @@ public class StreamWriteFunctionWrapper<I> {
writeFunction.setRuntimeContext(runtimeContext);
writeFunction.setOperatorEventGateway(gateway);
writeFunction.open(conf);
if (conf.getBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED)) {
compactFunctionWrapper.openFunction();
}
}
public void invoke(I record) throws Exception {
@@ -149,6 +158,13 @@ public class StreamWriteFunctionWrapper<I> {
coordinator.checkpointComplete(checkpointId);
this.bucketAssignerFunction.notifyCheckpointComplete(checkpointId);
this.writeFunction.notifyCheckpointComplete(checkpointId);
if (conf.getBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED)) {
try {
compactFunctionWrapper.compact(checkpointId);
} catch (Exception e) {
throw new HoodieException(e);
}
}
}
public void checkpointFails(long checkpointId) {

View File

@@ -22,10 +22,12 @@ import org.apache.hudi.client.FlinkTaskContextSupplier;
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.StringData;
@@ -36,6 +38,7 @@ import org.apache.flink.table.data.writer.BinaryWriter;
import org.apache.flink.table.runtime.types.InternalSerializers;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.Strings;
import org.apache.parquet.avro.AvroParquetReader;
@@ -49,6 +52,8 @@ import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import static junit.framework.TestCase.assertEquals;
@@ -107,6 +112,28 @@ public class TestData {
TimestampData.fromEpochMillis(1), StringData.fromString("par1"))));
}
public static List<RowData> DATA_SET_FOUR = Arrays.asList(
// update: advance the age by 1
binaryRow(StringData.fromString("id1"), StringData.fromString("Danny"), 24,
TimestampData.fromEpochMillis(2), StringData.fromString("par1")),
binaryRow(StringData.fromString("id2"), StringData.fromString("Stephen"), 34,
TimestampData.fromEpochMillis(3), StringData.fromString("par1")),
binaryRow(StringData.fromString("id3"), StringData.fromString("Julian"), 54,
TimestampData.fromEpochMillis(4), StringData.fromString("par2")),
binaryRow(StringData.fromString("id4"), StringData.fromString("Fabian"), 32,
TimestampData.fromEpochMillis(5), StringData.fromString("par2")),
// same with before
binaryRow(StringData.fromString("id5"), StringData.fromString("Sophia"), 18,
TimestampData.fromEpochMillis(6), StringData.fromString("par3")),
// new data
binaryRow(StringData.fromString("id9"), StringData.fromString("Jane"), 19,
TimestampData.fromEpochMillis(6), StringData.fromString("par3")),
binaryRow(StringData.fromString("id10"), StringData.fromString("Ella"), 38,
TimestampData.fromEpochMillis(7), StringData.fromString("par4")),
binaryRow(StringData.fromString("id11"), StringData.fromString("Phoebe"), 52,
TimestampData.fromEpochMillis(8), StringData.fromString("par4"))
);
/**
* Checks the source data TestConfigurations.DATA_SET_ONE are written as expected.
*
@@ -201,6 +228,75 @@ public class TestData {
}
/**
* Checks the MERGE_ON_READ source data are written as expected.
*
* <p>Note: Replace it with the Flink reader when it is supported.
*
* @param fs The file system
* @param latestInstant The latest committed instant of current table
* @param baseFile The file base to check, should be a directory
* @param expected The expected results mapping, the key should be the partition path
* @param partitions The expected partition number
* @param schema The read schema
*/
public static void checkWrittenDataMOR(
FileSystem fs,
String latestInstant,
File baseFile,
Map<String, String> expected,
int partitions,
Schema schema) {
assert baseFile.isDirectory() : "Base path should be a directory";
FileFilter partitionFilter = file -> !file.getName().startsWith(".");
File[] partitionDirs = baseFile.listFiles(partitionFilter);
assertNotNull(partitionDirs);
assertThat(partitionDirs.length, is(partitions));
for (File partitionDir : partitionDirs) {
File[] dataFiles = partitionDir.listFiles(file ->
file.getName().contains(".log.") && !file.getName().startsWith(".."));
assertNotNull(dataFiles);
HoodieMergedLogRecordScanner scanner = getScanner(
fs, baseFile.getPath(), Arrays.stream(dataFiles).map(File::getAbsolutePath)
.sorted(Comparator.naturalOrder()).collect(Collectors.toList()),
schema, latestInstant);
List<String> readBuffer = scanner.getRecords().values().stream()
.map(hoodieRecord -> {
try {
return filterOutVariables((GenericRecord) hoodieRecord.getData().getInsertValue(schema, new Properties()).get());
} catch (IOException e) {
throw new RuntimeException(e);
}
})
.sorted(Comparator.naturalOrder())
.collect(Collectors.toList());
assertThat(readBuffer.toString(), is(expected.get(partitionDir.getName())));
}
}
/**
* Returns the scanner to read avro log files.
*/
private static HoodieMergedLogRecordScanner getScanner(
FileSystem fs,
String basePath,
List<String> logPaths,
Schema readSchema,
String instant) {
return HoodieMergedLogRecordScanner.newBuilder()
.withFileSystem(fs)
.withBasePath(basePath)
.withLogFilePaths(logPaths)
.withReaderSchema(readSchema)
.withLatestInstantTime(instant)
.withReadBlocksLazily(false)
.withReverseReader(false)
.withBufferSize(16 * 1024 * 1024)
.withMaxMemorySizeInBytes(1024 * 1024L)
.withSpillableMapBasePath("/tmp/")
.build();
}
/**
* Filter out the variables like file name.
*/