1
0

[HUDI-1522] Add a new pipeline for Flink writer (#2430)

* [HUDI-1522] Add a new pipeline for Flink writer
This commit is contained in:
Danny Chan
2021-01-28 08:53:13 +08:00
committed by GitHub
parent 7b2e658ac0
commit bc0325f6ea
40 changed files with 3613 additions and 302 deletions

View File

@@ -0,0 +1,303 @@
/*
* 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.HoodieFlinkWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
import org.apache.hudi.operator.utils.StreamWriteFunctionWrapper;
import org.apache.hudi.operator.utils.TestConfigurations;
import org.apache.hudi.operator.utils.TestData;
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
import org.apache.flink.table.data.RowData;
import org.hamcrest.MatcherAssert;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import java.io.File;
import java.util.Comparator;
import java.util.HashMap;
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;
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.
*/
public class StreamWriteFunctionTest {
private static final Map<String, String> EXPECTED = new HashMap<>();
static {
EXPECTED.put("par1", "[id1,par1,id1,Danny,23,1,par1, id2,par1,id2,Stephen,33,2,par1]");
EXPECTED.put("par2", "[id3,par2,id3,Julian,53,3,par2, id4,par2,id4,Fabian,31,4,par2]");
EXPECTED.put("par3", "[id5,par3,id5,Sophia,18,5,par3, id6,par3,id6,Emma,20,6,par3]");
EXPECTED.put("par4", "[id7,par4,id7,Bob,44,7,par4, id8,par4,id8,Han,56,8,par4]");
}
private static final Map<String, String> EXPECTED2 = new HashMap<>();
static {
EXPECTED2.put("par1", "[id1,par1,id1,Danny,24,1,par1, id2,par1,id2,Stephen,34,2,par1]");
EXPECTED2.put("par2", "[id3,par2,id3,Julian,54,3,par2, id4,par2,id4,Fabian,32,4,par2]");
EXPECTED2.put("par3", "[id5,par3,id5,Sophia,18,5,par3, id6,par3,id6,Emma,20,6,par3, "
+ "id9,par3,id9,Jane,19,6,par3]");
EXPECTED2.put("par4", "[id10,par4,id10,Ella,38,7,par4, id11,par4,id11,Phoebe,52,8,par4, "
+ "id7,par4,id7,Bob,44,7,par4, id8,par4,id8,Han,56,8,par4]");
}
private StreamWriteFunctionWrapper<RowData> funcWrapper;
@TempDir
File tempFile;
@BeforeEach
public void before() throws Exception {
this.funcWrapper = new StreamWriteFunctionWrapper<>(
tempFile.getAbsolutePath(),
TestConfigurations.SERIALIZER);
}
@AfterEach
public void after() throws Exception {
funcWrapper.close();
}
@Test
public void testCheckpoint() throws Exception {
// open the function and ingest data
funcWrapper.openFunction();
for (RowData rowData : TestData.DATA_SET_ONE) {
funcWrapper.invoke(rowData);
}
// no checkpoint, so the coordinator does not accept any events
assertTrue(
funcWrapper.getEventBuffer().length == 1
&& funcWrapper.getEventBuffer()[0] == null, "The coordinator events buffer expect to be empty");
// this triggers the data write and event send
funcWrapper.checkpointFunction(1);
String instant = funcWrapper.getWriteClient()
.getInflightAndRequestedInstant("COPY_ON_WRITE");
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
MatcherAssert.assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
List<WriteStatus> writeStatuses = ((BatchWriteSuccessEvent) nextEvent).getWriteStatuses();
assertNotNull(writeStatuses);
MatcherAssert.assertThat(writeStatuses.size(), is(4)); // write 4 partition files
assertThat(writeStatuses.stream()
.map(WriteStatus::getPartitionPath).sorted(Comparator.naturalOrder())
.collect(Collectors.joining(",")),
is("par1,par2,par3,par4"));
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, instant);
funcWrapper.checkpointComplete(1);
// the coordinator checkpoint commits the inflight instant.
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
// checkpoint for next round, no data input, so after the checkpoint,
// there should not be REQUESTED Instant
// this triggers the data write and event send
funcWrapper.checkpointFunction(2);
String instant2 = funcWrapper.getWriteClient()
.getInflightAndRequestedInstant("COPY_ON_WRITE");
assertNotEquals(instant, instant2);
final OperatorEvent nextEvent2 = funcWrapper.getNextEvent();
assertThat("The operator expect to send an event", nextEvent2, instanceOf(BatchWriteSuccessEvent.class));
List<WriteStatus> writeStatuses2 = ((BatchWriteSuccessEvent) nextEvent2).getWriteStatuses();
assertNotNull(writeStatuses2);
assertThat(writeStatuses2.size(), is(0)); // write empty statuses
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent2);
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
funcWrapper.checkpointComplete(2);
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, null);
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
}
@Test
public void testCheckpointFails() throws Exception {
// open the function and ingest data
funcWrapper.openFunction();
// no data written and triggers checkpoint fails,
// then we should revert the start instant
// this triggers the data write and event send
funcWrapper.checkpointFunction(1);
String instant = funcWrapper.getWriteClient()
.getInflightAndRequestedInstant("COPY_ON_WRITE");
assertNotNull(instant);
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
List<WriteStatus> writeStatuses = ((BatchWriteSuccessEvent) nextEvent).getWriteStatuses();
assertNotNull(writeStatuses);
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");
// the instant metadata should be cleared
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, null);
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, null);
for (RowData rowData : TestData.DATA_SET_ONE) {
funcWrapper.invoke(rowData);
}
// this triggers the data write and event send
funcWrapper.checkpointFunction(2);
// Do not sent the write event and fails the checkpoint
assertThrows(HoodieException.class,
() -> funcWrapper.checkpointFails(2),
"The last checkpoint was aborted, roll back the last write and throw");
}
@Test
public void testInsert() throws Exception {
// open the function and ingest data
funcWrapper.openFunction();
for (RowData rowData : TestData.DATA_SET_ONE) {
funcWrapper.invoke(rowData);
}
assertEmptyDataFiles();
// this triggers the data write and event send
funcWrapper.checkpointFunction(1);
String instant = funcWrapper.getWriteClient()
.getInflightAndRequestedInstant("COPY_ON_WRITE");
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
checkWrittenData(tempFile, EXPECTED);
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, instant);
funcWrapper.checkpointComplete(1);
// the coordinator checkpoint commits the inflight instant.
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
checkWrittenData(tempFile, EXPECTED);
}
@Test
public void testUpsert() throws Exception {
// open the function and ingest data
funcWrapper.openFunction();
for (RowData rowData : TestData.DATA_SET_ONE) {
funcWrapper.invoke(rowData);
}
assertEmptyDataFiles();
// this triggers the data write and event send
funcWrapper.checkpointFunction(1);
OperatorEvent nextEvent = funcWrapper.getNextEvent();
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
funcWrapper.checkpointComplete(1);
// upsert another data buffer
for (RowData rowData : TestData.DATA_SET_TWO) {
funcWrapper.invoke(rowData);
}
// the data is not flushed yet
checkWrittenData(tempFile, EXPECTED);
// this triggers the data write and event send
funcWrapper.checkpointFunction(2);
String instant = funcWrapper.getWriteClient()
.getInflightAndRequestedInstant("COPY_ON_WRITE");
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");
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, instant);
funcWrapper.checkpointComplete(2);
// the coordinator checkpoint commits the inflight instant.
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
checkWrittenData(tempFile, EXPECTED2);
}
// -------------------------------------------------------------------------
// Utilities
// -------------------------------------------------------------------------
@SuppressWarnings("rawtypes")
private void checkInstantState(
HoodieFlinkWriteClient writeClient,
HoodieInstant.State state,
String instantStr) {
final String instant;
switch (state) {
case REQUESTED:
instant = writeClient.getInflightAndRequestedInstant("COPY_ON_WRITE");
break;
case COMPLETED:
instant = writeClient.getLastCompletedInstant("COPY_ON_WRITE");
break;
default:
throw new AssertionError("Unexpected state");
}
assertThat(instant, is(instantStr));
}
/**
* Asserts the data files are empty.
*/
private void assertEmptyDataFiles() {
File[] dataFiles = tempFile.listFiles(file -> !file.getName().startsWith("."));
assertNotNull(dataFiles);
assertThat(dataFiles.length, is(0));
}
}

View File

@@ -0,0 +1,129 @@
/*
* 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.operator.utils.TestConfigurations;
import org.apache.hudi.operator.utils.TestData;
import org.apache.hudi.util.AvroSchemaConverter;
import org.apache.hudi.util.StreamerUtil;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.api.common.io.FilePathFilter;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.io.TextInputFormat;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.core.fs.Path;
import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
import org.apache.flink.formats.json.TimestampFormat;
import org.apache.flink.streaming.api.CheckpointingMode;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.util.TestLogger;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import java.io.File;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
/**
* Integration test for Flink Hoodie stream sink.
*/
public class StreamWriteITCase extends TestLogger {
private static final Map<String, String> EXPECTED = new HashMap<>();
static {
EXPECTED.put("par1", "[id1,par1,id1,Danny,23,1000,par1, id2,par1,id2,Stephen,33,2000,par1]");
EXPECTED.put("par2", "[id3,par2,id3,Julian,53,3000,par2, id4,par2,id4,Fabian,31,4000,par2]");
EXPECTED.put("par3", "[id5,par3,id5,Sophia,18,5000,par3, id6,par3,id6,Emma,20,6000,par3]");
EXPECTED.put("par4", "[id7,par4,id7,Bob,44,7000,par4, id8,par4,id8,Han,56,8000,par4]");
}
@TempDir
File tempFile;
@Test
public void testWriteToHoodie() throws Exception {
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
execEnv.getConfig().disableObjectReuse();
execEnv.setParallelism(4);
// 1 second a time
execEnv.enableCheckpointing(2000, CheckpointingMode.EXACTLY_ONCE);
// Read from kafka source
RowType rowType =
(RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf))
.getLogicalType();
StreamWriteOperatorFactory<RowData> operatorFactory =
new StreamWriteOperatorFactory<>(rowType, conf, 4);
int partitionFieldIndex = rowType.getFieldIndex(conf.getString(FlinkOptions.PARTITION_PATH_FIELD));
final RowData.FieldGetter partitionFieldGetter =
RowData.createFieldGetter(rowType.getTypeAt(partitionFieldIndex), partitionFieldIndex);
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");
DataStream<Object> dataStream = 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)))
// Key-by partition path, to avoid multiple subtasks write to a partition at the same time
.keyBy(partitionFieldGetter::getFieldOrNull)
.transform("hoodie_stream_write", null, operatorFactory)
.uid("uid_hoodie_stream_write")
.setParallelism(4);
execEnv.addOperator(dataStream.getTransformation());
JobClient client = execEnv.executeAsync(execEnv.getStreamGraph(conf.getString(FlinkOptions.TABLE_NAME)));
if (client.getJobStatus().get() != JobStatus.FAILED) {
try {
TimeUnit.SECONDS.sleep(10);
client.cancel();
} catch (Throwable var1) {
// ignored
}
}
TestData.checkWrittenData(tempFile, EXPECTED);
}
}

View File

@@ -0,0 +1,101 @@
/*
* 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.fs.FSUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
import org.apache.hudi.operator.utils.TestConfigurations;
import org.apache.hudi.util.StreamerUtil;
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import java.io.File;
import java.io.IOException;
import java.util.Collections;
import java.util.concurrent.CompletableFuture;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Test cases for StreamingSinkOperatorCoordinator.
*/
public class StreamWriteOperatorCoordinatorTest {
private StreamWriteOperatorCoordinator coordinator;
@TempDir
File tempFile;
@BeforeEach
public void before() throws Exception {
coordinator = new StreamWriteOperatorCoordinator(
TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()), 2);
coordinator.start();
}
@AfterEach
public void after() {
coordinator.close();
}
@Test
public void testTableInitialized() throws IOException {
final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
String basePath = tempFile.getAbsolutePath();
try (FileSystem fs = FSUtils.getFs(basePath, hadoopConf)) {
assertTrue(fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME)));
}
}
@Test
public void testCheckpointAndRestore() throws Exception {
CompletableFuture<byte[]> future = new CompletableFuture<>();
coordinator.checkpointCoordinator(1, future);
coordinator.resetToCheckpoint(future.get());
}
@Test
public void testReceiveInvalidEvent() {
CompletableFuture<byte[]> future = new CompletableFuture<>();
coordinator.checkpointCoordinator(1, future);
OperatorEvent event = new BatchWriteSuccessEvent(0, "abc", Collections.emptyList());
assertThrows(IllegalStateException.class,
() -> coordinator.handleEventFromOperator(0, event),
"Receive an unexpected event for instant abc from task 0");
}
@Test
public void testCheckpointInvalid() {
final CompletableFuture<byte[]> future = new CompletableFuture<>();
coordinator.checkpointCoordinator(1, future);
String inflightInstant = coordinator.getInFlightInstant();
OperatorEvent event = new BatchWriteSuccessEvent(0, inflightInstant, Collections.emptyList());
coordinator.handleEventFromOperator(0, event);
final CompletableFuture<byte[]> future2 = new CompletableFuture<>();
coordinator.checkpointCoordinator(2, future2);
assertTrue(future2.isCompletedExceptionally());
}
}

View File

@@ -0,0 +1,48 @@
/*
* 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.flink.api.common.state.KeyedStateStore;
import org.apache.flink.runtime.state.FunctionInitializationContext;
/**
* A {@link FunctionInitializationContext} for testing purpose.
*/
public class MockFunctionInitializationContext implements FunctionInitializationContext {
private final MockOperatorStateStore operatorStateStore;
public MockFunctionInitializationContext() {
operatorStateStore = new MockOperatorStateStore();
}
@Override
public boolean isRestored() {
throw new UnsupportedOperationException();
}
@Override
public MockOperatorStateStore getOperatorStateStore() {
return operatorStateStore;
}
@Override
public KeyedStateStore getKeyedStateStore() {
return operatorStateStore;
}
}

View File

@@ -0,0 +1,90 @@
/*
* 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.flink.api.common.state.MapState;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
/**
* Mock map state for testing.
*
* @param <K> Type of state key
* @param <V> Type of state value
*/
public class MockMapState<K, V> implements MapState<K, V> {
private final Map<K, V> backingMap = new HashMap<>();
@Override
public V get(K uk) {
return backingMap.get(uk);
}
@Override
public void put(K uk, V uv) {
backingMap.put(uk, uv);
}
@Override
public void putAll(Map<K, V> map) {
backingMap.putAll(map);
}
@Override
public void remove(K uk) {
backingMap.remove(uk);
}
@Override
public boolean contains(K uk) {
return backingMap.containsKey(uk);
}
@Override
public Iterable<Map.Entry<K, V>> entries() {
return backingMap.entrySet();
}
@Override
public Iterable<K> keys() {
return backingMap.keySet();
}
@Override
public Iterable<V> values() {
return backingMap.values();
}
@Override
public Iterator<Map.Entry<K, V>> iterator() {
return backingMap.entrySet().iterator();
}
@Override
public boolean isEmpty() {
return backingMap.isEmpty();
}
@Override
public void clear() {
backingMap.clear();
}
}

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.operator.utils;
import org.apache.flink.api.common.state.AggregatingState;
import org.apache.flink.api.common.state.AggregatingStateDescriptor;
import org.apache.flink.api.common.state.BroadcastState;
import org.apache.flink.api.common.state.FoldingState;
import org.apache.flink.api.common.state.FoldingStateDescriptor;
import org.apache.flink.api.common.state.KeyedStateStore;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.state.MapState;
import org.apache.flink.api.common.state.MapStateDescriptor;
import org.apache.flink.api.common.state.OperatorStateStore;
import org.apache.flink.api.common.state.ReducingState;
import org.apache.flink.api.common.state.ReducingStateDescriptor;
import org.apache.flink.api.common.state.ValueState;
import org.apache.flink.api.common.state.ValueStateDescriptor;
import org.apache.flink.streaming.api.functions.sink.filesystem.TestUtils;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
/**
* An {@link OperatorStateStore} for testing purpose.
*/
@SuppressWarnings("rawtypes")
public class MockOperatorStateStore implements KeyedStateStore, OperatorStateStore {
private final Map<Long, Map<String, TestUtils.MockListState>> historyStateMap;
private Map<String, TestUtils.MockListState> currentStateMap;
private Map<String, TestUtils.MockListState> lastSuccessStateMap;
private MapState mapState;
public MockOperatorStateStore() {
this.historyStateMap = new HashMap<>();
this.currentStateMap = new HashMap<>();
this.lastSuccessStateMap = new HashMap<>();
this.mapState = new MockMapState<>();
}
@Override
public <K, V> BroadcastState<K, V> getBroadcastState(MapStateDescriptor<K, V> stateDescriptor) throws Exception {
return null;
}
@Override
public <T> ValueState<T> getState(ValueStateDescriptor<T> valueStateDescriptor) {
return null;
}
@Override
@SuppressWarnings("unchecked")
public <S> ListState<S> getListState(ListStateDescriptor<S> stateDescriptor) {
String name = stateDescriptor.getName();
currentStateMap.putIfAbsent(name, new TestUtils.MockListState());
return currentStateMap.get(name);
}
@Override
public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> reducingStateDescriptor) {
return null;
}
@Override
public <I, A, O> AggregatingState<I, O> getAggregatingState(AggregatingStateDescriptor<I, A, O> aggregatingStateDescriptor) {
return null;
}
@Override
public <T, A> FoldingState<T, A> getFoldingState(FoldingStateDescriptor<T, A> foldingStateDescriptor) {
return null;
}
@Override
@SuppressWarnings("unchecked")
public <K, V> MapState<K, V> getMapState(MapStateDescriptor<K, V> mapStateDescriptor) {
return this.mapState;
}
@Override
public <S> ListState<S> getUnionListState(ListStateDescriptor<S> stateDescriptor) throws Exception {
throw new UnsupportedOperationException();
}
@Override
public Set<String> getRegisteredStateNames() {
throw new UnsupportedOperationException();
}
@Override
public Set<String> getRegisteredBroadcastStateNames() {
throw new UnsupportedOperationException();
}
public void checkpointBegin(long checkpointId) {
Map<String, TestUtils.MockListState> copiedStates = Collections.unmodifiableMap(copyStates(currentStateMap));
historyStateMap.put(checkpointId, copiedStates);
}
public void checkpointSuccess(long checkpointId) {
lastSuccessStateMap = historyStateMap.get(checkpointId);
}
public void rollBackToLastSuccessCheckpoint() {
this.currentStateMap = copyStates(lastSuccessStateMap);
}
@SuppressWarnings("unchecked")
private Map<String, TestUtils.MockListState> copyStates(Map<String, TestUtils.MockListState> stateMap) {
Map<String, TestUtils.MockListState> copiedStates = new HashMap<>();
for (Map.Entry<String, TestUtils.MockListState> entry : stateMap.entrySet()) {
TestUtils.MockListState copiedState = new TestUtils.MockListState();
copiedState.addAll(entry.getValue().getBackingList());
copiedStates.put(entry.getKey(), copiedState);
}
return copiedStates;
}
}

View File

@@ -0,0 +1,124 @@
/*
* 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.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.state.KeyedStateStore;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
import org.apache.flink.runtime.jobgraph.OperatorID;
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.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
import java.util.HashMap;
/**
* Mock {@link StreamingRuntimeContext} to use in tests.
*
* <p>NOTE: Adapted from Apache Flink, the MockStreamOperator is modified to support MapState.
*/
public class MockStreamingRuntimeContext extends StreamingRuntimeContext {
private final boolean isCheckpointingEnabled;
private final int numParallelSubtasks;
private final int subtaskIndex;
public MockStreamingRuntimeContext(
boolean isCheckpointingEnabled,
int numParallelSubtasks,
int subtaskIndex) {
this(isCheckpointingEnabled, numParallelSubtasks, subtaskIndex, new MockEnvironmentBuilder()
.setTaskName("mockTask")
.setManagedMemorySize(4 * MemoryManager.DEFAULT_PAGE_SIZE)
.build());
}
public MockStreamingRuntimeContext(
boolean isCheckpointingEnabled,
int numParallelSubtasks,
int subtaskIndex,
MockEnvironment environment) {
super(new MockStreamOperator(), environment, new HashMap<>());
this.isCheckpointingEnabled = isCheckpointingEnabled;
this.numParallelSubtasks = numParallelSubtasks;
this.subtaskIndex = subtaskIndex;
}
@Override
public MetricGroup getMetricGroup() {
return new UnregisteredMetricsGroup();
}
@Override
public boolean isCheckpointingEnabled() {
return isCheckpointingEnabled;
}
@Override
public int getIndexOfThisSubtask() {
return subtaskIndex;
}
@Override
public int getNumberOfParallelSubtasks() {
return numParallelSubtasks;
}
private static class MockStreamOperator extends AbstractStreamOperator<Integer> {
private static final long serialVersionUID = -1153976702711944427L;
private transient TestProcessingTimeService testProcessingTimeService;
private transient MockOperatorStateStore mockOperatorStateStore;
@Override
public ExecutionConfig getExecutionConfig() {
return new ExecutionConfig();
}
@Override
public OperatorID getOperatorID() {
return new OperatorID();
}
@Override
public ProcessingTimeService getProcessingTimeService() {
if (testProcessingTimeService == null) {
testProcessingTimeService = new TestProcessingTimeService();
}
return testProcessingTimeService;
}
@Override
public KeyedStateStore getKeyedStateStore() {
if (mockOperatorStateStore == null) {
mockOperatorStateStore = new MockOperatorStateStore();
}
return mockOperatorStateStore;
}
}
}

View File

@@ -0,0 +1,122 @@
/*
* 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.flink.api.common.typeutils.TypeSerializer;
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.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.MockFunctionSnapshotContext;
import org.apache.flink.streaming.api.operators.collect.utils.MockOperatorEventGateway;
import org.apache.hudi.client.HoodieFlinkWriteClient;
import org.apache.hudi.operator.StreamWriteFunction;
import org.apache.hudi.operator.StreamWriteOperatorCoordinator;
import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
import java.util.concurrent.CompletableFuture;
/**
* A wrapper class to manipulate the {@link StreamWriteFunction} instance for testing.
*
* @param <I> Input type
*/
public class StreamWriteFunctionWrapper<I> {
private final TypeSerializer<I> serializer;
private final Configuration conf;
private final IOManager ioManager;
private final StreamingRuntimeContext runtimeContext;
private final MockOperatorEventGateway gateway;
private final StreamWriteOperatorCoordinator coordinator;
private final MockFunctionInitializationContext functionInitializationContext;
private StreamWriteFunction<Object, I, Object> function;
public StreamWriteFunctionWrapper(String tablePath, TypeSerializer<I> serializer) throws Exception {
this.serializer = serializer;
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.gateway = new MockOperatorEventGateway();
this.conf = TestConfigurations.getDefaultConf(tablePath);
// one function
this.coordinator = new StreamWriteOperatorCoordinator(conf, 1);
this.coordinator.start();
this.functionInitializationContext = new MockFunctionInitializationContext();
}
public void openFunction() throws Exception {
function = new StreamWriteFunction<>(TestConfigurations.ROW_TYPE, this.conf);
function.setRuntimeContext(runtimeContext);
function.setOperatorEventGateway(gateway);
function.open(this.conf);
}
public void invoke(I record) throws Exception {
function.processElement(record, null, null);
}
public BatchWriteSuccessEvent[] getEventBuffer() {
return this.coordinator.getEventBuffer();
}
public OperatorEvent getNextEvent() {
return this.gateway.getNextEvent();
}
@SuppressWarnings("rawtypes")
public HoodieFlinkWriteClient getWriteClient() {
return this.function.getWriteClient();
}
public void checkpointFunction(long checkpointId) throws Exception {
// checkpoint the coordinator first
this.coordinator.checkpointCoordinator(checkpointId, new CompletableFuture<>());
function.snapshotState(new MockFunctionSnapshotContext(checkpointId));
functionInitializationContext.getOperatorStateStore().checkpointBegin(checkpointId);
}
public void checkpointComplete(long checkpointId) {
functionInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId);
coordinator.checkpointComplete(checkpointId);
}
public void checkpointFails(long checkpointId) {
coordinator.notifyCheckpointAborted(checkpointId);
}
public void close() throws Exception {
coordinator.close();
ioManager.close();
}
public StreamWriteOperatorCoordinator getCoordinator() {
return coordinator;
}
}

View File

@@ -0,0 +1,59 @@
/*
* 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.operator.FlinkOptions;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
import org.apache.flink.table.types.logical.RowType;
import java.util.Objects;
/**
* Configurations for the test.
*/
public class TestConfigurations {
private TestConfigurations() {
}
public static final RowType ROW_TYPE = (RowType) DataTypes.ROW(
DataTypes.FIELD("uuid", DataTypes.VARCHAR(20)),// record key
DataTypes.FIELD("name", DataTypes.VARCHAR(10)),
DataTypes.FIELD("age", DataTypes.INT()),
DataTypes.FIELD("ts", DataTypes.TIMESTAMP(3)), // precombine field
DataTypes.FIELD("partition", DataTypes.VARCHAR(10)))
.notNull()
.getLogicalType();
public static final RowDataSerializer SERIALIZER = new RowDataSerializer(new ExecutionConfig(), ROW_TYPE);
public static Configuration getDefaultConf(String tablePath) {
Configuration conf = new Configuration();
conf.setString(FlinkOptions.PATH, tablePath);
conf.setString(FlinkOptions.READ_SCHEMA_FILE_PATH,
Objects.requireNonNull(Thread.currentThread()
.getContextClassLoader().getResource("test_read_schema.avsc")).toString());
conf.setString(FlinkOptions.TABLE_NAME, "TestHoodieTable");
conf.setString(FlinkOptions.PARTITION_PATH_FIELD, "partition");
return conf;
}
}

View File

@@ -0,0 +1,164 @@
/*
* 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.common.fs.FSUtils;
import org.apache.avro.generic.GenericRecord;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.StringData;
import org.apache.flink.table.data.TimestampData;
import org.apache.flink.table.data.binary.BinaryRowData;
import org.apache.flink.table.data.writer.BinaryRowWriter;
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.Path;
import org.apache.parquet.Strings;
import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.hadoop.ParquetReader;
import java.io.File;
import java.io.FileFilter;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import static junit.framework.TestCase.assertEquals;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertNotNull;
/** Data set for testing, also some utilities to check the results. */
public class TestData {
public static List<RowData> DATA_SET_ONE = Arrays.asList(
binaryRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23,
TimestampData.fromEpochMillis(1), StringData.fromString("par1")),
binaryRow(StringData.fromString("id2"), StringData.fromString("Stephen"), 33,
TimestampData.fromEpochMillis(2), StringData.fromString("par1")),
binaryRow(StringData.fromString("id3"), StringData.fromString("Julian"), 53,
TimestampData.fromEpochMillis(3), StringData.fromString("par2")),
binaryRow(StringData.fromString("id4"), StringData.fromString("Fabian"), 31,
TimestampData.fromEpochMillis(4), StringData.fromString("par2")),
binaryRow(StringData.fromString("id5"), StringData.fromString("Sophia"), 18,
TimestampData.fromEpochMillis(5), StringData.fromString("par3")),
binaryRow(StringData.fromString("id6"), StringData.fromString("Emma"), 20,
TimestampData.fromEpochMillis(6), StringData.fromString("par3")),
binaryRow(StringData.fromString("id7"), StringData.fromString("Bob"), 44,
TimestampData.fromEpochMillis(7), StringData.fromString("par4")),
binaryRow(StringData.fromString("id8"), StringData.fromString("Han"), 56,
TimestampData.fromEpochMillis(8), StringData.fromString("par4"))
);
public static List<RowData> DATA_SET_TWO = Arrays.asList(
// advance the age by 1
binaryRow(StringData.fromString("id1"), StringData.fromString("Danny"), 24,
TimestampData.fromEpochMillis(1), StringData.fromString("par1")),
binaryRow(StringData.fromString("id2"), StringData.fromString("Stephen"), 34,
TimestampData.fromEpochMillis(2), StringData.fromString("par1")),
binaryRow(StringData.fromString("id3"), StringData.fromString("Julian"), 54,
TimestampData.fromEpochMillis(3), StringData.fromString("par2")),
binaryRow(StringData.fromString("id4"), StringData.fromString("Fabian"), 32,
TimestampData.fromEpochMillis(4), StringData.fromString("par2")),
// same with before
binaryRow(StringData.fromString("id5"), StringData.fromString("Sophia"), 18,
TimestampData.fromEpochMillis(5), 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.
*
* <p>Note: Replace it with the Flink reader when it is supported.
*
* @param baseFile The file base to check, should be a directly
* @param expected The expected results mapping, the key should be the partition path
*/
public static void checkWrittenData(File baseFile, Map<String, String> expected) throws IOException {
assert baseFile.isDirectory();
FileFilter filter = file -> !file.getName().startsWith(".");
File[] partitionDirs = baseFile.listFiles(filter);
assertNotNull(partitionDirs);
assertThat(partitionDirs.length, is(4));
for (File partitionDir : partitionDirs) {
File[] dataFiles = partitionDir.listFiles(file -> file.getName().endsWith(".parquet"));
assertNotNull(dataFiles);
File latestDataFile = Arrays.stream(dataFiles)
.max(Comparator.comparing(f -> FSUtils.getCommitTime(f.getName())))
.orElse(dataFiles[0]);
ParquetReader<GenericRecord> reader = AvroParquetReader
.<GenericRecord>builder(new Path(latestDataFile.getAbsolutePath())).build();
List<String> readBuffer = new ArrayList<>();
GenericRecord nextRecord = reader.read();
while (nextRecord != null) {
readBuffer.add(filterOutVariables(nextRecord));
nextRecord = reader.read();
}
readBuffer.sort(Comparator.naturalOrder());
assertThat(readBuffer.toString(), is(expected.get(partitionDir.getName())));
}
}
/**
* Filter out the variables like file name.
*/
private static String filterOutVariables(GenericRecord genericRecord) {
List<String> fields = new ArrayList<>();
fields.add(genericRecord.get("_hoodie_record_key").toString());
fields.add(genericRecord.get("_hoodie_partition_path").toString());
fields.add(genericRecord.get("uuid").toString());
fields.add(genericRecord.get("name").toString());
fields.add(genericRecord.get("age").toString());
fields.add(genericRecord.get("ts").toString());
fields.add(genericRecord.get("partition").toString());
return Strings.join(fields, ",");
}
private static BinaryRowData binaryRow(Object... fields) {
LogicalType[] types = TestConfigurations.ROW_TYPE.getFields().stream().map(RowType.RowField::getType)
.toArray(LogicalType[]::new);
assertEquals(
"Filed count inconsistent with type information",
fields.length,
types.length);
BinaryRowData row = new BinaryRowData(fields.length);
BinaryRowWriter writer = new BinaryRowWriter(row);
writer.reset();
for (int i = 0; i < fields.length; i++) {
Object field = fields[i];
if (field == null) {
writer.setNullAt(i);
} else {
BinaryWriter.write(writer, i, field, types[i], InternalSerializers.create(types[i]));
}
}
writer.complete();
return row;
}
}

View File

@@ -70,7 +70,7 @@ public class TestJsonStringToHoodieRecordMapFunction extends HoodieFlinkClientTe
props.put(HoodieWriteConfig.WRITE_PAYLOAD_CLASS, OverwriteWithLatestAvroPayload.class.getName());
props.put(HoodieWriteConfig.PRECOMBINE_FIELD_PROP, "timestamp");
props.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key");
props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionPath");
props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "current_date");
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(2);

View File

@@ -0,0 +1,45 @@
/*
* 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.
*/
{
"type" : "record",
"name" : "record",
"fields" : [ {
"name" : "uuid",
"type" : [ "null", "string" ],
"default" : null
}, {
"name" : "name",
"type" : [ "null", "string" ],
"default" : null
}, {
"name" : "age",
"type" : [ "null", "int" ],
"default" : null
}, {
"name" : "ts",
"type" : [ "null", {
"type" : "long",
"logicalType" : "timestamp-millis"
} ],
"default" : null
}, {
"name" : "partition",
"type" : [ "null", "string" ],
"default" : null
} ]
}

View File

@@ -0,0 +1,8 @@
{"uuid": "id1", "name": "Danny", "age": 23, "ts": "1970-01-01T00:00:01", "partition": "par1"}
{"uuid": "id2", "name": "Stephen", "age": 33, "ts": "1970-01-01T00:00:02", "partition": "par1"}
{"uuid": "id3", "name": "Julian", "age": 53, "ts": "1970-01-01T00:00:03", "partition": "par2"}
{"uuid": "id4", "name": "Fabian", "age": 31, "ts": "1970-01-01T00:00:04", "partition": "par2"}
{"uuid": "id5", "name": "Sophia", "age": 18, "ts": "1970-01-01T00:00:05", "partition": "par3"}
{"uuid": "id6", "name": "Emma", "age": 20, "ts": "1970-01-01T00:00:06", "partition": "par3"}
{"uuid": "id7", "name": "Bob", "age": 44, "ts": "1970-01-01T00:00:07", "partition": "par4"}
{"uuid": "id8", "name": "Han", "age": 56, "ts": "1970-01-01T00:00:08", "partition": "par4"}