[HUDI-3665] Support flink multiple versions (#5072)
This commit is contained in:
@@ -0,0 +1,245 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.sink.transform.ChainedTransformer;
|
||||
import org.apache.hudi.sink.transform.Transformer;
|
||||
import org.apache.hudi.sink.utils.Pipelines;
|
||||
import org.apache.hudi.util.AvroSchemaConverter;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
import org.apache.hudi.utils.TestData;
|
||||
import org.apache.hudi.utils.source.ContinuousFileSource;
|
||||
|
||||
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.common.TimestampFormat;
|
||||
import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
|
||||
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.GenericRowData;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
|
||||
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 org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.io.File;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* Integration test for Flink Hoodie stream sink.
|
||||
*/
|
||||
public class ITTestDataStreamWrite extends TestLogger {
|
||||
|
||||
private static final Map<String, List<String>> EXPECTED = new HashMap<>();
|
||||
private static final Map<String, List<String>> EXPECTED_TRANSFORMER = new HashMap<>();
|
||||
private static final Map<String, List<String>> EXPECTED_CHAINED_TRANSFORMER = new HashMap<>();
|
||||
|
||||
static {
|
||||
EXPECTED.put("par1", Arrays.asList("id1,par1,id1,Danny,23,1000,par1", "id2,par1,id2,Stephen,33,2000,par1"));
|
||||
EXPECTED.put("par2", Arrays.asList("id3,par2,id3,Julian,53,3000,par2", "id4,par2,id4,Fabian,31,4000,par2"));
|
||||
EXPECTED.put("par3", Arrays.asList("id5,par3,id5,Sophia,18,5000,par3", "id6,par3,id6,Emma,20,6000,par3"));
|
||||
EXPECTED.put("par4", Arrays.asList("id7,par4,id7,Bob,44,7000,par4", "id8,par4,id8,Han,56,8000,par4"));
|
||||
|
||||
EXPECTED_TRANSFORMER.put("par1", Arrays.asList("id1,par1,id1,Danny,24,1000,par1", "id2,par1,id2,Stephen,34,2000,par1"));
|
||||
EXPECTED_TRANSFORMER.put("par2", Arrays.asList("id3,par2,id3,Julian,54,3000,par2", "id4,par2,id4,Fabian,32,4000,par2"));
|
||||
EXPECTED_TRANSFORMER.put("par3", Arrays.asList("id5,par3,id5,Sophia,19,5000,par3", "id6,par3,id6,Emma,21,6000,par3"));
|
||||
EXPECTED_TRANSFORMER.put("par4", Arrays.asList("id7,par4,id7,Bob,45,7000,par4", "id8,par4,id8,Han,57,8000,par4"));
|
||||
|
||||
EXPECTED_CHAINED_TRANSFORMER.put("par1", Arrays.asList("id1,par1,id1,Danny,25,1000,par1", "id2,par1,id2,Stephen,35,2000,par1"));
|
||||
EXPECTED_CHAINED_TRANSFORMER.put("par2", Arrays.asList("id3,par2,id3,Julian,55,3000,par2", "id4,par2,id4,Fabian,33,4000,par2"));
|
||||
EXPECTED_CHAINED_TRANSFORMER.put("par3", Arrays.asList("id5,par3,id5,Sophia,20,5000,par3", "id6,par3,id6,Emma,22,6000,par3"));
|
||||
EXPECTED_CHAINED_TRANSFORMER.put("par4", Arrays.asList("id7,par4,id7,Bob,46,7000,par4", "id8,par4,id8,Han,58,8000,par4"));
|
||||
}
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@Test
|
||||
public void testTransformerBeforeWriting() throws Exception {
|
||||
Transformer transformer = (ds) -> ds.map((rowdata) -> {
|
||||
if (rowdata instanceof GenericRowData) {
|
||||
GenericRowData genericRD = (GenericRowData) rowdata;
|
||||
//update age field to age + 1
|
||||
genericRD.setField(2, genericRD.getInt(2) + 1);
|
||||
return genericRD;
|
||||
} else {
|
||||
throw new RuntimeException("Unrecognized row type information: " + rowdata.getClass().getSimpleName());
|
||||
}
|
||||
});
|
||||
|
||||
testWriteToHoodie(transformer, EXPECTED_TRANSFORMER);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testChainedTransformersBeforeWriting() throws Exception {
|
||||
Transformer t1 = (ds) -> ds.map((rowdata) -> {
|
||||
if (rowdata instanceof GenericRowData) {
|
||||
GenericRowData genericRD = (GenericRowData) rowdata;
|
||||
//update age field to age + 1
|
||||
genericRD.setField(2, genericRD.getInt(2) + 1);
|
||||
return genericRD;
|
||||
} else {
|
||||
throw new RuntimeException("Unrecognized row type : " + rowdata.getClass().getSimpleName());
|
||||
}
|
||||
});
|
||||
|
||||
ChainedTransformer chainedTransformer = new ChainedTransformer(Arrays.asList(t1, t1));
|
||||
|
||||
testWriteToHoodie(chainedTransformer, EXPECTED_CHAINED_TRANSFORMER);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteToHoodieWithoutTransformer() throws Exception {
|
||||
testWriteToHoodie(null, EXPECTED);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {"BUCKET", "FLINK_STATE"})
|
||||
public void testMergeOnReadWriteWithCompaction(String indexType) throws Exception {
|
||||
int parallelism = 4;
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setString(FlinkOptions.INDEX_TYPE, indexType);
|
||||
conf.setInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS, 4);
|
||||
conf.setString(FlinkOptions.INDEX_KEY_FIELD, "id");
|
||||
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(parallelism);
|
||||
// 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();
|
||||
|
||||
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
|
||||
rowType,
|
||||
InternalTypeInfo.of(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<RowData> 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)))
|
||||
.setParallelism(parallelism);
|
||||
|
||||
DataStream<HoodieRecord> hoodieRecordDataStream = Pipelines.bootstrap(conf, rowType, parallelism, dataStream);
|
||||
DataStream<Object> pipeline = Pipelines.hoodieStreamWrite(conf, parallelism, hoodieRecordDataStream);
|
||||
Pipelines.clean(conf, pipeline);
|
||||
Pipelines.compact(conf, pipeline);
|
||||
JobClient client = execEnv.executeAsync("mor-write-with-compact");
|
||||
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);
|
||||
}
|
||||
|
||||
private void testWriteToHoodie(
|
||||
Transformer transformer,
|
||||
Map<String, List<String>> expected) throws Exception {
|
||||
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
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();
|
||||
|
||||
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
|
||||
rowType,
|
||||
InternalTypeInfo.of(rowType),
|
||||
false,
|
||||
true,
|
||||
TimestampFormat.ISO_8601
|
||||
);
|
||||
String sourcePath = Objects.requireNonNull(Thread.currentThread()
|
||||
.getContextClassLoader().getResource("test_source.data")).toString();
|
||||
|
||||
DataStream<RowData> dataStream = execEnv
|
||||
// use continuous file source to trigger checkpoint
|
||||
.addSource(new ContinuousFileSource.BoundedSourceFunction(new Path(sourcePath), 2))
|
||||
.name("continuous_file_source")
|
||||
.setParallelism(1)
|
||||
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)))
|
||||
.setParallelism(4);
|
||||
|
||||
if (transformer != null) {
|
||||
dataStream = transformer.apply(dataStream);
|
||||
}
|
||||
|
||||
int parallelism = execEnv.getParallelism();
|
||||
DataStream<HoodieRecord> hoodieRecordDataStream = Pipelines.bootstrap(conf, rowType, parallelism, dataStream);
|
||||
DataStream<Object> pipeline = Pipelines.hoodieStreamWrite(conf, parallelism, hoodieRecordDataStream);
|
||||
execEnv.addOperator(pipeline.getTransformation());
|
||||
|
||||
JobClient client = execEnv.executeAsync(conf.getString(FlinkOptions.TABLE_NAME));
|
||||
// wait for the streaming job to finish
|
||||
client.getJobExecutionResult().get();
|
||||
|
||||
TestData.checkWrittenFullData(tempFile, expected);
|
||||
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,304 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadata;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
import org.apache.hudi.sink.utils.MockCoordinatorExecutor;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
import org.apache.hudi.utils.TestUtils;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.jobgraph.OperatorID;
|
||||
import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
|
||||
import org.apache.flink.util.FileUtils;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
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.hamcrest.CoreMatchers.instanceOf;
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Test cases for StreamingSinkOperatorCoordinator.
|
||||
*/
|
||||
public class TestStreamWriteOperatorCoordinator {
|
||||
private StreamWriteOperatorCoordinator coordinator;
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@BeforeEach
|
||||
public void before() throws Exception {
|
||||
OperatorCoordinator.Context context = new MockOperatorCoordinatorContext(new OperatorID(), 2);
|
||||
coordinator = new StreamWriteOperatorCoordinator(
|
||||
TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()), context);
|
||||
coordinator.start();
|
||||
coordinator.setExecutor(new MockCoordinatorExecutor(context));
|
||||
|
||||
coordinator.handleEventFromOperator(0, WriteMetadataEvent.emptyBootstrap(0));
|
||||
coordinator.handleEventFromOperator(1, WriteMetadataEvent.emptyBootstrap(1));
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void after() throws Exception {
|
||||
coordinator.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testInstantState() {
|
||||
String instant = coordinator.getInstant();
|
||||
assertNotEquals("", instant);
|
||||
|
||||
OperatorEvent event0 = createOperatorEvent(0, instant, "par1", true, 0.1);
|
||||
OperatorEvent event1 = createOperatorEvent(1, instant, "par2", false, 0.2);
|
||||
coordinator.handleEventFromOperator(0, event0);
|
||||
coordinator.handleEventFromOperator(1, event1);
|
||||
|
||||
coordinator.notifyCheckpointComplete(1);
|
||||
String inflight = TestUtils.getLastPendingInstant(tempFile.getAbsolutePath());
|
||||
String lastCompleted = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath());
|
||||
assertThat("Instant should be complete", lastCompleted, is(instant));
|
||||
assertNotEquals("", inflight, "Should start a new instant");
|
||||
assertNotEquals(instant, inflight, "Should start a new instant");
|
||||
}
|
||||
|
||||
@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(1, future.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReceiveInvalidEvent() {
|
||||
CompletableFuture<byte[]> future = new CompletableFuture<>();
|
||||
coordinator.checkpointCoordinator(1, future);
|
||||
OperatorEvent event = WriteMetadataEvent.builder()
|
||||
.taskID(0)
|
||||
.instantTime("abc")
|
||||
.writeStatus(Collections.emptyList())
|
||||
.build();
|
||||
|
||||
assertError(() -> coordinator.handleEventFromOperator(0, event),
|
||||
"Receive an unexpected event for instant abc from task 0");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckpointCompleteWithPartialEvents() {
|
||||
final CompletableFuture<byte[]> future = new CompletableFuture<>();
|
||||
coordinator.checkpointCoordinator(1, future);
|
||||
String instant = coordinator.getInstant();
|
||||
OperatorEvent event = WriteMetadataEvent.builder()
|
||||
.taskID(0)
|
||||
.instantTime(instant)
|
||||
.writeStatus(Collections.emptyList())
|
||||
.build();
|
||||
coordinator.handleEventFromOperator(0, event);
|
||||
|
||||
assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(1),
|
||||
"Returns early for empty write results");
|
||||
String lastCompleted = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath());
|
||||
assertNull(lastCompleted, "Returns early for empty write results");
|
||||
assertNull(coordinator.getEventBuffer()[0]);
|
||||
|
||||
OperatorEvent event1 = createOperatorEvent(1, instant, "par2", false, 0.2);
|
||||
coordinator.handleEventFromOperator(1, event1);
|
||||
assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(2),
|
||||
"Commits the instant with partial events anyway");
|
||||
lastCompleted = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath());
|
||||
assertThat("Commits the instant with partial events anyway", lastCompleted, is(instant));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHiveSyncInvoked() throws Exception {
|
||||
// reset
|
||||
reset();
|
||||
// override the default configuration
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setBoolean(FlinkOptions.HIVE_SYNC_ENABLED, true);
|
||||
OperatorCoordinator.Context context = new MockOperatorCoordinatorContext(new OperatorID(), 1);
|
||||
coordinator = new StreamWriteOperatorCoordinator(conf, context);
|
||||
coordinator.start();
|
||||
coordinator.setExecutor(new MockCoordinatorExecutor(context));
|
||||
|
||||
final WriteMetadataEvent event0 = WriteMetadataEvent.emptyBootstrap(0);
|
||||
|
||||
coordinator.handleEventFromOperator(0, event0);
|
||||
|
||||
String instant = mockWriteWithMetadata();
|
||||
assertNotEquals("", instant);
|
||||
|
||||
// never throw for hive synchronization now
|
||||
assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(1));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSyncMetadataTable() throws Exception {
|
||||
// reset
|
||||
reset();
|
||||
// override the default configuration
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setBoolean(FlinkOptions.METADATA_ENABLED, true);
|
||||
conf.setInteger(FlinkOptions.METADATA_COMPACTION_DELTA_COMMITS, 5);
|
||||
OperatorCoordinator.Context context = new MockOperatorCoordinatorContext(new OperatorID(), 1);
|
||||
coordinator = new StreamWriteOperatorCoordinator(conf, context);
|
||||
coordinator.start();
|
||||
coordinator.setExecutor(new MockCoordinatorExecutor(context));
|
||||
|
||||
final WriteMetadataEvent event0 = WriteMetadataEvent.emptyBootstrap(0);
|
||||
|
||||
coordinator.handleEventFromOperator(0, event0);
|
||||
|
||||
String instant = coordinator.getInstant();
|
||||
assertNotEquals("", instant);
|
||||
|
||||
final String metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(tempFile.getAbsolutePath());
|
||||
HoodieTableMetaClient metadataTableMetaClient = StreamerUtil.createMetaClient(metadataTableBasePath);
|
||||
HoodieTimeline completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants();
|
||||
assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(1L));
|
||||
assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP));
|
||||
|
||||
// test metadata table compaction
|
||||
// write another 3 commits
|
||||
for (int i = 1; i < 4; i++) {
|
||||
instant = mockWriteWithMetadata();
|
||||
metadataTableMetaClient.reloadActiveTimeline();
|
||||
completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants();
|
||||
assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(i + 1L));
|
||||
assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant));
|
||||
}
|
||||
// the 5th commit triggers the compaction
|
||||
instant = mockWriteWithMetadata();
|
||||
metadataTableMetaClient.reloadActiveTimeline();
|
||||
completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
|
||||
assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(6L));
|
||||
assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant + "001"));
|
||||
assertThat(completedTimeline.lastInstant().get().getAction(), is(HoodieTimeline.COMMIT_ACTION));
|
||||
// write another 2 commits
|
||||
for (int i = 6; i < 8; i++) {
|
||||
instant = mockWriteWithMetadata();
|
||||
metadataTableMetaClient.reloadActiveTimeline();
|
||||
completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants();
|
||||
assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(i + 1L));
|
||||
assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant));
|
||||
}
|
||||
|
||||
// write another commit to trigger clean
|
||||
instant = mockWriteWithMetadata();
|
||||
metadataTableMetaClient.reloadActiveTimeline();
|
||||
completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
|
||||
assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(10L));
|
||||
assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant + "002"));
|
||||
assertThat(completedTimeline.lastInstant().get().getAction(), is(HoodieTimeline.CLEAN_ACTION));
|
||||
|
||||
// write another commit
|
||||
mockWriteWithMetadata();
|
||||
// write another commit to trigger compaction
|
||||
instant = mockWriteWithMetadata();
|
||||
metadataTableMetaClient.reloadActiveTimeline();
|
||||
completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
|
||||
assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(13L));
|
||||
assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant + "001"));
|
||||
assertThat(completedTimeline.lastInstant().get().getAction(), is(HoodieTimeline.COMMIT_ACTION));
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
private String mockWriteWithMetadata() {
|
||||
final String instant = coordinator.getInstant();
|
||||
OperatorEvent event = createOperatorEvent(0, instant, "par1", true, 0.1);
|
||||
|
||||
coordinator.handleEventFromOperator(0, event);
|
||||
coordinator.notifyCheckpointComplete(0);
|
||||
return instant;
|
||||
}
|
||||
|
||||
private static WriteMetadataEvent createOperatorEvent(
|
||||
int taskId,
|
||||
String instant,
|
||||
String partitionPath,
|
||||
boolean trackSuccessRecords,
|
||||
double failureFraction) {
|
||||
final WriteStatus writeStatus = new WriteStatus(trackSuccessRecords, failureFraction);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
|
||||
HoodieWriteStat writeStat = new HoodieWriteStat();
|
||||
writeStat.setPartitionPath(partitionPath);
|
||||
writeStat.setFileId("fileId123");
|
||||
writeStat.setPath("path123");
|
||||
writeStat.setFileSizeInBytes(123);
|
||||
writeStat.setTotalWriteBytes(123);
|
||||
writeStat.setNumWrites(1);
|
||||
|
||||
writeStatus.setStat(writeStat);
|
||||
|
||||
return WriteMetadataEvent.builder()
|
||||
.taskID(taskId)
|
||||
.instantTime(instant)
|
||||
.writeStatus(Collections.singletonList(writeStatus))
|
||||
.lastBatch(true)
|
||||
.build();
|
||||
}
|
||||
|
||||
private void reset() throws Exception {
|
||||
FileUtils.cleanDirectory(tempFile);
|
||||
}
|
||||
|
||||
private void assertError(Runnable runnable, String message) {
|
||||
runnable.run();
|
||||
// wait a little while for the task to finish
|
||||
assertThat(coordinator.getContext(), instanceOf(MockOperatorCoordinatorContext.class));
|
||||
MockOperatorCoordinatorContext context = (MockOperatorCoordinatorContext) coordinator.getContext();
|
||||
assertTrue(context.isJobFailed(), message);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,432 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
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.sink.utils.TestWriteBase;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
import org.apache.hudi.utils.TestData;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
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.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertSame;
|
||||
|
||||
/**
|
||||
* Test cases for stream write.
|
||||
*/
|
||||
public class TestWriteCopyOnWrite extends TestWriteBase {
|
||||
|
||||
protected Configuration conf;
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@BeforeEach
|
||||
public void before() {
|
||||
conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, getTableType().name());
|
||||
setUp(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Override to have custom configuration.
|
||||
*/
|
||||
protected void setUp(Configuration conf) {
|
||||
// for sub-class extension
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckpoint() throws Exception {
|
||||
preparePipeline()
|
||||
.consume(TestData.DATA_SET_INSERT)
|
||||
// no checkpoint, so the coordinator does not accept any events
|
||||
.emptyEventBuffer()
|
||||
.checkpoint(1)
|
||||
.assertNextEvent(4, "par1,par2,par3,par4")
|
||||
.checkpointComplete(1)
|
||||
// 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
|
||||
.checkpoint(2)
|
||||
.assertEmptyEvent()
|
||||
.emptyCheckpoint(2)
|
||||
.end();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckpointFails() throws Exception {
|
||||
// reset the config option
|
||||
conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, 1L);
|
||||
preparePipeline(conf)
|
||||
// no data written and triggers checkpoint fails,
|
||||
// then we should revert the start instant
|
||||
.checkpoint(1)
|
||||
.assertEmptyEvent()
|
||||
.checkpointFails(1)
|
||||
.consume(TestData.DATA_SET_INSERT)
|
||||
//.checkpointThrows(2,
|
||||
// "Timeout(1000ms) while waiting for instant initialize")
|
||||
// do not send the write event and fails the checkpoint,
|
||||
// behaves like the last checkpoint is successful.
|
||||
.checkpointFails(2)
|
||||
.end();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSubtaskFails() throws Exception {
|
||||
// open the function and ingest data
|
||||
preparePipeline()
|
||||
.checkpoint(1)
|
||||
.assertEmptyEvent()
|
||||
.subTaskFails(0)
|
||||
.noCompleteInstant()
|
||||
.end();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsert() throws Exception {
|
||||
// open the function and ingest data
|
||||
preparePipeline()
|
||||
.consume(TestData.DATA_SET_INSERT)
|
||||
.assertEmptyDataFiles()
|
||||
.checkpoint(1)
|
||||
.assertNextEvent()
|
||||
.checkpointComplete(1)
|
||||
.checkWrittenData(EXPECTED1)
|
||||
.end();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertDuplicates() throws Exception {
|
||||
// reset the config option
|
||||
conf.setBoolean(FlinkOptions.PRE_COMBINE, true);
|
||||
preparePipeline(conf)
|
||||
.consume(TestData.DATA_SET_INSERT_DUPLICATES)
|
||||
.assertEmptyDataFiles()
|
||||
.checkpoint(1)
|
||||
.assertNextEvent()
|
||||
.checkpointComplete(1)
|
||||
.checkWrittenData(EXPECTED3, 1)
|
||||
// insert duplicates again
|
||||
.consume(TestData.DATA_SET_INSERT_DUPLICATES)
|
||||
.checkpoint(2)
|
||||
.assertNextEvent()
|
||||
.checkpointComplete(2)
|
||||
.checkWrittenData(EXPECTED3, 1)
|
||||
.end();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpsert() throws Exception {
|
||||
// open the function and ingest data
|
||||
preparePipeline()
|
||||
.consume(TestData.DATA_SET_INSERT)
|
||||
.assertEmptyDataFiles()
|
||||
.checkpoint(1)
|
||||
.assertNextEvent()
|
||||
.checkpointComplete(1)
|
||||
// upsert another data buffer
|
||||
.consume(TestData.DATA_SET_UPDATE_INSERT)
|
||||
// the data is not flushed yet
|
||||
.checkWrittenData(EXPECTED1)
|
||||
.checkpoint(2)
|
||||
.assertNextEvent()
|
||||
.checkpointComplete(2)
|
||||
.checkWrittenData(EXPECTED2)
|
||||
.end();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpsertWithDelete() throws Exception {
|
||||
// open the function and ingest data
|
||||
preparePipeline()
|
||||
.consume(TestData.DATA_SET_INSERT)
|
||||
.assertEmptyDataFiles()
|
||||
.checkpoint(1)
|
||||
.assertNextEvent()
|
||||
.checkpointComplete(1)
|
||||
.consume(TestData.DATA_SET_UPDATE_DELETE)
|
||||
.checkWrittenData(EXPECTED1)
|
||||
.checkpoint(2)
|
||||
.assertNextEvent()
|
||||
.checkpointComplete(2)
|
||||
.checkWrittenData(getUpsertWithDeleteExpected())
|
||||
.end();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertWithMiniBatches() throws Exception {
|
||||
// reset the config option
|
||||
conf.setDouble(FlinkOptions.WRITE_BATCH_SIZE, 0.0008); // 839 bytes batch size
|
||||
|
||||
Map<String, String> expected = getMiniBatchExpected();
|
||||
|
||||
preparePipeline(conf)
|
||||
// record (operation: 'I') is 304 bytes and record (operation: 'U') is 352 bytes.
|
||||
// so 3 records expect to trigger a mini-batch write
|
||||
.consume(TestData.DATA_SET_INSERT_DUPLICATES)
|
||||
.assertDataBuffer(1, 2)
|
||||
.checkpoint(1)
|
||||
.allDataFlushed()
|
||||
.handleEvents(2)
|
||||
.checkpointComplete(1)
|
||||
.checkWrittenData(expected, 1)
|
||||
.consume(TestData.DATA_SET_INSERT_DUPLICATES)
|
||||
.checkpoint(2)
|
||||
.handleEvents(2)
|
||||
.checkpointComplete(2)
|
||||
.checkWrittenData(expected, 1)
|
||||
.end();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertWithDeduplication() throws Exception {
|
||||
// reset the config option
|
||||
conf.setDouble(FlinkOptions.WRITE_BATCH_SIZE, 0.0008); // 839 bytes batch size
|
||||
conf.setBoolean(FlinkOptions.PRE_COMBINE, true);
|
||||
|
||||
Map<String, String> expected = new HashMap<>();
|
||||
expected.put("par1", "[id1,par1,id1,Danny,23,4,par1]");
|
||||
|
||||
preparePipeline(conf)
|
||||
// record (operation: 'I') is 304 bytes and record (operation: 'U') is 352 bytes.
|
||||
// so 3 records expect to trigger a mini-batch write
|
||||
.consume(TestData.DATA_SET_INSERT_SAME_KEY)
|
||||
.assertDataBuffer(1, 2)
|
||||
.checkpoint(1)
|
||||
.allDataFlushed()
|
||||
.handleEvents(2)
|
||||
.checkpointComplete(1)
|
||||
.checkWrittenData(expected, 1)
|
||||
.consume(TestData.DATA_SET_INSERT_SAME_KEY)
|
||||
.checkpoint(2)
|
||||
.handleEvents(2)
|
||||
.checkpointComplete(2)
|
||||
.checkWrittenData(expected, 1)
|
||||
.end();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertAppendMode() throws Exception {
|
||||
prepareInsertPipeline()
|
||||
// Each record is 208 bytes. so 4 records expect to trigger a mini-batch write
|
||||
.consume(TestData.DATA_SET_INSERT_SAME_KEY)
|
||||
.checkpoint(1)
|
||||
.assertNextEvent()
|
||||
.checkpointComplete(1)
|
||||
.checkWrittenAllData(EXPECTED4, 1)
|
||||
.consume(TestData.DATA_SET_INSERT_SAME_KEY)
|
||||
.checkpoint(2)
|
||||
.assertNextEvent()
|
||||
.checkpointComplete(2)
|
||||
.checkWrittenFullData(EXPECTED5)
|
||||
.end();
|
||||
}
|
||||
|
||||
/**
|
||||
* The test is almost same with {@link #testInsertWithSmallBufferSize} except that
|
||||
* it is with insert clustering mode.
|
||||
*/
|
||||
@Test
|
||||
public void testInsertClustering() throws Exception {
|
||||
// reset the config option
|
||||
conf.setString(FlinkOptions.OPERATION, "insert");
|
||||
conf.setBoolean(FlinkOptions.INSERT_CLUSTER, true);
|
||||
conf.setDouble(FlinkOptions.WRITE_TASK_MAX_SIZE, 200.0008); // 839 bytes buffer size
|
||||
|
||||
TestWriteMergeOnRead.TestHarness.instance()
|
||||
// record (operation: 'I') is 304 bytes and record (operation: 'U') is 352 bytes.
|
||||
// so 3 records expect to trigger a mini-batch write
|
||||
// flush the max size bucket once at a time.
|
||||
.preparePipeline(tempFile, conf)
|
||||
.consume(TestData.DATA_SET_INSERT_SAME_KEY)
|
||||
.assertDataBuffer(1, 2)
|
||||
.checkpoint(1)
|
||||
.allDataFlushed()
|
||||
.handleEvents(2)
|
||||
.checkpointComplete(1)
|
||||
.checkWrittenData(EXPECTED4, 1)
|
||||
// insert duplicates again
|
||||
.consume(TestData.DATA_SET_INSERT_SAME_KEY)
|
||||
.checkpoint(2)
|
||||
.handleEvents(2)
|
||||
.checkpointComplete(2)
|
||||
.checkWrittenFullData(EXPECTED5)
|
||||
.end();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertWithSmallBufferSize() throws Exception {
|
||||
// reset the config option
|
||||
conf.setDouble(FlinkOptions.WRITE_TASK_MAX_SIZE, 200.0008); // 839 bytes buffer size
|
||||
|
||||
Map<String, String> expected = getMiniBatchExpected();
|
||||
|
||||
preparePipeline(conf)
|
||||
// record (operation: 'I') is 304 bytes and record (operation: 'U') is 352 bytes.
|
||||
// so 3 records expect to trigger a mini-batch write
|
||||
// flush the max size bucket once at a time.
|
||||
.consume(TestData.DATA_SET_INSERT_DUPLICATES)
|
||||
.assertDataBuffer(1, 2)
|
||||
.checkpoint(1)
|
||||
.allDataFlushed()
|
||||
.handleEvents(2)
|
||||
.checkpointComplete(1)
|
||||
.checkWrittenData(expected, 1)
|
||||
// insert duplicates again
|
||||
.consume(TestData.DATA_SET_INSERT_DUPLICATES)
|
||||
.checkpoint(2)
|
||||
.handleEvents(2)
|
||||
.checkpointComplete(2)
|
||||
// Same the original base file content.
|
||||
.checkWrittenData(expected, 1)
|
||||
.end();
|
||||
}
|
||||
|
||||
protected Map<String, String> getMiniBatchExpected() {
|
||||
Map<String, String> expected = new HashMap<>();
|
||||
// the last 2 lines are merged
|
||||
expected.put("par1", "["
|
||||
+ "id1,par1,id1,Danny,23,1,par1, "
|
||||
+ "id1,par1,id1,Danny,23,1,par1, "
|
||||
+ "id1,par1,id1,Danny,23,1,par1" + "]");
|
||||
return expected;
|
||||
}
|
||||
|
||||
protected Map<String, String> getUpsertWithDeleteExpected() {
|
||||
Map<String, String> expected = new HashMap<>();
|
||||
// id3, id5 were deleted and id9 is ignored
|
||||
expected.put("par1", "[id1,par1,id1,Danny,24,1,par1, id2,par1,id2,Stephen,34,2,par1]");
|
||||
expected.put("par2", "[id4,par2,id4,Fabian,31,4,par2]");
|
||||
expected.put("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]");
|
||||
return expected;
|
||||
}
|
||||
|
||||
protected Map<String, String> getExpectedBeforeCheckpointComplete() {
|
||||
return EXPECTED2;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIndexStateBootstrap() throws Exception {
|
||||
// open the function and ingest data
|
||||
preparePipeline()
|
||||
.consume(TestData.DATA_SET_INSERT)
|
||||
.assertEmptyDataFiles()
|
||||
.checkpoint(1)
|
||||
.assertNextEvent()
|
||||
.checkpointComplete(1)
|
||||
.checkWrittenData(EXPECTED1, 4)
|
||||
.end();
|
||||
|
||||
// reset the config option
|
||||
conf.setBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED, true);
|
||||
preparePipeline(conf)
|
||||
.consume(TestData.DATA_SET_UPDATE_INSERT)
|
||||
.checkIndexLoaded(
|
||||
new HoodieKey("id1", "par1"),
|
||||
new HoodieKey("id2", "par1"),
|
||||
new HoodieKey("id3", "par2"),
|
||||
new HoodieKey("id4", "par2"),
|
||||
new HoodieKey("id5", "par3"),
|
||||
new HoodieKey("id6", "par3"),
|
||||
new HoodieKey("id7", "par4"),
|
||||
new HoodieKey("id8", "par4"),
|
||||
new HoodieKey("id9", "par3"),
|
||||
new HoodieKey("id10", "par4"),
|
||||
new HoodieKey("id11", "par4"))
|
||||
.checkpoint(1)
|
||||
.assertBootstrapped()
|
||||
.assertNextEvent()
|
||||
.checkWrittenData(getExpectedBeforeCheckpointComplete())
|
||||
.checkpointComplete(1)
|
||||
.checkWrittenData(EXPECTED2)
|
||||
.end();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteExactlyOnce() throws Exception {
|
||||
// reset the config option
|
||||
conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, 1L);
|
||||
conf.setDouble(FlinkOptions.WRITE_TASK_MAX_SIZE, 200.0006); // 630 bytes buffer size
|
||||
preparePipeline(conf)
|
||||
.consume(TestData.DATA_SET_INSERT)
|
||||
.emptyEventBuffer()
|
||||
.checkpoint(1)
|
||||
.assertConfirming()
|
||||
.handleEvents(4)
|
||||
.checkpointComplete(1)
|
||||
.consume(TestData.DATA_SET_INSERT)
|
||||
.assertNotConfirming()
|
||||
.checkpoint(2)
|
||||
.assertConsumeThrows(TestData.DATA_SET_INSERT,
|
||||
"Timeout(1000ms) while waiting for instant initialize")
|
||||
.end();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReuseEmbeddedServer() throws IOException {
|
||||
conf.setInteger("hoodie.filesystem.view.remote.timeout.secs", 500);
|
||||
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf);
|
||||
FileSystemViewStorageConfig viewStorageConfig = writeClient.getConfig().getViewStorageConfig();
|
||||
|
||||
assertSame(viewStorageConfig.getStorageType(), FileSystemViewStorageType.REMOTE_FIRST);
|
||||
|
||||
// get another write client
|
||||
writeClient = StreamerUtil.createWriteClient(conf);
|
||||
assertSame(writeClient.getConfig().getViewStorageConfig().getStorageType(), FileSystemViewStorageType.REMOTE_FIRST);
|
||||
assertEquals(viewStorageConfig.getRemoteViewServerPort(), writeClient.getConfig().getViewStorageConfig().getRemoteViewServerPort());
|
||||
assertEquals(viewStorageConfig.getRemoteTimelineClientTimeoutSecs(), 500);
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
private TestHarness preparePipeline() throws Exception {
|
||||
return TestHarness.instance().preparePipeline(tempFile, conf);
|
||||
}
|
||||
|
||||
private TestHarness preparePipeline(Configuration conf) throws Exception {
|
||||
return TestHarness.instance().preparePipeline(tempFile, conf);
|
||||
}
|
||||
|
||||
protected TestHarness prepareInsertPipeline() throws Exception {
|
||||
return TestHarness.instance().preparePipeline(tempFile, conf, true);
|
||||
}
|
||||
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.COPY_ON_WRITE;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,60 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Test cases for delta stream write.
|
||||
*/
|
||||
public class TestWriteMergeOnRead extends TestWriteCopyOnWrite {
|
||||
|
||||
@Override
|
||||
protected void setUp(Configuration conf) {
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void testInsertClustering() {
|
||||
// insert clustering is only valid for cow table.
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, String> getExpectedBeforeCheckpointComplete() {
|
||||
return EXPECTED1;
|
||||
}
|
||||
|
||||
protected 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 HoodieTableType getTableType() {
|
||||
return HoodieTableType.MERGE_ON_READ;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,61 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void testInsertClustering() {
|
||||
// insert clustering is only valid for cow table.
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, String> getExpectedBeforeCheckpointComplete() {
|
||||
return EXPECTED1;
|
||||
}
|
||||
|
||||
protected 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 HoodieTableType getTableType() {
|
||||
return HoodieTableType.MERGE_ON_READ;
|
||||
}
|
||||
}
|
||||
@@ -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.sink.bulk;
|
||||
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieKeyException;
|
||||
import org.apache.hudi.table.HoodieTableFactory;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.data.StringData;
|
||||
import org.apache.flink.table.data.TimestampData;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.apache.hudi.utils.TestData.insertRow;
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
/**
|
||||
* Test cases for {@link RowDataKeyGen}.
|
||||
*/
|
||||
public class TestRowDataKeyGen {
|
||||
@Test
|
||||
void testSimpleKeyAndPartition() {
|
||||
Configuration conf = TestConfigurations.getDefaultConf("path1");
|
||||
final RowData rowData1 = insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23,
|
||||
TimestampData.fromEpochMillis(1), StringData.fromString("par1"));
|
||||
final RowDataKeyGen keyGen1 = RowDataKeyGen.instance(conf, TestConfigurations.ROW_TYPE);
|
||||
assertThat(keyGen1.getRecordKey(rowData1), is("id1"));
|
||||
assertThat(keyGen1.getPartitionPath(rowData1), is("par1"));
|
||||
|
||||
// null record key and partition path
|
||||
final RowData rowData2 = insertRow(TestConfigurations.ROW_TYPE, null, StringData.fromString("Danny"), 23,
|
||||
TimestampData.fromEpochMillis(1), null);
|
||||
assertThrows(HoodieKeyException.class, () -> keyGen1.getRecordKey(rowData2));
|
||||
assertThat(keyGen1.getPartitionPath(rowData2), is("default"));
|
||||
// empty record key and partition path
|
||||
final RowData rowData3 = insertRow(StringData.fromString(""), StringData.fromString("Danny"), 23,
|
||||
TimestampData.fromEpochMillis(1), StringData.fromString(""));
|
||||
assertThrows(HoodieKeyException.class, () -> keyGen1.getRecordKey(rowData3));
|
||||
assertThat(keyGen1.getPartitionPath(rowData3), is("default"));
|
||||
|
||||
// hive style partitioning
|
||||
conf.set(FlinkOptions.HIVE_STYLE_PARTITIONING, true);
|
||||
final RowDataKeyGen keyGen2 = RowDataKeyGen.instance(conf, TestConfigurations.ROW_TYPE);
|
||||
assertThat(keyGen2.getPartitionPath(rowData1), is("partition=par1"));
|
||||
assertThat(keyGen2.getPartitionPath(rowData2), is("partition=default"));
|
||||
assertThat(keyGen2.getPartitionPath(rowData3), is("partition=default"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testComplexKeyAndPartition() {
|
||||
Configuration conf = TestConfigurations.getDefaultConf("path1");
|
||||
conf.set(FlinkOptions.RECORD_KEY_FIELD, "uuid,name");
|
||||
conf.set(FlinkOptions.PARTITION_PATH_FIELD, "partition,ts");
|
||||
RowData rowData1 = insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23,
|
||||
TimestampData.fromEpochMillis(1), StringData.fromString("par1"));
|
||||
RowDataKeyGen keyGen1 = RowDataKeyGen.instance(conf, TestConfigurations.ROW_TYPE);
|
||||
assertThat(keyGen1.getRecordKey(rowData1), is("uuid:id1,name:Danny"));
|
||||
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);
|
||||
assertThrows(HoodieKeyException.class, () -> keyGen1.getRecordKey(rowData2));
|
||||
assertThat(keyGen1.getPartitionPath(rowData2), is("default/default"));
|
||||
// empty record key and partition path
|
||||
final RowData rowData3 = insertRow(StringData.fromString(""), StringData.fromString(""), 23,
|
||||
TimestampData.fromEpochMillis(1), StringData.fromString(""));
|
||||
assertThrows(HoodieKeyException.class, () -> keyGen1.getRecordKey(rowData3));
|
||||
assertThat(keyGen1.getPartitionPath(rowData3), is("default/1970-01-01T00:00:00.001"));
|
||||
|
||||
// hive style partitioning
|
||||
conf.set(FlinkOptions.HIVE_STYLE_PARTITIONING, true);
|
||||
final RowDataKeyGen keyGen2 = RowDataKeyGen.instance(conf, TestConfigurations.ROW_TYPE);
|
||||
assertThat(keyGen2.getPartitionPath(rowData1), is("partition=par1/ts=1970-01-01T00:00:00.001"));
|
||||
assertThat(keyGen2.getPartitionPath(rowData2), is("partition=default/ts=default"));
|
||||
assertThat(keyGen2.getPartitionPath(rowData3), is("partition=default/ts=1970-01-01T00:00:00.001"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testTimestampBasedKeyGenerator() {
|
||||
Configuration conf = TestConfigurations.getDefaultConf("path1");
|
||||
conf.setString(FlinkOptions.PARTITION_PATH_FIELD, "ts");
|
||||
HoodieTableFactory.setupTimestampKeygenOptions(conf, DataTypes.TIMESTAMP(3));
|
||||
final RowData rowData1 = insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23,
|
||||
TimestampData.fromEpochMillis(7200000), StringData.fromString("par1"));
|
||||
final RowDataKeyGen keyGen1 = RowDataKeyGen.instance(conf, TestConfigurations.ROW_TYPE);
|
||||
|
||||
assertThat(keyGen1.getRecordKey(rowData1), is("id1"));
|
||||
assertThat(keyGen1.getPartitionPath(rowData1), is("1970010102"));
|
||||
|
||||
// null record key and partition path
|
||||
final RowData rowData2 = insertRow(TestConfigurations.ROW_TYPE, null, StringData.fromString("Danny"), 23,
|
||||
null, StringData.fromString("par1"));
|
||||
assertThrows(HoodieKeyException.class, () -> keyGen1.getRecordKey(rowData2));
|
||||
assertThat(keyGen1.getPartitionPath(rowData2), is("1970010100"));
|
||||
// empty record key and partition path
|
||||
final RowData rowData3 = insertRow(StringData.fromString(""), StringData.fromString("Danny"), 23,
|
||||
TimestampData.fromEpochMillis(1), StringData.fromString("par1"));
|
||||
assertThrows(HoodieKeyException.class, () -> keyGen1.getRecordKey(rowData3));
|
||||
assertThat(keyGen1.getPartitionPath(rowData3), is("1970010100"));
|
||||
|
||||
// hive style partitioning
|
||||
conf.set(FlinkOptions.HIVE_STYLE_PARTITIONING, true);
|
||||
final RowDataKeyGen keyGen2 = RowDataKeyGen.instance(conf, TestConfigurations.ROW_TYPE);
|
||||
assertThat(keyGen2.getPartitionPath(rowData1), is("ts=1970010102"));
|
||||
assertThat(keyGen2.getPartitionPath(rowData2), is("ts=1970010100"));
|
||||
assertThat(keyGen2.getPartitionPath(rowData3), is("ts=1970010100"));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,206 @@
|
||||
/*
|
||||
* 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.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
import org.apache.hudi.util.CompactionUtil;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
import org.apache.hudi.utils.TestData;
|
||||
import org.apache.hudi.utils.TestSQL;
|
||||
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||||
import org.apache.flink.streaming.api.operators.ProcessOperator;
|
||||
import org.apache.flink.table.api.EnvironmentSettings;
|
||||
import org.apache.flink.table.api.TableEnvironment;
|
||||
import org.apache.flink.table.api.config.ExecutionConfigOptions;
|
||||
import org.apache.flink.table.api.internal.TableEnvironmentImpl;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* IT cases for {@link org.apache.hudi.common.model.HoodieRecord}.
|
||||
*/
|
||||
public class ITTestHoodieFlinkCompactor {
|
||||
|
||||
protected static final Logger LOG = LoggerFactory.getLogger(ITTestHoodieFlinkCompactor.class);
|
||||
|
||||
private static final Map<String, List<String>> EXPECTED1 = new HashMap<>();
|
||||
|
||||
private static final Map<String, List<String>> EXPECTED2 = new HashMap<>();
|
||||
|
||||
static {
|
||||
EXPECTED1.put("par1", Arrays.asList("id1,par1,id1,Danny,23,1000,par1", "id2,par1,id2,Stephen,33,2000,par1"));
|
||||
EXPECTED1.put("par2", Arrays.asList("id3,par2,id3,Julian,53,3000,par2", "id4,par2,id4,Fabian,31,4000,par2"));
|
||||
EXPECTED1.put("par3", Arrays.asList("id5,par3,id5,Sophia,18,5000,par3", "id6,par3,id6,Emma,20,6000,par3"));
|
||||
EXPECTED1.put("par4", Arrays.asList("id7,par4,id7,Bob,44,7000,par4", "id8,par4,id8,Han,56,8000,par4"));
|
||||
|
||||
EXPECTED2.put("par1", Arrays.asList("id1,par1,id1,Danny,24,1000,par1", "id2,par1,id2,Stephen,34,2000,par1"));
|
||||
EXPECTED2.put("par2", Arrays.asList("id3,par2,id3,Julian,54,3000,par2", "id4,par2,id4,Fabian,32,4000,par2"));
|
||||
EXPECTED2.put("par3", Arrays.asList("id5,par3,id5,Sophia,18,5000,par3", "id6,par3,id6,Emma,20,6000,par3", "id9,par3,id9,Jane,19,6000,par3"));
|
||||
EXPECTED2.put("par4", Arrays.asList("id7,par4,id7,Bob,44,7000,par4", "id8,par4,id8,Han,56,8000,par4", "id10,par4,id10,Ella,38,7000,par4", "id11,par4,id11,Phoebe,52,8000,par4"));
|
||||
}
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testHoodieFlinkCompactor(boolean enableChangelog) throws Exception {
|
||||
// Create hoodie table and insert into data.
|
||||
EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
|
||||
TableEnvironment tableEnv = TableEnvironmentImpl.create(settings);
|
||||
tableEnv.getConfig().getConfiguration()
|
||||
.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.COMPACTION_ASYNC_ENABLED.key(), "false");
|
||||
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
|
||||
options.put(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ");
|
||||
options.put(FlinkOptions.CHANGELOG_ENABLED.key(), enableChangelog + "");
|
||||
String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options);
|
||||
tableEnv.executeSql(hoodieTableDDL);
|
||||
tableEnv.executeSql(TestSQL.INSERT_T1).await();
|
||||
|
||||
// wait for the asynchronous commit to finish
|
||||
TimeUnit.SECONDS.sleep(3);
|
||||
|
||||
// Make configuration and setAvroSchema.
|
||||
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
|
||||
FlinkCompactionConfig cfg = new FlinkCompactionConfig();
|
||||
cfg.path = tempFile.getAbsolutePath();
|
||||
Configuration conf = FlinkCompactionConfig.toFlinkConfig(cfg);
|
||||
conf.setString(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ");
|
||||
|
||||
// create metaClient
|
||||
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf);
|
||||
|
||||
// set the table name
|
||||
conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName());
|
||||
|
||||
// set table schema
|
||||
CompactionUtil.setAvroSchema(conf, metaClient);
|
||||
|
||||
// infer changelog mode
|
||||
CompactionUtil.inferChangelogMode(conf, metaClient);
|
||||
|
||||
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf);
|
||||
|
||||
boolean scheduled = false;
|
||||
// judge whether have operation
|
||||
// To compute the compaction instant time and do compaction.
|
||||
Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient);
|
||||
if (compactionInstantTimeOption.isPresent()) {
|
||||
scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty());
|
||||
}
|
||||
|
||||
String compactionInstantTime = compactionInstantTimeOption.get();
|
||||
|
||||
assertTrue(scheduled, "The compaction plan should be scheduled");
|
||||
|
||||
HoodieFlinkTable<?> table = writeClient.getHoodieTable();
|
||||
// generate compaction plan
|
||||
// should support configurable commit metadata
|
||||
HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(
|
||||
table.getMetaClient(), compactionInstantTime);
|
||||
|
||||
HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
|
||||
// Mark instant as compaction inflight
|
||||
table.getActiveTimeline().transitionCompactionRequestedToInflight(instant);
|
||||
|
||||
env.addSource(new CompactionPlanSourceFunction(compactionPlan, compactionInstantTime))
|
||||
.name("compaction_source")
|
||||
.uid("uid_compaction_source")
|
||||
.rebalance()
|
||||
.transform("compact_task",
|
||||
TypeInformation.of(CompactionCommitEvent.class),
|
||||
new ProcessOperator<>(new CompactFunction(conf)))
|
||||
.setParallelism(compactionPlan.getOperations().size())
|
||||
.addSink(new CompactionCommitSink(conf))
|
||||
.name("clean_commits")
|
||||
.uid("uid_clean_commits")
|
||||
.setParallelism(1);
|
||||
|
||||
env.execute("flink_hudi_compaction");
|
||||
writeClient.close();
|
||||
TestData.checkWrittenFullData(tempFile, EXPECTED1);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testHoodieFlinkCompactorService(boolean enableChangelog) throws Exception {
|
||||
// Create hoodie table and insert into data.
|
||||
EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
|
||||
TableEnvironment tableEnv = TableEnvironmentImpl.create(settings);
|
||||
tableEnv.getConfig().getConfiguration()
|
||||
.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.COMPACTION_ASYNC_ENABLED.key(), "false");
|
||||
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
|
||||
options.put(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ");
|
||||
options.put(FlinkOptions.CHANGELOG_ENABLED.key(), enableChangelog + "");
|
||||
String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options);
|
||||
tableEnv.executeSql(hoodieTableDDL);
|
||||
|
||||
// insert dataset
|
||||
tableEnv.executeSql(TestSQL.INSERT_T1).await();
|
||||
// update the dataset
|
||||
tableEnv.executeSql(TestSQL.UPDATE_INSERT_T1).await();
|
||||
|
||||
// Make configuration and setAvroSchema.
|
||||
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
|
||||
FlinkCompactionConfig cfg = new FlinkCompactionConfig();
|
||||
cfg.path = tempFile.getAbsolutePath();
|
||||
cfg.minCompactionIntervalSeconds = 3;
|
||||
cfg.schedule = true;
|
||||
Configuration conf = FlinkCompactionConfig.toFlinkConfig(cfg);
|
||||
conf.setString(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ");
|
||||
|
||||
HoodieFlinkCompactor.AsyncCompactionService asyncCompactionService = new HoodieFlinkCompactor.AsyncCompactionService(cfg, conf, env);
|
||||
asyncCompactionService.start(null);
|
||||
|
||||
// wait for the asynchronous commit to finish
|
||||
TimeUnit.SECONDS.sleep(5);
|
||||
|
||||
asyncCompactionService.shutDown();
|
||||
|
||||
TestData.checkWrittenFullData(tempFile, EXPECTED2);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,75 @@
|
||||
/*
|
||||
* 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.meta;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
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.stream.IntStream;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
|
||||
/**
|
||||
* Test cases for {@link CkpMetadata}.
|
||||
*/
|
||||
public class TestCkpMetadata {
|
||||
|
||||
private CkpMetadata metadata;
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@BeforeEach
|
||||
public void beforeEach() throws Exception {
|
||||
String basePath = tempFile.getAbsolutePath();
|
||||
FileSystem fs = FSUtils.getFs(tempFile.getAbsolutePath(), StreamerUtil.getHadoopConf());
|
||||
|
||||
Configuration conf = TestConfigurations.getDefaultConf(basePath);
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
|
||||
this.metadata = CkpMetadata.getInstance(fs, basePath);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testWriteAndReadMessage() {
|
||||
// write and read 5 committed checkpoints
|
||||
IntStream.range(0, 3).forEach(i -> metadata.startInstant(i + ""));
|
||||
|
||||
assertThat(metadata.lastPendingInstant(), is("2"));
|
||||
metadata.commitInstant("2");
|
||||
assertThat(metadata.lastPendingInstant(), is("1"));
|
||||
|
||||
// test cleaning
|
||||
IntStream.range(3, 6).forEach(i -> metadata.startInstant(i + ""));
|
||||
assertThat(metadata.getMessages().size(), is(3));
|
||||
// commit and abort instant does not trigger cleaning
|
||||
metadata.commitInstant("6");
|
||||
metadata.abortInstant("7");
|
||||
assertThat(metadata.getMessages().size(), is(5));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,471 @@
|
||||
/*
|
||||
* 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.partitioner;
|
||||
|
||||
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.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.sink.partitioner.profile.WriteProfile;
|
||||
import org.apache.hudi.table.action.commit.BucketInfo;
|
||||
import org.apache.hudi.table.action.commit.BucketType;
|
||||
import org.apache.hudi.table.action.commit.SmallFile;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
import org.apache.hudi.utils.TestData;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
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.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
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.assertTrue;
|
||||
|
||||
/**
|
||||
* Test cases for {@link BucketAssigner}.
|
||||
*/
|
||||
public class TestBucketAssigner {
|
||||
private HoodieWriteConfig writeConfig;
|
||||
private HoodieFlinkEngineContext context;
|
||||
private Configuration conf;
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@BeforeEach
|
||||
public void before() throws IOException {
|
||||
final String basePath = tempFile.getAbsolutePath();
|
||||
conf = TestConfigurations.getDefaultConf(basePath);
|
||||
|
||||
writeConfig = StreamerUtil.getHoodieClientConfig(conf);
|
||||
context = new HoodieFlinkEngineContext(
|
||||
new SerializableConfiguration(StreamerUtil.getHadoopConf()),
|
||||
new FlinkTaskContextSupplier(null));
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that the file ids generated by the task can finally shuffled to itself.
|
||||
*/
|
||||
@Test
|
||||
void testSmallFilesOfThisTask() {
|
||||
MockBucketAssigner mockBucketAssigner1 = new MockBucketAssigner(context, writeConfig);
|
||||
String fileId1 = mockBucketAssigner1.createFileIdOfThisTask();
|
||||
SmallFile smallFile1 = new SmallFile();
|
||||
smallFile1.location = new HoodieRecordLocation("t0", fileId1);
|
||||
smallFile1.sizeBytes = 123;
|
||||
List<SmallFile> smallFiles1 = mockBucketAssigner1.smallFilesOfThisTask(Collections.singletonList(smallFile1));
|
||||
assertThat(smallFiles1.size(), is(1));
|
||||
|
||||
// modify the parallelism and test again
|
||||
MockBucketAssigner mockBucketAssigner2 = new MockBucketAssigner(123, 200, context, writeConfig, Collections.emptyMap());
|
||||
String fileId2 = mockBucketAssigner2.createFileIdOfThisTask();
|
||||
SmallFile smallFile2 = new SmallFile();
|
||||
smallFile2.location = new HoodieRecordLocation("t0", fileId2);
|
||||
smallFile2.sizeBytes = 123;
|
||||
|
||||
String fileId3 = mockBucketAssigner2.createFileIdOfThisTask();
|
||||
SmallFile smallFile3 = new SmallFile();
|
||||
smallFile3.location = new HoodieRecordLocation("t0", fileId3);
|
||||
smallFile3.sizeBytes = 456;
|
||||
|
||||
List<SmallFile> smallFiles2 = mockBucketAssigner1.smallFilesOfThisTask(Arrays.asList(smallFile2, smallFile3));
|
||||
assertThat(smallFiles2.size(), is(2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddUpdate() {
|
||||
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(context, writeConfig);
|
||||
BucketInfo bucketInfo = mockBucketAssigner.addUpdate("par1", "file_id_0");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "file_id_0");
|
||||
|
||||
mockBucketAssigner.addUpdate("par1", "file_id_0");
|
||||
bucketInfo = mockBucketAssigner.addUpdate("par1", "file_id_0");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "file_id_0");
|
||||
|
||||
mockBucketAssigner.addUpdate("par1", "file_id_1");
|
||||
bucketInfo = mockBucketAssigner.addUpdate("par1", "file_id_1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "file_id_1");
|
||||
|
||||
bucketInfo = mockBucketAssigner.addUpdate("par2", "file_id_0");
|
||||
assertBucketEquals(bucketInfo, "par2", BucketType.UPDATE, "file_id_0");
|
||||
|
||||
bucketInfo = mockBucketAssigner.addUpdate("par3", "file_id_2");
|
||||
assertBucketEquals(bucketInfo, "par3", BucketType.UPDATE, "file_id_2");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddInsert() {
|
||||
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(context, writeConfig);
|
||||
BucketInfo bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.INSERT);
|
||||
|
||||
mockBucketAssigner.addInsert("par1");
|
||||
bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.INSERT);
|
||||
|
||||
mockBucketAssigner.addInsert("par2");
|
||||
bucketInfo = mockBucketAssigner.addInsert("par2");
|
||||
assertBucketEquals(bucketInfo, "par2", BucketType.INSERT);
|
||||
|
||||
bucketInfo = mockBucketAssigner.addInsert("par3");
|
||||
assertBucketEquals(bucketInfo, "par3", BucketType.INSERT);
|
||||
|
||||
bucketInfo = mockBucketAssigner.addInsert("par3");
|
||||
assertBucketEquals(bucketInfo, "par3", BucketType.INSERT);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertOverBucketAssigned() {
|
||||
conf.setInteger(HoodieCompactionConfig.COPY_ON_WRITE_INSERT_SPLIT_SIZE.key(), 2);
|
||||
writeConfig = StreamerUtil.getHoodieClientConfig(conf);
|
||||
|
||||
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(context, writeConfig);
|
||||
BucketInfo bucketInfo1 = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo1, "par1", BucketType.INSERT);
|
||||
|
||||
BucketInfo bucketInfo2 = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.INSERT);
|
||||
|
||||
assertEquals(bucketInfo1, bucketInfo2);
|
||||
|
||||
BucketInfo bucketInfo3 = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo3, "par1", BucketType.INSERT);
|
||||
|
||||
assertNotEquals(bucketInfo1, bucketInfo3);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertWithSmallFiles() {
|
||||
SmallFile f0 = new SmallFile();
|
||||
f0.location = new HoodieRecordLocation("t0", "f0");
|
||||
f0.sizeBytes = 12;
|
||||
|
||||
SmallFile f1 = new SmallFile();
|
||||
f1.location = new HoodieRecordLocation("t0", "f1");
|
||||
f1.sizeBytes = 122879; // no left space to append new records to this bucket
|
||||
|
||||
SmallFile f2 = new SmallFile();
|
||||
f2.location = new HoodieRecordLocation("t0", "f2");
|
||||
f2.sizeBytes = 56;
|
||||
|
||||
Map<String, List<SmallFile>> smallFilesMap = new HashMap<>();
|
||||
smallFilesMap.put("par1", Arrays.asList(f0, f1));
|
||||
smallFilesMap.put("par2", Collections.singletonList(f2));
|
||||
|
||||
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(context, writeConfig, smallFilesMap);
|
||||
BucketInfo bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
|
||||
|
||||
mockBucketAssigner.addInsert("par1");
|
||||
bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
|
||||
|
||||
mockBucketAssigner.addInsert("par2");
|
||||
bucketInfo = mockBucketAssigner.addInsert("par2");
|
||||
assertBucketEquals(bucketInfo, "par2", BucketType.UPDATE, "f2");
|
||||
|
||||
bucketInfo = mockBucketAssigner.addInsert("par3");
|
||||
assertBucketEquals(bucketInfo, "par3", BucketType.INSERT);
|
||||
|
||||
bucketInfo = mockBucketAssigner.addInsert("par3");
|
||||
assertBucketEquals(bucketInfo, "par3", BucketType.INSERT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that only partial small files are assigned to the task.
|
||||
*/
|
||||
@Test
|
||||
public void testInsertWithPartialSmallFiles() {
|
||||
SmallFile f0 = new SmallFile();
|
||||
f0.location = new HoodieRecordLocation("t0", "f0");
|
||||
f0.sizeBytes = 12;
|
||||
|
||||
SmallFile f1 = new SmallFile();
|
||||
f1.location = new HoodieRecordLocation("t0", "f1");
|
||||
f1.sizeBytes = 122879; // no left space to append new records to this bucket
|
||||
|
||||
SmallFile f2 = new SmallFile();
|
||||
f2.location = new HoodieRecordLocation("t0", "f2");
|
||||
f2.sizeBytes = 56;
|
||||
|
||||
Map<String, List<SmallFile>> smallFilesMap = new HashMap<>();
|
||||
smallFilesMap.put("par1", Arrays.asList(f0, f1, f2));
|
||||
|
||||
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(0, 2, context, writeConfig, smallFilesMap);
|
||||
BucketInfo bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f2");
|
||||
|
||||
mockBucketAssigner.addInsert("par1");
|
||||
bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f2");
|
||||
|
||||
bucketInfo = mockBucketAssigner.addInsert("par3");
|
||||
assertBucketEquals(bucketInfo, "par3", BucketType.INSERT);
|
||||
|
||||
bucketInfo = mockBucketAssigner.addInsert("par3");
|
||||
assertBucketEquals(bucketInfo, "par3", BucketType.INSERT);
|
||||
|
||||
MockBucketAssigner mockBucketAssigner2 = new MockBucketAssigner(1, 2, context, writeConfig, smallFilesMap);
|
||||
BucketInfo bucketInfo2 = mockBucketAssigner2.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f0");
|
||||
|
||||
mockBucketAssigner2.addInsert("par1");
|
||||
bucketInfo2 = mockBucketAssigner2.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f0");
|
||||
|
||||
bucketInfo2 = mockBucketAssigner2.addInsert("par3");
|
||||
assertBucketEquals(bucketInfo2, "par3", BucketType.INSERT);
|
||||
|
||||
bucketInfo2 = mockBucketAssigner2.addInsert("par3");
|
||||
assertBucketEquals(bucketInfo2, "par3", BucketType.INSERT);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateAndInsertWithSmallFiles() {
|
||||
SmallFile f0 = new SmallFile();
|
||||
f0.location = new HoodieRecordLocation("t0", "f0");
|
||||
f0.sizeBytes = 12;
|
||||
|
||||
SmallFile f1 = new SmallFile();
|
||||
f1.location = new HoodieRecordLocation("t0", "f1");
|
||||
f1.sizeBytes = 122879; // no left space to append new records to this bucket
|
||||
|
||||
SmallFile f2 = new SmallFile();
|
||||
f2.location = new HoodieRecordLocation("t0", "f2");
|
||||
f2.sizeBytes = 56;
|
||||
|
||||
Map<String, List<SmallFile>> smallFilesMap = new HashMap<>();
|
||||
smallFilesMap.put("par1", Arrays.asList(f0, f1));
|
||||
smallFilesMap.put("par2", Collections.singletonList(f2));
|
||||
|
||||
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(context, writeConfig, smallFilesMap);
|
||||
mockBucketAssigner.addUpdate("par1", "f0");
|
||||
|
||||
BucketInfo bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
|
||||
|
||||
mockBucketAssigner.addInsert("par1");
|
||||
bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
|
||||
|
||||
mockBucketAssigner.addUpdate("par1", "f2");
|
||||
|
||||
mockBucketAssigner.addInsert("par1");
|
||||
bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
|
||||
|
||||
mockBucketAssigner.addUpdate("par2", "f0");
|
||||
|
||||
mockBucketAssigner.addInsert("par2");
|
||||
bucketInfo = mockBucketAssigner.addInsert("par2");
|
||||
assertBucketEquals(bucketInfo, "par2", BucketType.UPDATE, "f2");
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that only partial small files are assigned to the task.
|
||||
*/
|
||||
@Test
|
||||
public void testUpdateAndInsertWithPartialSmallFiles() {
|
||||
SmallFile f0 = new SmallFile();
|
||||
f0.location = new HoodieRecordLocation("t0", "f0");
|
||||
f0.sizeBytes = 12;
|
||||
|
||||
SmallFile f1 = new SmallFile();
|
||||
f1.location = new HoodieRecordLocation("t0", "f1");
|
||||
f1.sizeBytes = 122879; // no left space to append new records to this bucket
|
||||
|
||||
SmallFile f2 = new SmallFile();
|
||||
f2.location = new HoodieRecordLocation("t0", "f2");
|
||||
f2.sizeBytes = 56;
|
||||
|
||||
Map<String, List<SmallFile>> smallFilesMap = new HashMap<>();
|
||||
smallFilesMap.put("par1", Arrays.asList(f0, f1, f2));
|
||||
|
||||
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(0, 2, context, writeConfig, smallFilesMap);
|
||||
mockBucketAssigner.addUpdate("par1", "f0");
|
||||
|
||||
BucketInfo bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f2");
|
||||
|
||||
mockBucketAssigner.addInsert("par1");
|
||||
bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f2");
|
||||
|
||||
mockBucketAssigner.addUpdate("par1", "f2");
|
||||
|
||||
mockBucketAssigner.addInsert("par1");
|
||||
bucketInfo = mockBucketAssigner.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f2");
|
||||
|
||||
|
||||
MockBucketAssigner mockBucketAssigner2 = new MockBucketAssigner(1, 2, context, writeConfig, smallFilesMap);
|
||||
mockBucketAssigner2.addUpdate("par1", "f0");
|
||||
|
||||
BucketInfo bucketInfo2 = mockBucketAssigner2.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f0");
|
||||
|
||||
mockBucketAssigner2.addInsert("par1");
|
||||
bucketInfo2 = mockBucketAssigner2.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f0");
|
||||
|
||||
mockBucketAssigner2.addUpdate("par1", "f2");
|
||||
|
||||
mockBucketAssigner2.addInsert("par1");
|
||||
bucketInfo2 = mockBucketAssigner2.addInsert("par1");
|
||||
assertBucketEquals(bucketInfo2, "par1", BucketType.UPDATE, "f0");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteProfileReload() throws Exception {
|
||||
WriteProfile writeProfile = new WriteProfile(writeConfig, context);
|
||||
List<SmallFile> smallFiles1 = writeProfile.getSmallFiles("par1");
|
||||
assertTrue(smallFiles1.isEmpty(), "Should have no small files");
|
||||
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
String instantOption = getLastCompleteInstant(writeProfile);
|
||||
assertNull(instantOption);
|
||||
|
||||
writeProfile.reload(1);
|
||||
String instant1 = getLastCompleteInstant(writeProfile);
|
||||
assertNotNull(instant1);
|
||||
List<SmallFile> smallFiles2 = writeProfile.getSmallFiles("par1");
|
||||
assertThat("Should have 1 small file", smallFiles2.size(), is(1));
|
||||
assertThat("Small file should have same timestamp as last complete instant",
|
||||
smallFiles2.get(0).location.getInstantTime(), is(instant1));
|
||||
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
List<SmallFile> smallFiles3 = writeProfile.getSmallFiles("par1");
|
||||
assertThat("Should have 1 small file", smallFiles3.size(), is(1));
|
||||
assertThat("Non-reloaded write profile has the same base file view as before",
|
||||
smallFiles3.get(0).location.getInstantTime(), is(instant1));
|
||||
|
||||
writeProfile.reload(2);
|
||||
String instant2 = getLastCompleteInstant(writeProfile);
|
||||
assertNotEquals(instant2, instant1, "Should have new complete instant");
|
||||
List<SmallFile> smallFiles4 = writeProfile.getSmallFiles("par1");
|
||||
assertThat("Should have 1 small file", smallFiles4.size(), is(1));
|
||||
assertThat("Small file should have same timestamp as last complete instant",
|
||||
smallFiles4.get(0).location.getInstantTime(), is(instant2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteProfileMetadataCache() throws Exception {
|
||||
WriteProfile writeProfile = new WriteProfile(writeConfig, context);
|
||||
assertTrue(writeProfile.getMetadataCache().isEmpty(), "Empty table should no have any instant metadata");
|
||||
|
||||
// write 3 instants of data
|
||||
for (int i = 0; i < 3; i++) {
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
}
|
||||
// the record profile triggers the metadata loading
|
||||
writeProfile.reload(1);
|
||||
assertThat("Metadata cache should have same number entries as timeline instants",
|
||||
writeProfile.getMetadataCache().size(), is(3));
|
||||
|
||||
writeProfile.getSmallFiles("par1");
|
||||
assertThat("The metadata should be reused",
|
||||
writeProfile.getMetadataCache().size(), is(3));
|
||||
}
|
||||
|
||||
private static String getLastCompleteInstant(WriteProfile profile) {
|
||||
return StreamerUtil.getLastCompletedInstant(profile.getMetaClient());
|
||||
}
|
||||
|
||||
private void assertBucketEquals(
|
||||
BucketInfo bucketInfo,
|
||||
String partition,
|
||||
BucketType bucketType,
|
||||
String fileId) {
|
||||
BucketInfo actual = new BucketInfo(bucketType, fileId, partition);
|
||||
assertThat(bucketInfo, is(actual));
|
||||
}
|
||||
|
||||
private void assertBucketEquals(
|
||||
BucketInfo bucketInfo,
|
||||
String partition,
|
||||
BucketType bucketType) {
|
||||
assertThat(bucketInfo.getPartitionPath(), is(partition));
|
||||
assertThat(bucketInfo.getBucketType(), is(bucketType));
|
||||
}
|
||||
|
||||
/**
|
||||
* Mock BucketAssigner that can specify small files explicitly.
|
||||
*/
|
||||
static class MockBucketAssigner extends BucketAssigner {
|
||||
|
||||
MockBucketAssigner(
|
||||
HoodieFlinkEngineContext context,
|
||||
HoodieWriteConfig config) {
|
||||
this(context, config, Collections.emptyMap());
|
||||
}
|
||||
|
||||
MockBucketAssigner(
|
||||
HoodieFlinkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
Map<String, List<SmallFile>> smallFilesMap) {
|
||||
this(0, 1, context, config, smallFilesMap);
|
||||
}
|
||||
|
||||
MockBucketAssigner(
|
||||
int taskID,
|
||||
int numTasks,
|
||||
HoodieFlinkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
Map<String, List<SmallFile>> smallFilesMap) {
|
||||
super(taskID, 1024, numTasks, new MockWriteProfile(config, context, smallFilesMap), config);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Mock WriteProfile that can specify small files explicitly.
|
||||
*/
|
||||
static class MockWriteProfile extends WriteProfile {
|
||||
private final Map<String, List<SmallFile>> smallFilesMap;
|
||||
|
||||
public MockWriteProfile(HoodieWriteConfig config, HoodieFlinkEngineContext context, Map<String, List<SmallFile>> smallFilesMap) {
|
||||
super(config, context);
|
||||
this.smallFilesMap = smallFilesMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<SmallFile> smallFilesProfile(String partitionPath) {
|
||||
if (this.smallFilesMap.containsKey(partitionPath)) {
|
||||
return this.smallFilesMap.get(partitionPath);
|
||||
}
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,71 @@
|
||||
/*
|
||||
* 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.adapter.OutputAdapter;
|
||||
|
||||
import org.apache.flink.streaming.api.operators.Output;
|
||||
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.OutputTag;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Collecting {@link Output} for {@link StreamRecord}.
|
||||
*/
|
||||
public class CollectorOutput<T> implements OutputAdapter<StreamRecord<T>> {
|
||||
|
||||
private final List<T> records;
|
||||
|
||||
public CollectorOutput() {
|
||||
this.records = new ArrayList<>();
|
||||
}
|
||||
|
||||
public List<T> getRecords() {
|
||||
return this.records;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void emitWatermark(Watermark mark) {
|
||||
// no operation
|
||||
}
|
||||
|
||||
@Override
|
||||
public void emitLatencyMarker(LatencyMarker latencyMarker) {
|
||||
// no operation
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(StreamRecord<T> record) {
|
||||
records.add(record.getValue());
|
||||
}
|
||||
|
||||
@Override
|
||||
public <X> void collect(OutputTag<X> outputTag, StreamRecord<X> record) {
|
||||
throw new UnsupportedOperationException("Side output not supported for CollectorOutput");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
this.records.clear();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,120 @@
|
||||
/*
|
||||
* 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.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.sink.compact.CompactFunction;
|
||||
import org.apache.hudi.sink.compact.CompactionCommitEvent;
|
||||
import org.apache.hudi.sink.compact.CompactionCommitSink;
|
||||
import org.apache.hudi.sink.compact.CompactionPlanEvent;
|
||||
import org.apache.hudi.sink.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.jobgraph.OperatorID;
|
||||
import org.apache.flink.runtime.memory.MemoryManager;
|
||||
import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext;
|
||||
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.util.Collector;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A wrapper class to manipulate the {@link org.apache.hudi.sink.compact.CompactFunction} instance for testing.
|
||||
*/
|
||||
public class CompactFunctionWrapper {
|
||||
private final Configuration conf;
|
||||
|
||||
private final IOManager ioManager;
|
||||
private final StreamingRuntimeContext runtimeContext;
|
||||
|
||||
/**
|
||||
* Function that generates the {@link HoodieCompactionPlan}.
|
||||
*/
|
||||
private CompactionPlanOperator compactionPlanOperator;
|
||||
/**
|
||||
* 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;
|
||||
}
|
||||
|
||||
public void openFunction() throws Exception {
|
||||
compactionPlanOperator = new CompactionPlanOperator(conf);
|
||||
compactionPlanOperator.open();
|
||||
|
||||
compactFunction = new CompactFunction(conf);
|
||||
compactFunction.setRuntimeContext(runtimeContext);
|
||||
compactFunction.open(conf);
|
||||
final NonThrownExecutor syncExecutor = new MockCoordinatorExecutor(
|
||||
new MockOperatorCoordinatorContext(new OperatorID(), 1));
|
||||
compactFunction.setExecutor(syncExecutor);
|
||||
|
||||
commitSink = new CompactionCommitSink(conf);
|
||||
commitSink.setRuntimeContext(runtimeContext);
|
||||
commitSink.open(conf);
|
||||
}
|
||||
|
||||
public void compact(long checkpointID) throws Exception {
|
||||
// collect the CompactEvents.
|
||||
CollectorOutput<CompactionPlanEvent> output = new CollectorOutput<>();
|
||||
compactionPlanOperator.setOutput(output);
|
||||
compactionPlanOperator.notifyCheckpointComplete(checkpointID);
|
||||
// collect the CompactCommitEvents
|
||||
List<CompactionCommitEvent> compactCommitEvents = new ArrayList<>();
|
||||
for (CompactionPlanEvent event : output.getRecords()) {
|
||||
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();
|
||||
}
|
||||
}
|
||||
@@ -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.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.MockFunctionSnapshotContext;
|
||||
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> implements TestFunctionWrapper<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();
|
||||
}
|
||||
|
||||
public void checkpointFunction(long checkpointId) throws Exception {
|
||||
// checkpoint the coordinator first
|
||||
this.coordinator.checkpointCoordinator(checkpointId, new CompletableFuture<>());
|
||||
|
||||
writeFunction.snapshotState(new MockFunctionSnapshotContext(checkpointId));
|
||||
stateInitializationContext.getOperatorStateStore().checkpointBegin(checkpointId);
|
||||
}
|
||||
|
||||
public void checkpointComplete(long checkpointId) {
|
||||
stateInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId);
|
||||
coordinator.notifyCheckpointComplete(checkpointId);
|
||||
}
|
||||
|
||||
public StreamWriteOperatorCoordinator getCoordinator() {
|
||||
return coordinator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
this.coordinator.close();
|
||||
}
|
||||
|
||||
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());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,60 @@
|
||||
/*
|
||||
* 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.exception.HoodieException;
|
||||
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
|
||||
import org.apache.flink.util.ExceptionUtils;
|
||||
import org.apache.flink.util.function.ThrowingRunnable;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* A mock {@link NonThrownExecutor} that executes the actions synchronously.
|
||||
*/
|
||||
public class MockCoordinatorExecutor extends NonThrownExecutor {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(MockCoordinatorExecutor.class);
|
||||
|
||||
public MockCoordinatorExecutor(OperatorCoordinator.Context context) {
|
||||
super(LOG, (errMsg, t) -> context.failJob(new HoodieException(errMsg, t)), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(
|
||||
ThrowingRunnable<Throwable> action,
|
||||
ExceptionHook hook,
|
||||
String actionName,
|
||||
Object... actionParams) {
|
||||
final String actionString = String.format(actionName, actionParams);
|
||||
try {
|
||||
action.run();
|
||||
LOG.info("Executor executes action [{}] success!", actionString);
|
||||
} catch (Throwable t) {
|
||||
// if we have a JVM critical error, promote it immediately, there is a good
|
||||
// chance the
|
||||
// logging or job failing will not succeed anymore
|
||||
ExceptionUtils.rethrowIfFatalErrorOrOOM(t);
|
||||
final String errMsg = String.format("Executor executes action [%s] error", actionString);
|
||||
if (hook != null) {
|
||||
hook.apply(errMsg, t);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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.sink.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();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,138 @@
|
||||
/*
|
||||
* 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.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.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;
|
||||
private Map<String, ValueState> valueStateMap;
|
||||
|
||||
public MockOperatorStateStore() {
|
||||
this.historyStateMap = new HashMap<>();
|
||||
|
||||
this.currentStateMap = new HashMap<>();
|
||||
this.lastSuccessStateMap = new HashMap<>();
|
||||
|
||||
this.mapState = new MockMapState<>();
|
||||
this.valueStateMap = new HashMap<>();
|
||||
}
|
||||
|
||||
@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) {
|
||||
String name = valueStateDescriptor.getName();
|
||||
valueStateMap.putIfAbsent(name, new MockValueState());
|
||||
return valueStateMap.get(name);
|
||||
}
|
||||
|
||||
@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
|
||||
@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;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,62 @@
|
||||
/*
|
||||
* 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.adapter.StateInitializationContextAdapter;
|
||||
|
||||
import org.apache.flink.api.common.state.KeyedStateStore;
|
||||
import org.apache.flink.runtime.state.FunctionInitializationContext;
|
||||
import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
|
||||
import org.apache.flink.runtime.state.StatePartitionStreamProvider;
|
||||
|
||||
/**
|
||||
* A {@link FunctionInitializationContext} for testing purpose.
|
||||
*/
|
||||
public class MockStateInitializationContext implements StateInitializationContextAdapter {
|
||||
|
||||
private final MockOperatorStateStore operatorStateStore;
|
||||
|
||||
public MockStateInitializationContext() {
|
||||
operatorStateStore = new MockOperatorStateStore();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isRestored() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MockOperatorStateStore getOperatorStateStore() {
|
||||
return operatorStateStore;
|
||||
}
|
||||
|
||||
@Override
|
||||
public KeyedStateStore getKeyedStateStore() {
|
||||
return operatorStateStore;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<StatePartitionStreamProvider> getRawOperatorStateInputs() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<KeyGroupStatePartitionStreamProvider> getRawKeyedStateInputs() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,119 @@
|
||||
/*
|
||||
* 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.adapter.StreamingRuntimeContextAdapter;
|
||||
|
||||
import org.apache.flink.api.common.ExecutionConfig;
|
||||
import org.apache.flink.api.common.state.KeyedStateStore;
|
||||
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 StreamingRuntimeContextAdapter {
|
||||
|
||||
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 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;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sink.utils;
|
||||
|
||||
import org.apache.flink.api.common.state.ValueState;
|
||||
|
||||
/**
|
||||
* Mock value state for testing.
|
||||
*
|
||||
* @param <V> Type of state value
|
||||
*/
|
||||
public class MockValueState<V> implements ValueState<V> {
|
||||
private V v = null;
|
||||
|
||||
@Override
|
||||
public V value() {
|
||||
return v;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void update(V value) {
|
||||
this.v = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
v = null;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,305 @@
|
||||
/*
|
||||
* 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.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.sink.StreamWriteFunction;
|
||||
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
|
||||
import org.apache.hudi.sink.bootstrap.BootstrapOperator;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignFunction;
|
||||
import org.apache.hudi.sink.transform.RowDataToHoodieFunction;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
|
||||
import org.apache.flink.api.common.ExecutionConfig;
|
||||
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.graph.StreamConfig;
|
||||
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.flink.streaming.util.MockStreamTask;
|
||||
import org.apache.flink.streaming.util.MockStreamTaskBuilder;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
||||
/**
|
||||
* A wrapper class to manipulate the {@link StreamWriteFunction} instance for testing.
|
||||
*
|
||||
* @param <I> Input type
|
||||
*/
|
||||
public class StreamWriteFunctionWrapper<I> implements TestFunctionWrapper<I> {
|
||||
private final Configuration conf;
|
||||
|
||||
private final IOManager ioManager;
|
||||
private final StreamingRuntimeContext runtimeContext;
|
||||
private final MockOperatorEventGateway gateway;
|
||||
private final MockOperatorCoordinatorContext coordinatorContext;
|
||||
private final StreamWriteOperatorCoordinator coordinator;
|
||||
private final MockStateInitializationContext stateInitializationContext;
|
||||
|
||||
/**
|
||||
* Function that converts row data to HoodieRecord.
|
||||
*/
|
||||
private RowDataToHoodieFunction<RowData, HoodieRecord<?>> toHoodieFunction;
|
||||
/**
|
||||
* Function that load index in state.
|
||||
*/
|
||||
private BootstrapOperator<HoodieRecord<?>, HoodieRecord<?>> bootstrapOperator;
|
||||
/**
|
||||
* Function that assigns bucket ID.
|
||||
*/
|
||||
private BucketAssignFunction<String, HoodieRecord<?>, HoodieRecord<?>> bucketAssignerFunction;
|
||||
/**
|
||||
* BucketAssignOperator context.
|
||||
**/
|
||||
private final MockBucketAssignFunctionContext bucketAssignFunctionContext;
|
||||
/**
|
||||
* Stream write function.
|
||||
*/
|
||||
private StreamWriteFunction<HoodieRecord<?>> writeFunction;
|
||||
|
||||
private CompactFunctionWrapper compactFunctionWrapper;
|
||||
|
||||
private final MockStreamTask streamTask;
|
||||
|
||||
private final StreamConfig streamConfig;
|
||||
|
||||
private final boolean asyncCompaction;
|
||||
|
||||
public StreamWriteFunctionWrapper(String tablePath) throws Exception {
|
||||
this(tablePath, TestConfigurations.getDefaultConf(tablePath));
|
||||
}
|
||||
|
||||
public StreamWriteFunctionWrapper(String tablePath, 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.gateway = new MockOperatorEventGateway();
|
||||
this.conf = conf;
|
||||
// one function
|
||||
this.coordinatorContext = new MockOperatorCoordinatorContext(new OperatorID(), 1);
|
||||
this.coordinator = new StreamWriteOperatorCoordinator(conf, this.coordinatorContext);
|
||||
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
|
||||
this.bucketAssignFunctionContext = new MockBucketAssignFunctionContext();
|
||||
this.stateInitializationContext = new MockStateInitializationContext();
|
||||
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
|
||||
this.asyncCompaction = StreamerUtil.needsAsyncCompaction(conf);
|
||||
this.streamConfig = new StreamConfig(conf);
|
||||
streamConfig.setOperatorID(new OperatorID());
|
||||
this.streamTask = new MockStreamTaskBuilder(environment)
|
||||
.setConfig(new StreamConfig(conf))
|
||||
.setExecutionConfig(new ExecutionConfig().enableObjectReuse())
|
||||
.build();
|
||||
}
|
||||
|
||||
public void openFunction() throws Exception {
|
||||
this.coordinator.start();
|
||||
this.coordinator.setExecutor(new MockCoordinatorExecutor(coordinatorContext));
|
||||
toHoodieFunction = new RowDataToHoodieFunction<>(TestConfigurations.ROW_TYPE, conf);
|
||||
toHoodieFunction.setRuntimeContext(runtimeContext);
|
||||
toHoodieFunction.open(conf);
|
||||
|
||||
bucketAssignerFunction = new BucketAssignFunction<>(conf);
|
||||
bucketAssignerFunction.setRuntimeContext(runtimeContext);
|
||||
bucketAssignerFunction.open(conf);
|
||||
bucketAssignerFunction.initializeState(this.stateInitializationContext);
|
||||
|
||||
if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) {
|
||||
bootstrapOperator = new BootstrapOperator<>(conf);
|
||||
CollectorOutput<HoodieRecord<?>> output = new CollectorOutput<>();
|
||||
bootstrapOperator.setup(streamTask, streamConfig, output);
|
||||
bootstrapOperator.initializeState(this.stateInitializationContext);
|
||||
|
||||
Collector<HoodieRecord<?>> collector = ScalaCollector.getInstance();
|
||||
for (HoodieRecord<?> bootstrapRecord : output.getRecords()) {
|
||||
bucketAssignerFunction.processElement(bootstrapRecord, null, collector);
|
||||
bucketAssignFunctionContext.setCurrentKey(bootstrapRecord.getRecordKey());
|
||||
}
|
||||
}
|
||||
|
||||
setupWriteFunction();
|
||||
|
||||
if (asyncCompaction) {
|
||||
compactFunctionWrapper.openFunction();
|
||||
}
|
||||
}
|
||||
|
||||
public void invoke(I record) throws Exception {
|
||||
HoodieRecord<?> hoodieRecord = toHoodieFunction.map((RowData) record);
|
||||
ScalaCollector<HoodieRecord<?>> collector = ScalaCollector.getInstance();
|
||||
bucketAssignerFunction.processElement(hoodieRecord, null, collector);
|
||||
bucketAssignFunctionContext.setCurrentKey(hoodieRecord.getRecordKey());
|
||||
writeFunction.processElement(collector.getVal(), null, null);
|
||||
}
|
||||
|
||||
public WriteMetadataEvent[] getEventBuffer() {
|
||||
return this.coordinator.getEventBuffer();
|
||||
}
|
||||
|
||||
public OperatorEvent getNextEvent() {
|
||||
return this.gateway.getNextEvent();
|
||||
}
|
||||
|
||||
public Map<String, List<HoodieRecord>> getDataBuffer() {
|
||||
return this.writeFunction.getDataBuffer();
|
||||
}
|
||||
|
||||
public void checkpointFunction(long checkpointId) throws Exception {
|
||||
// checkpoint the coordinator first
|
||||
this.coordinator.checkpointCoordinator(checkpointId, new CompletableFuture<>());
|
||||
if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) {
|
||||
bootstrapOperator.snapshotState(null);
|
||||
}
|
||||
bucketAssignerFunction.snapshotState(null);
|
||||
|
||||
writeFunction.snapshotState(new MockFunctionSnapshotContext(checkpointId));
|
||||
stateInitializationContext.getOperatorStateStore().checkpointBegin(checkpointId);
|
||||
}
|
||||
|
||||
public void endInput() {
|
||||
writeFunction.endInput();
|
||||
}
|
||||
|
||||
public void checkpointComplete(long checkpointId) {
|
||||
stateInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId);
|
||||
coordinator.notifyCheckpointComplete(checkpointId);
|
||||
this.bucketAssignerFunction.notifyCheckpointComplete(checkpointId);
|
||||
if (asyncCompaction) {
|
||||
try {
|
||||
compactFunctionWrapper.compact(checkpointId);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void checkpointFails(long checkpointId) {
|
||||
coordinator.notifyCheckpointAborted(checkpointId);
|
||||
}
|
||||
|
||||
public void subTaskFails(int taskID) throws Exception {
|
||||
coordinator.subtaskFailed(taskID, new RuntimeException("Dummy exception"));
|
||||
setupWriteFunction();
|
||||
}
|
||||
|
||||
public void close() throws Exception {
|
||||
coordinator.close();
|
||||
ioManager.close();
|
||||
bucketAssignerFunction.close();
|
||||
writeFunction.close();
|
||||
if (compactFunctionWrapper != null) {
|
||||
compactFunctionWrapper.close();
|
||||
}
|
||||
}
|
||||
|
||||
public StreamWriteOperatorCoordinator getCoordinator() {
|
||||
return coordinator;
|
||||
}
|
||||
|
||||
public MockOperatorCoordinatorContext getCoordinatorContext() {
|
||||
return coordinatorContext;
|
||||
}
|
||||
|
||||
public boolean isKeyInState(HoodieKey hoodieKey) {
|
||||
return this.bucketAssignFunctionContext.isKeyInState(hoodieKey.getRecordKey());
|
||||
}
|
||||
|
||||
public boolean isConforming() {
|
||||
return this.writeFunction.isConfirming();
|
||||
}
|
||||
|
||||
public boolean isAlreadyBootstrap() throws Exception {
|
||||
return this.bootstrapOperator.isAlreadyBootstrap();
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
private void setupWriteFunction() throws Exception {
|
||||
writeFunction = new StreamWriteFunction<>(conf);
|
||||
writeFunction.setRuntimeContext(runtimeContext);
|
||||
writeFunction.setOperatorEventGateway(gateway);
|
||||
writeFunction.initializeState(this.stateInitializationContext);
|
||||
writeFunction.open(conf);
|
||||
|
||||
// handle the bootstrap event
|
||||
coordinator.handleEventFromOperator(0, getNextEvent());
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Inner Class
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
private static class MockBucketAssignFunctionContext {
|
||||
private final Set<Object> updateKeys = new HashSet<>();
|
||||
|
||||
public void setCurrentKey(Object key) {
|
||||
this.updateKeys.add(key);
|
||||
}
|
||||
|
||||
public boolean isKeyInState(String key) {
|
||||
return this.updateKeys.contains(key);
|
||||
}
|
||||
}
|
||||
|
||||
private static class ScalaCollector<T> implements Collector<T> {
|
||||
private T val;
|
||||
|
||||
public static <T> ScalaCollector<T> getInstance() {
|
||||
return new ScalaCollector<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(T t) {
|
||||
this.val = t;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
this.val = null;
|
||||
}
|
||||
|
||||
public T getVal() {
|
||||
return val;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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.sink.utils;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
|
||||
import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Define the common interfaces for test function wrappers.
|
||||
*/
|
||||
public interface TestFunctionWrapper<I> {
|
||||
/**
|
||||
* Open all the functions within this wrapper.
|
||||
*/
|
||||
void openFunction() throws Exception;
|
||||
|
||||
/**
|
||||
* Process the given input record {@code record}.
|
||||
*/
|
||||
void invoke(I record) throws Exception;
|
||||
|
||||
/**
|
||||
* Returns the event buffer sent by the write tasks.
|
||||
*/
|
||||
WriteMetadataEvent[] getEventBuffer();
|
||||
|
||||
/**
|
||||
* Returns the next event.
|
||||
*/
|
||||
OperatorEvent getNextEvent();
|
||||
|
||||
/**
|
||||
* Snapshot all the functions in the wrapper.
|
||||
*/
|
||||
void checkpointFunction(long checkpointId) throws Exception;
|
||||
|
||||
/**
|
||||
* Mark checkpoint with id {code checkpointId} as success.
|
||||
*/
|
||||
void checkpointComplete(long checkpointId);
|
||||
|
||||
/**
|
||||
* Returns the operator coordinator.
|
||||
*/
|
||||
StreamWriteOperatorCoordinator getCoordinator();
|
||||
|
||||
/**
|
||||
* Returns the data buffer of the write task.
|
||||
*/
|
||||
default Map<String, List<HoodieRecord>> getDataBuffer() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Mark checkpoint with id {code checkpointId} as failed.
|
||||
*/
|
||||
default void checkpointFails(long checkpointId) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the context of the coordinator.
|
||||
*/
|
||||
default MockOperatorCoordinatorContext getCoordinatorContext() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Mark sub-task with id {@code taskId} as failed.
|
||||
*/
|
||||
default void subTaskFails(int taskId) throws Exception {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the given key {@code key} is in the state store.
|
||||
*/
|
||||
default boolean isKeyInState(HoodieKey key) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the bootstrap function already bootstrapped.
|
||||
*/
|
||||
default boolean isAlreadyBootstrap() throws Exception {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the write task is confirming.
|
||||
*/
|
||||
default boolean isConforming() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Close this function wrapper.
|
||||
*/
|
||||
void close() throws Exception;
|
||||
}
|
||||
@@ -0,0 +1,424 @@
|
||||
/*
|
||||
* 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.WriteStatus;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.OptionsResolver;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestData;
|
||||
import org.apache.hudi.utils.TestUtils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.hamcrest.MatcherAssert;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
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.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.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Base class for write test cases.
|
||||
*/
|
||||
public class TestWriteBase {
|
||||
protected static final Map<String, String> EXPECTED1 = new HashMap<>();
|
||||
|
||||
protected static final Map<String, String> EXPECTED2 = new HashMap<>();
|
||||
|
||||
protected static final Map<String, String> EXPECTED3 = new HashMap<>();
|
||||
|
||||
protected static final Map<String, String> EXPECTED4 = new HashMap<>();
|
||||
|
||||
protected static final Map<String, List<String>> EXPECTED5 = new HashMap<>();
|
||||
|
||||
static {
|
||||
EXPECTED1.put("par1", "[id1,par1,id1,Danny,23,1,par1, id2,par1,id2,Stephen,33,2,par1]");
|
||||
EXPECTED1.put("par2", "[id3,par2,id3,Julian,53,3,par2, id4,par2,id4,Fabian,31,4,par2]");
|
||||
EXPECTED1.put("par3", "[id5,par3,id5,Sophia,18,5,par3, id6,par3,id6,Emma,20,6,par3]");
|
||||
EXPECTED1.put("par4", "[id7,par4,id7,Bob,44,7,par4, id8,par4,id8,Han,56,8,par4]");
|
||||
|
||||
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]");
|
||||
|
||||
EXPECTED3.put("par1", "[id1,par1,id1,Danny,23,1,par1]");
|
||||
|
||||
EXPECTED4.put("par1", "["
|
||||
+ "id1,par1,id1,Danny,23,0,par1, "
|
||||
+ "id1,par1,id1,Danny,23,1,par1, "
|
||||
+ "id1,par1,id1,Danny,23,2,par1, "
|
||||
+ "id1,par1,id1,Danny,23,3,par1, "
|
||||
+ "id1,par1,id1,Danny,23,4,par1]");
|
||||
|
||||
EXPECTED5.put("par1", Arrays.asList(
|
||||
"id1,par1,id1,Danny,23,0,par1",
|
||||
"id1,par1,id1,Danny,23,0,par1",
|
||||
"id1,par1,id1,Danny,23,1,par1",
|
||||
"id1,par1,id1,Danny,23,1,par1",
|
||||
"id1,par1,id1,Danny,23,2,par1",
|
||||
"id1,par1,id1,Danny,23,2,par1",
|
||||
"id1,par1,id1,Danny,23,3,par1",
|
||||
"id1,par1,id1,Danny,23,3,par1",
|
||||
"id1,par1,id1,Danny,23,4,par1",
|
||||
"id1,par1,id1,Danny,23,4,par1"));
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Inner Class
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Utils to composite the test stages.
|
||||
*/
|
||||
public static class TestHarness {
|
||||
public static TestHarness instance() {
|
||||
return new TestHarness();
|
||||
}
|
||||
|
||||
private File baseFile;
|
||||
private String basePath;
|
||||
private Configuration conf;
|
||||
private TestFunctionWrapper<RowData> pipeline;
|
||||
|
||||
private String lastPending;
|
||||
private String lastComplete;
|
||||
|
||||
public TestHarness preparePipeline(File basePath, Configuration conf) throws Exception {
|
||||
preparePipeline(basePath, conf, false);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestHarness preparePipeline(File basePath, Configuration conf, boolean append) throws Exception {
|
||||
this.baseFile = basePath;
|
||||
this.basePath = this.baseFile.getAbsolutePath();
|
||||
this.conf = conf;
|
||||
this.pipeline = append
|
||||
? new InsertFunctionWrapper<>(this.basePath, conf)
|
||||
: new StreamWriteFunctionWrapper<>(this.basePath, conf);
|
||||
// open the function and ingest data
|
||||
this.pipeline.openFunction();
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestHarness consume(List<RowData> inputs) throws Exception {
|
||||
for (RowData rowData : inputs) {
|
||||
this.pipeline.invoke(rowData);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestHarness assertConsumeThrows(List<RowData> inputs, String message) {
|
||||
assertThrows(HoodieException.class, () -> consume(inputs), message);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Assert the event buffer is empty.
|
||||
*/
|
||||
public TestHarness emptyEventBuffer() {
|
||||
assertTrue(
|
||||
this.pipeline.getEventBuffer().length == 1
|
||||
&& this.pipeline.getEventBuffer()[0] == null,
|
||||
"The coordinator events buffer expect to be empty");
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Assert the next event exists and handle over it to the coordinator.
|
||||
*/
|
||||
public TestHarness assertNextEvent() {
|
||||
final OperatorEvent nextEvent = this.pipeline.getNextEvent();
|
||||
MatcherAssert.assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
this.pipeline.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(this.pipeline.getEventBuffer()[0], "The coordinator missed the event");
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Assert the next event exists and handle over it to the coordinator.
|
||||
*
|
||||
* @param numWriteStatus The expected write status num reported by the event
|
||||
* @param partitions The written partitions reported by the event
|
||||
*/
|
||||
public TestHarness assertNextEvent(int numWriteStatus, String partitions) {
|
||||
final OperatorEvent nextEvent = this.pipeline.getNextEvent();
|
||||
MatcherAssert.assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
List<WriteStatus> writeStatuses = ((WriteMetadataEvent) nextEvent).getWriteStatuses();
|
||||
assertNotNull(writeStatuses);
|
||||
MatcherAssert.assertThat(writeStatuses.size(), is(numWriteStatus));
|
||||
assertThat(writeStatuses.stream()
|
||||
.map(WriteStatus::getPartitionPath).sorted(Comparator.naturalOrder())
|
||||
.collect(Collectors.joining(",")),
|
||||
is(partitions));
|
||||
this.pipeline.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(this.pipeline.getEventBuffer()[0], "The coordinator missed the event");
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Assert the next event exists and handle over it to the coordinator.
|
||||
*
|
||||
* <p>Validates that the write metadata reported by the event is empty.
|
||||
*/
|
||||
public TestHarness assertEmptyEvent() {
|
||||
final OperatorEvent nextEvent = this.pipeline.getNextEvent();
|
||||
MatcherAssert.assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
List<WriteStatus> writeStatuses = ((WriteMetadataEvent) nextEvent).getWriteStatuses();
|
||||
assertNotNull(writeStatuses);
|
||||
MatcherAssert.assertThat(writeStatuses.size(), is(0));
|
||||
this.pipeline.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(this.pipeline.getEventBuffer()[0], "The coordinator missed the event");
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Assert the data buffer with given number of buckets and records.
|
||||
*/
|
||||
public TestHarness assertDataBuffer(int numBuckets, int numRecords) {
|
||||
Map<String, List<HoodieRecord>> dataBuffer = this.pipeline.getDataBuffer();
|
||||
assertThat("Should have " + numBuckets + " data bucket", dataBuffer.size(), is(numBuckets));
|
||||
assertThat(numRecords + " records expect to flush out as a mini-batch",
|
||||
dataBuffer.values().stream().findFirst().map(List::size).orElse(-1),
|
||||
is(numRecords));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Checkpoints the pipeline, which triggers the data write and event send.
|
||||
*/
|
||||
public TestHarness checkpoint(long checkpointId) throws Exception {
|
||||
this.pipeline.checkpointFunction(checkpointId);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestHarness allDataFlushed() {
|
||||
Map<String, List<HoodieRecord>> dataBuffer = this.pipeline.getDataBuffer();
|
||||
assertThat("All data should be flushed out", dataBuffer.size(), is(0));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Handle the next {@code numEvents} events and handle over them to the coordinator.
|
||||
*/
|
||||
public TestHarness handleEvents(int numEvents) {
|
||||
for (int i = 0; i < numEvents; i++) {
|
||||
final OperatorEvent event = this.pipeline.getNextEvent(); // remove the first event first
|
||||
assertThat("The operator expect to send an event", event, instanceOf(WriteMetadataEvent.class));
|
||||
this.pipeline.getCoordinator().handleEventFromOperator(0, event);
|
||||
}
|
||||
assertNotNull(this.pipeline.getEventBuffer()[0], "The coordinator missed the event");
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Mark the checkpoint with id {@code checkpointId} as finished.
|
||||
*/
|
||||
public TestHarness checkpointComplete(long checkpointId) {
|
||||
this.lastPending = lastPendingInstant();
|
||||
this.pipeline.checkpointComplete(checkpointId);
|
||||
// started a new instant already
|
||||
checkInflightInstant();
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, lastPending);
|
||||
this.lastComplete = lastPending;
|
||||
this.lastPending = lastPendingInstant(); // refresh last pending instant
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Mark the checkpoint finished with empty write metadata.
|
||||
*/
|
||||
public TestHarness emptyCheckpoint(long checkpointId) {
|
||||
String lastPending = lastPendingInstant();
|
||||
this.pipeline.checkpointComplete(checkpointId);
|
||||
// last pending instant was reused
|
||||
assertEquals(this.lastPending, lastPending);
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, lastComplete);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Mark the checkpoint with id {@code checkpointId} as failed.
|
||||
*/
|
||||
public TestHarness checkpointFails(long checkpointId) {
|
||||
this.pipeline.checkpointFails(checkpointId);
|
||||
assertFalse(this.pipeline.getCoordinatorContext().isJobFailed(),
|
||||
"The last checkpoint was aborted, ignore the events");
|
||||
// no complete instant
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, null);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestHarness checkpointThrows(long checkpointId, String message) {
|
||||
// this returns early because there is no inflight instant
|
||||
assertThrows(HoodieException.class, () -> checkpoint(checkpointId), message);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Mark the task with id {@code taskId} as failed.
|
||||
*/
|
||||
public TestHarness subTaskFails(int taskId) throws Exception {
|
||||
// fails the subtask
|
||||
String instant1 = lastPendingInstant();
|
||||
this.pipeline.subTaskFails(taskId);
|
||||
|
||||
String instant2 = lastPendingInstant();
|
||||
assertNotEquals(instant2, instant1, "The previous instant should be rolled back when starting new instant");
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestHarness noCompleteInstant() {
|
||||
// no complete instant
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, null);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Asserts the data files are empty.
|
||||
*/
|
||||
public TestHarness assertEmptyDataFiles() {
|
||||
File[] dataFiles = baseFile.listFiles(file -> !file.getName().startsWith("."));
|
||||
assertNotNull(dataFiles);
|
||||
assertThat(dataFiles.length, is(0));
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestHarness checkWrittenData(Map<String, String> expected) throws Exception {
|
||||
checkWrittenData(expected, 4);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestHarness checkWrittenData(
|
||||
Map<String, String> expected,
|
||||
int partitions) throws Exception {
|
||||
if (OptionsResolver.isCowTable(conf) || conf.getBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED)) {
|
||||
TestData.checkWrittenData(this.baseFile, expected, partitions);
|
||||
} else {
|
||||
checkWrittenDataMor(baseFile, expected, partitions);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
private void checkWrittenDataMor(File baseFile, Map<String, String> expected, int partitions) throws Exception {
|
||||
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(basePath);
|
||||
Schema schema = new TableSchemaResolver(metaClient).getTableAvroSchema();
|
||||
String latestInstant = lastCompleteInstant();
|
||||
FileSystem fs = FSUtils.getFs(basePath, new org.apache.hadoop.conf.Configuration());
|
||||
TestData.checkWrittenDataMOR(fs, latestInstant, baseFile, expected, partitions, schema);
|
||||
}
|
||||
|
||||
public TestHarness checkWrittenFullData(Map<String, List<String>> expected) throws IOException {
|
||||
TestData.checkWrittenFullData(this.baseFile, expected);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestHarness checkWrittenAllData(Map<String, String> expected, int partitions) throws IOException {
|
||||
TestData.checkWrittenAllData(baseFile, expected, partitions);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestHarness checkIndexLoaded(HoodieKey... keys) {
|
||||
for (HoodieKey key : keys) {
|
||||
assertTrue(this.pipeline.isKeyInState(key),
|
||||
"Key: " + key + " assumes to be in the index state");
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestHarness assertBootstrapped() throws Exception {
|
||||
assertTrue(this.pipeline.isAlreadyBootstrap());
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestHarness assertConfirming() {
|
||||
assertTrue(this.pipeline.isConforming(),
|
||||
"The write function should be waiting for the instant to commit");
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestHarness assertNotConfirming() {
|
||||
assertFalse(this.pipeline.isConforming(),
|
||||
"The write function should finish waiting for the instant to commit");
|
||||
return this;
|
||||
}
|
||||
|
||||
public void end() throws Exception {
|
||||
this.pipeline.close();
|
||||
}
|
||||
|
||||
private String lastPendingInstant() {
|
||||
return TestUtils.getLastPendingInstant(basePath);
|
||||
}
|
||||
|
||||
private void checkInflightInstant() {
|
||||
final String instant = TestUtils.getLastPendingInstant(basePath);
|
||||
assertNotNull(instant);
|
||||
}
|
||||
|
||||
private void checkInstantState(HoodieInstant.State state, String instantStr) {
|
||||
final String instant;
|
||||
switch (state) {
|
||||
case REQUESTED:
|
||||
instant = lastPendingInstant();
|
||||
break;
|
||||
case COMPLETED:
|
||||
instant = lastCompleteInstant();
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError("Unexpected state");
|
||||
}
|
||||
assertThat(instant, is(instantStr));
|
||||
}
|
||||
|
||||
protected String lastCompleteInstant() {
|
||||
return OptionsResolver.isMorTable(conf)
|
||||
? TestUtils.getLastDeltaCompleteInstant(basePath)
|
||||
: TestUtils.getLastCompleteInstant(basePath);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,105 @@
|
||||
/*
|
||||
* 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.source;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.keygen.NonpartitionedAvroKeyGenerator;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
import org.apache.hudi.utils.TestData;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Test cases for {@link FileIndex}.
|
||||
*/
|
||||
public class TestFileIndex {
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
void testFileListingUsingMetadata(boolean hiveStylePartitioning) throws Exception {
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setBoolean(FlinkOptions.METADATA_ENABLED, true);
|
||||
conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning);
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
FileIndex fileIndex = FileIndex.instance(new Path(tempFile.getAbsolutePath()), conf);
|
||||
List<String> partitionKeys = Collections.singletonList("partition");
|
||||
List<Map<String, String>> partitions = fileIndex.getPartitions(partitionKeys, "default", hiveStylePartitioning);
|
||||
assertTrue(partitions.stream().allMatch(m -> m.size() == 1));
|
||||
String partitionPaths = partitions.stream()
|
||||
.map(Map::values).flatMap(Collection::stream).sorted().collect(Collectors.joining(","));
|
||||
assertThat("should have 4 partitions", partitionPaths, is("par1,par2,par3,par4"));
|
||||
|
||||
FileStatus[] fileStatuses = fileIndex.getFilesInPartitions();
|
||||
assertThat(fileStatuses.length, is(4));
|
||||
assertTrue(Arrays.stream(fileStatuses)
|
||||
.allMatch(fileStatus -> fileStatus.getPath().toString().endsWith(HoodieFileFormat.PARQUET.getFileExtension())));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFileListingUsingMetadataNonPartitionedTable() throws Exception {
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setString(FlinkOptions.PARTITION_PATH_FIELD, "");
|
||||
conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, NonpartitionedAvroKeyGenerator.class.getName());
|
||||
conf.setBoolean(FlinkOptions.METADATA_ENABLED, true);
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
FileIndex fileIndex = FileIndex.instance(new Path(tempFile.getAbsolutePath()), conf);
|
||||
List<String> partitionKeys = Collections.singletonList("");
|
||||
List<Map<String, String>> partitions = fileIndex.getPartitions(partitionKeys, "default", false);
|
||||
assertThat(partitions.size(), is(0));
|
||||
|
||||
FileStatus[] fileStatuses = fileIndex.getFilesInPartitions();
|
||||
assertThat(fileStatuses.length, is(1));
|
||||
assertTrue(fileStatuses[0].getPath().toString().endsWith(HoodieFileFormat.PARQUET.getFileExtension()));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
void testFileListingEmptyTable(boolean enableMetadata) {
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setBoolean(FlinkOptions.METADATA_ENABLED, enableMetadata);
|
||||
FileIndex fileIndex = FileIndex.instance(new Path(tempFile.getAbsolutePath()), conf);
|
||||
List<String> partitionKeys = Collections.singletonList("partition");
|
||||
List<Map<String, String>> partitions = fileIndex.getPartitions(partitionKeys, "default", false);
|
||||
assertThat(partitions.size(), is(0));
|
||||
|
||||
FileStatus[] fileStatuses = fileIndex.getFilesInPartitions();
|
||||
assertThat(fileStatuses.length, is(0));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,329 @@
|
||||
/*
|
||||
* 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.source;
|
||||
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
import org.apache.hudi.utils.TestData;
|
||||
import org.apache.hudi.utils.TestUtils;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
|
||||
import org.apache.flink.streaming.api.functions.source.SourceFunction;
|
||||
import org.apache.flink.streaming.api.operators.StreamSource;
|
||||
import org.apache.flink.streaming.api.watermark.Watermark;
|
||||
import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
|
||||
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.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Test cases for {@link StreamReadMonitoringFunction}.
|
||||
*/
|
||||
public class TestStreamReadMonitoringFunction {
|
||||
private static final long WAIT_TIME_MILLIS = 5 * 1000L;
|
||||
|
||||
private Configuration conf;
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@BeforeEach
|
||||
public void before() throws Exception {
|
||||
final String basePath = tempFile.getAbsolutePath();
|
||||
conf = TestConfigurations.getDefaultConf(basePath);
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
conf.setInteger(FlinkOptions.READ_STREAMING_CHECK_INTERVAL, 2); // check every 2 seconds
|
||||
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConsumeFromLatestCommit() throws Exception {
|
||||
// write 2 commits first, and all the splits should come from the second commit.
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
TestData.writeData(TestData.DATA_SET_UPDATE_INSERT, conf);
|
||||
StreamReadMonitoringFunction function = TestUtils.getMonitorFunc(conf);
|
||||
try (AbstractStreamOperatorTestHarness<MergeOnReadInputSplit> harness = createHarness(function)) {
|
||||
harness.setup();
|
||||
harness.open();
|
||||
|
||||
CountDownLatch latch = new CountDownLatch(4);
|
||||
CollectingSourceContext sourceContext = new CollectingSourceContext(latch);
|
||||
|
||||
runAsync(sourceContext, function);
|
||||
|
||||
assertTrue(latch.await(WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS), "Should finish splits generation");
|
||||
assertThat("Should produce the expected splits",
|
||||
sourceContext.getPartitionPaths(), is("par1,par2,par3,par4"));
|
||||
|
||||
assertTrue(sourceContext.splits.stream().allMatch(split -> split.getInstantRange().isPresent()),
|
||||
"All the instants should have range limit");
|
||||
String latestCommit = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath());
|
||||
assertTrue(sourceContext.splits.stream().allMatch(split -> split.getLatestCommit().equals(latestCommit)),
|
||||
"All the splits should be with latestCommit instant time");
|
||||
|
||||
// Stop the stream task.
|
||||
function.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConsumeFromLastCommit() throws Exception {
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
StreamReadMonitoringFunction function = TestUtils.getMonitorFunc(conf);
|
||||
try (AbstractStreamOperatorTestHarness<MergeOnReadInputSplit> harness = createHarness(function)) {
|
||||
harness.setup();
|
||||
harness.open();
|
||||
|
||||
CountDownLatch latch = new CountDownLatch(4);
|
||||
CollectingSourceContext sourceContext = new CollectingSourceContext(latch);
|
||||
|
||||
runAsync(sourceContext, function);
|
||||
|
||||
assertTrue(latch.await(WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS), "Should finish splits generation");
|
||||
assertThat("Should produce the expected splits",
|
||||
sourceContext.getPartitionPaths(), is("par1,par2,par3,par4"));
|
||||
assertTrue(sourceContext.splits.stream().allMatch(split -> split.getInstantRange().isPresent()),
|
||||
"All instants should have range limit");
|
||||
|
||||
Thread.sleep(1000L);
|
||||
|
||||
// reset the source context
|
||||
latch = new CountDownLatch(4);
|
||||
sourceContext.reset(latch);
|
||||
|
||||
// write another instant and validate
|
||||
TestData.writeData(TestData.DATA_SET_UPDATE_INSERT, conf);
|
||||
|
||||
assertTrue(latch.await(WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS), "Should finish splits generation");
|
||||
assertThat("Should produce the expected splits",
|
||||
sourceContext.getPartitionPaths(), is("par1,par2,par3,par4"));
|
||||
assertTrue(sourceContext.splits.stream().allMatch(split -> split.getInstantRange().isPresent()),
|
||||
"All the instants should have range limit");
|
||||
|
||||
// Stop the stream task.
|
||||
function.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConsumeFromSpecifiedCommit() throws Exception {
|
||||
// write 2 commits first, use the second commit time as the specified start instant,
|
||||
// all the splits should come from the second commit.
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
TestData.writeData(TestData.DATA_SET_UPDATE_INSERT, conf);
|
||||
String specifiedCommit = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath());
|
||||
conf.setString(FlinkOptions.READ_START_COMMIT, specifiedCommit);
|
||||
StreamReadMonitoringFunction function = TestUtils.getMonitorFunc(conf);
|
||||
try (AbstractStreamOperatorTestHarness<MergeOnReadInputSplit> harness = createHarness(function)) {
|
||||
harness.setup();
|
||||
harness.open();
|
||||
|
||||
CountDownLatch latch = new CountDownLatch(4);
|
||||
CollectingSourceContext sourceContext = new CollectingSourceContext(latch);
|
||||
|
||||
runAsync(sourceContext, function);
|
||||
|
||||
assertTrue(latch.await(WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS), "Should finish splits generation");
|
||||
assertThat("Should produce the expected splits",
|
||||
sourceContext.getPartitionPaths(), is("par1,par2,par3,par4"));
|
||||
assertTrue(sourceContext.splits.stream().allMatch(split -> split.getInstantRange().isPresent()),
|
||||
"All the instants should have range limit");
|
||||
assertTrue(sourceContext.splits.stream().allMatch(split -> split.getLatestCommit().equals(specifiedCommit)),
|
||||
"All the splits should be with specified instant time");
|
||||
|
||||
// Stop the stream task.
|
||||
function.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConsumeFromEarliestCommit() throws Exception {
|
||||
// write 2 commits first, then specify the start commit as 'earliest',
|
||||
// all the splits should come from the earliest commit.
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
TestData.writeData(TestData.DATA_SET_UPDATE_INSERT, conf);
|
||||
String specifiedCommit = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath());
|
||||
conf.setString(FlinkOptions.READ_START_COMMIT, FlinkOptions.START_COMMIT_EARLIEST);
|
||||
StreamReadMonitoringFunction function = TestUtils.getMonitorFunc(conf);
|
||||
try (AbstractStreamOperatorTestHarness<MergeOnReadInputSplit> harness = createHarness(function)) {
|
||||
harness.setup();
|
||||
harness.open();
|
||||
|
||||
CountDownLatch latch = new CountDownLatch(4);
|
||||
CollectingSourceContext sourceContext = new CollectingSourceContext(latch);
|
||||
|
||||
runAsync(sourceContext, function);
|
||||
|
||||
assertTrue(latch.await(WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS), "Should finish splits generation");
|
||||
assertThat("Should produce the expected splits",
|
||||
sourceContext.getPartitionPaths(), is("par1,par2,par3,par4"));
|
||||
assertTrue(sourceContext.splits.stream().noneMatch(split -> split.getInstantRange().isPresent()),
|
||||
"No instants should have range limit");
|
||||
assertTrue(sourceContext.splits.stream().allMatch(split -> split.getLatestCommit().equals(specifiedCommit)),
|
||||
"All the splits should be with specified instant time");
|
||||
|
||||
// Stop the stream task.
|
||||
function.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckpointRestore() throws Exception {
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
|
||||
StreamReadMonitoringFunction function = TestUtils.getMonitorFunc(conf);
|
||||
OperatorSubtaskState state;
|
||||
try (AbstractStreamOperatorTestHarness<MergeOnReadInputSplit> harness = createHarness(function)) {
|
||||
harness.setup();
|
||||
harness.open();
|
||||
|
||||
CountDownLatch latch = new CountDownLatch(4);
|
||||
CollectingSourceContext sourceContext = new CollectingSourceContext(latch);
|
||||
runAsync(sourceContext, function);
|
||||
|
||||
assertTrue(latch.await(WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS), "Should finish splits generation");
|
||||
Thread.sleep(1000L);
|
||||
|
||||
state = harness.snapshot(1, 1);
|
||||
|
||||
// Stop the stream task.
|
||||
function.close();
|
||||
|
||||
assertTrue(latch.await(WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS), "Should finish splits generation");
|
||||
assertThat("Should produce the expected splits",
|
||||
sourceContext.getPartitionPaths(), is("par1,par2,par3,par4"));
|
||||
assertTrue(sourceContext.splits.stream().allMatch(split -> split.getInstantRange().isPresent()),
|
||||
"All instants should have range limit");
|
||||
|
||||
}
|
||||
|
||||
TestData.writeData(TestData.DATA_SET_UPDATE_INSERT, conf);
|
||||
StreamReadMonitoringFunction function2 = TestUtils.getMonitorFunc(conf);
|
||||
try (AbstractStreamOperatorTestHarness<MergeOnReadInputSplit> harness = createHarness(function2)) {
|
||||
harness.setup();
|
||||
// Recover to process the remaining snapshots.
|
||||
harness.initializeState(state);
|
||||
harness.open();
|
||||
|
||||
CountDownLatch latch = new CountDownLatch(4);
|
||||
CollectingSourceContext sourceContext = new CollectingSourceContext(latch);
|
||||
runAsync(sourceContext, function2);
|
||||
|
||||
// Stop the stream task.
|
||||
function.close();
|
||||
|
||||
assertTrue(latch.await(WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS), "Should finish splits generation");
|
||||
assertThat("Should produce the expected splits",
|
||||
sourceContext.getPartitionPaths(), is("par1,par2,par3,par4"));
|
||||
assertTrue(sourceContext.splits.stream().allMatch(split -> split.getInstantRange().isPresent()),
|
||||
"All the instants should have range limit");
|
||||
}
|
||||
}
|
||||
|
||||
private AbstractStreamOperatorTestHarness<MergeOnReadInputSplit> createHarness(
|
||||
StreamReadMonitoringFunction function) throws Exception {
|
||||
StreamSource<MergeOnReadInputSplit, StreamReadMonitoringFunction> streamSource = new StreamSource<>(function);
|
||||
return new AbstractStreamOperatorTestHarness<>(streamSource, 1, 1, 0);
|
||||
}
|
||||
|
||||
private void runAsync(
|
||||
CollectingSourceContext sourceContext,
|
||||
StreamReadMonitoringFunction function) {
|
||||
Thread task = new Thread(() -> {
|
||||
try {
|
||||
function.run(sourceContext);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
task.start();
|
||||
}
|
||||
|
||||
/**
|
||||
* Source context that collects the outputs in to a list.
|
||||
*/
|
||||
private static class CollectingSourceContext implements SourceFunction.SourceContext<MergeOnReadInputSplit> {
|
||||
private final List<MergeOnReadInputSplit> splits = new ArrayList<>();
|
||||
private final Object checkpointLock = new Object();
|
||||
private volatile CountDownLatch latch;
|
||||
|
||||
CollectingSourceContext(CountDownLatch latch) {
|
||||
this.latch = latch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(MergeOnReadInputSplit element) {
|
||||
splits.add(element);
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collectWithTimestamp(MergeOnReadInputSplit element, long timestamp) {
|
||||
collect(element);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void emitWatermark(Watermark mark) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void markAsTemporarilyIdle() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCheckpointLock() {
|
||||
return checkpointLock;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
|
||||
}
|
||||
|
||||
public void reset(CountDownLatch latch) {
|
||||
this.latch = latch;
|
||||
this.splits.clear();
|
||||
}
|
||||
|
||||
public String getPartitionPaths() {
|
||||
return this.splits.stream()
|
||||
.map(TestUtils::getSplitPartitionPath)
|
||||
.sorted(Comparator.naturalOrder())
|
||||
.collect(Collectors.joining(","));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,287 @@
|
||||
/*
|
||||
* 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.source;
|
||||
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputFormat;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadTableState;
|
||||
import org.apache.hudi.util.AvroSchemaConverter;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
import org.apache.hudi.utils.TestData;
|
||||
import org.apache.hudi.utils.TestUtils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
|
||||
import org.apache.flink.streaming.api.TimeCharacteristic;
|
||||
import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
|
||||
import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor;
|
||||
import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction;
|
||||
import org.apache.flink.streaming.runtime.tasks.mailbox.SteppingMailboxProcessor;
|
||||
import org.apache.flink.streaming.util.CollectingSourceContext;
|
||||
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
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.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Test cases for {@link StreamReadOperator}.
|
||||
*/
|
||||
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]");
|
||||
EXPECTED.put("par3", "+I[id5, Sophia, 18, 1970-01-01T00:00:00.005, par3], +I[id6, Emma, 20, 1970-01-01T00:00:00.006, par3]");
|
||||
EXPECTED.put("par4", "+I[id7, Bob, 44, 1970-01-01T00:00:00.007, par4], +I[id8, Han, 56, 1970-01-01T00:00:00.008, par4]");
|
||||
}
|
||||
|
||||
private Configuration conf;
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@BeforeEach
|
||||
public void before() throws Exception {
|
||||
final String basePath = tempFile.getAbsolutePath();
|
||||
conf = TestConfigurations.getDefaultConf(basePath);
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testWriteRecords() throws Exception {
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
try (OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> harness = createReader()) {
|
||||
harness.setup();
|
||||
harness.open();
|
||||
|
||||
SteppingMailboxProcessor processor = createLocalMailbox(harness);
|
||||
StreamReadMonitoringFunction func = TestUtils.getMonitorFunc(conf);
|
||||
|
||||
List<MergeOnReadInputSplit> splits = generateSplits(func);
|
||||
assertThat("Should have 4 splits", splits.size(), is(4));
|
||||
for (MergeOnReadInputSplit split : splits) {
|
||||
// Process this element to enqueue to mail-box.
|
||||
harness.processElement(split, -1);
|
||||
|
||||
// Run the mail-box once to read all records from the given split.
|
||||
assertThat("Should process 1 split", processor.runMailboxStep());
|
||||
}
|
||||
// Assert the output has expected elements.
|
||||
TestData.assertRowDataEquals(harness.extractOutputValues(), TestData.DATA_SET_INSERT);
|
||||
|
||||
TestData.writeData(TestData.DATA_SET_UPDATE_INSERT, conf);
|
||||
final List<MergeOnReadInputSplit> splits2 = generateSplits(func);
|
||||
assertThat("Should have 4 splits", splits2.size(), is(4));
|
||||
for (MergeOnReadInputSplit split : splits2) {
|
||||
// Process this element to enqueue to mail-box.
|
||||
harness.processElement(split, -1);
|
||||
|
||||
// Run the mail-box once to read all records from the given split.
|
||||
assertThat("Should processed 1 split", processor.runMailboxStep());
|
||||
}
|
||||
// The result sets behaves like append only: DATA_SET_ONE + DATA_SET_TWO
|
||||
List<RowData> expected = new ArrayList<>(TestData.DATA_SET_INSERT);
|
||||
expected.addAll(TestData.DATA_SET_UPDATE_INSERT);
|
||||
TestData.assertRowDataEquals(harness.extractOutputValues(), expected);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckpoint() throws Exception {
|
||||
// Received emitted splits: split1, split2, split3, split4, checkpoint request is triggered
|
||||
// when reading records from split1.
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
long timestamp = 0;
|
||||
try (OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> harness = createReader()) {
|
||||
harness.setup();
|
||||
harness.open();
|
||||
|
||||
SteppingMailboxProcessor processor = createLocalMailbox(harness);
|
||||
StreamReadMonitoringFunction func = TestUtils.getMonitorFunc(conf);
|
||||
|
||||
List<MergeOnReadInputSplit> splits = generateSplits(func);
|
||||
assertThat("Should have 4 splits", splits.size(), is(4));
|
||||
|
||||
for (MergeOnReadInputSplit split : splits) {
|
||||
harness.processElement(split, ++timestamp);
|
||||
}
|
||||
|
||||
// Trigger snapshot state, it will start to work once all records from split0 are read.
|
||||
processor.getMainMailboxExecutor()
|
||||
.execute(() -> harness.snapshot(1, 3), "Trigger snapshot");
|
||||
|
||||
assertTrue(processor.runMailboxStep(), "Should have processed the split0");
|
||||
assertTrue(processor.runMailboxStep(), "Should have processed the snapshot state action");
|
||||
|
||||
assertThat(TestData.rowDataToString(harness.extractOutputValues()),
|
||||
is(getSplitExpected(Collections.singletonList(splits.get(0)), EXPECTED)));
|
||||
|
||||
// Read records from split1.
|
||||
assertTrue(processor.runMailboxStep(), "Should have processed the split1");
|
||||
|
||||
// Read records from split2.
|
||||
assertTrue(processor.runMailboxStep(), "Should have processed the split2");
|
||||
|
||||
// Read records from split3.
|
||||
assertTrue(processor.runMailboxStep(), "Should have processed the split3");
|
||||
|
||||
// Assert the output has expected elements.
|
||||
TestData.assertRowDataEquals(harness.extractOutputValues(), TestData.DATA_SET_INSERT);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckpointRestore() throws Exception {
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
|
||||
OperatorSubtaskState state;
|
||||
final List<MergeOnReadInputSplit> splits;
|
||||
try (OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> harness = createReader()) {
|
||||
harness.setup();
|
||||
harness.open();
|
||||
|
||||
StreamReadMonitoringFunction func = TestUtils.getMonitorFunc(conf);
|
||||
|
||||
splits = generateSplits(func);
|
||||
assertThat("Should have 4 splits", splits.size(), is(4));
|
||||
|
||||
// Enqueue all the splits.
|
||||
for (MergeOnReadInputSplit split : splits) {
|
||||
harness.processElement(split, -1);
|
||||
}
|
||||
|
||||
// Read all records from the first 2 splits.
|
||||
SteppingMailboxProcessor localMailbox = createLocalMailbox(harness);
|
||||
for (int i = 0; i < 2; i++) {
|
||||
assertTrue(localMailbox.runMailboxStep(), "Should have processed the split#" + i);
|
||||
}
|
||||
|
||||
assertThat(TestData.rowDataToString(harness.extractOutputValues()),
|
||||
is(getSplitExpected(splits.subList(0, 2), EXPECTED)));
|
||||
|
||||
// Snapshot state now, there are 2 splits left in the state.
|
||||
state = harness.snapshot(1, 1);
|
||||
}
|
||||
|
||||
try (OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> harness = createReader()) {
|
||||
harness.setup();
|
||||
// Recover to process the remaining splits.
|
||||
harness.initializeState(state);
|
||||
harness.open();
|
||||
|
||||
SteppingMailboxProcessor localMailbox = createLocalMailbox(harness);
|
||||
|
||||
for (int i = 2; i < 4; i++) {
|
||||
assertTrue(localMailbox.runMailboxStep(), "Should have processed one split#" + i);
|
||||
}
|
||||
|
||||
// expect to output the left data
|
||||
assertThat(TestData.rowDataToString(harness.extractOutputValues()),
|
||||
is(getSplitExpected(splits.subList(2, 4), EXPECTED)));
|
||||
}
|
||||
}
|
||||
|
||||
private static String getSplitExpected(List<MergeOnReadInputSplit> splits, Map<String, String> expected) {
|
||||
return splits.stream()
|
||||
.map(TestUtils::getSplitPartitionPath)
|
||||
.map(expected::get)
|
||||
.sorted(Comparator.naturalOrder())
|
||||
.collect(Collectors.toList()).toString();
|
||||
}
|
||||
|
||||
private List<MergeOnReadInputSplit> generateSplits(StreamReadMonitoringFunction func) throws Exception {
|
||||
final List<MergeOnReadInputSplit> splits = new ArrayList<>();
|
||||
func.open(conf);
|
||||
func.monitorDirAndForwardSplits(new CollectingSourceContext<>(new Object(), splits));
|
||||
return splits;
|
||||
}
|
||||
|
||||
private OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> createReader() throws Exception {
|
||||
final String basePath = tempFile.getAbsolutePath();
|
||||
final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(hadoopConf).setBasePath(basePath).build();
|
||||
final List<String> partitionKeys = Collections.singletonList("partition");
|
||||
|
||||
// This input format is used to opening the emitted split.
|
||||
TableSchemaResolver schemaResolver = new TableSchemaResolver(metaClient);
|
||||
final Schema tableAvroSchema;
|
||||
try {
|
||||
tableAvroSchema = schemaResolver.getTableAvroSchema();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Get table avro schema error", e);
|
||||
}
|
||||
final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema);
|
||||
final RowType rowType = (RowType) rowDataType.getLogicalType();
|
||||
final MergeOnReadTableState hoodieTableState = new MergeOnReadTableState(
|
||||
rowType,
|
||||
TestConfigurations.ROW_TYPE,
|
||||
tableAvroSchema.toString(),
|
||||
AvroSchemaConverter.convertToSchema(TestConfigurations.ROW_TYPE).toString(),
|
||||
Collections.emptyList(),
|
||||
new String[0]);
|
||||
MergeOnReadInputFormat inputFormat = MergeOnReadInputFormat.builder()
|
||||
.config(conf)
|
||||
.tableState(hoodieTableState)
|
||||
.fieldTypes(rowDataType.getChildren())
|
||||
.defaultPartName("default").limit(1000L)
|
||||
.emitDelete(true)
|
||||
.build();
|
||||
|
||||
OneInputStreamOperatorFactory<MergeOnReadInputSplit, RowData> factory = StreamReadOperator.factory(inputFormat);
|
||||
OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> harness = new OneInputStreamOperatorTestHarness<>(
|
||||
factory, 1, 1, 0);
|
||||
harness.getStreamConfig().setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
|
||||
|
||||
return harness;
|
||||
}
|
||||
|
||||
private SteppingMailboxProcessor createLocalMailbox(
|
||||
OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> harness) {
|
||||
return new SteppingMailboxProcessor(
|
||||
MailboxDefaultAction.Controller::suspendDefaultAction,
|
||||
harness.getTaskMailbox(),
|
||||
StreamTaskActionExecutor.IMMEDIATE);
|
||||
}
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@@ -0,0 +1,516 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import org.apache.hudi.common.model.DefaultHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.EventTimeAvroPayload;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieValidationException;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.keygen.ComplexAvroKeyGenerator;
|
||||
import org.apache.hudi.keygen.NonpartitionedAvroKeyGenerator;
|
||||
import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.SchemaBuilder;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.configuration.ReadableConfig;
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.api.Schema;
|
||||
import org.apache.flink.table.catalog.CatalogTable;
|
||||
import org.apache.flink.table.catalog.ObjectIdentifier;
|
||||
import org.apache.flink.table.catalog.ResolvedCatalogTable;
|
||||
import org.apache.flink.table.catalog.ResolvedSchema;
|
||||
import org.apache.flink.table.factories.DynamicTableFactory;
|
||||
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.List;
|
||||
import java.util.Objects;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
/**
|
||||
* Test cases for {@link HoodieTableFactory}.
|
||||
*/
|
||||
public class TestHoodieTableFactory {
|
||||
private static final String AVRO_SCHEMA_FILE_PATH = Objects.requireNonNull(Thread.currentThread()
|
||||
.getContextClassLoader().getResource("test_read_schema.avsc")).toString();
|
||||
private static final String INFERRED_SCHEMA = "{\"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}]}";
|
||||
|
||||
private Configuration conf;
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@BeforeEach
|
||||
void beforeEach() throws IOException {
|
||||
this.conf = new Configuration();
|
||||
this.conf.setString(FlinkOptions.PATH, tempFile.getAbsolutePath());
|
||||
this.conf.setString(FlinkOptions.TABLE_NAME, "t1");
|
||||
StreamerUtil.initTableIfNotExists(this.conf);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRequiredOptionsForSource() {
|
||||
// miss pk and precombine key will throw exception
|
||||
ResolvedSchema schema1 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.build();
|
||||
final MockContext sourceContext1 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSource(sourceContext1));
|
||||
assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSink(sourceContext1));
|
||||
|
||||
// a non-exists precombine key will throw exception
|
||||
ResolvedSchema schema2 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.build();
|
||||
this.conf.setString(FlinkOptions.PRECOMBINE_FIELD, "non_exist_field");
|
||||
final MockContext sourceContext2 = MockContext.getInstance(this.conf, schema2, "f2");
|
||||
assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSource(sourceContext2));
|
||||
assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSink(sourceContext2));
|
||||
this.conf.setString(FlinkOptions.PRECOMBINE_FIELD, FlinkOptions.PRECOMBINE_FIELD.defaultValue());
|
||||
|
||||
// given the pk but miss the pre combine key will be ok
|
||||
ResolvedSchema schema3 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0")
|
||||
.build();
|
||||
final MockContext sourceContext3 = MockContext.getInstance(this.conf, schema3, "f2");
|
||||
HoodieTableSource tableSource = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext3);
|
||||
HoodieTableSink tableSink = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sourceContext3);
|
||||
// the precombine field is overwritten
|
||||
assertThat(tableSource.getConf().getString(FlinkOptions.PRECOMBINE_FIELD), is(FlinkOptions.NO_PRE_COMBINE));
|
||||
assertThat(tableSink.getConf().getString(FlinkOptions.PRECOMBINE_FIELD), is(FlinkOptions.NO_PRE_COMBINE));
|
||||
// precombine field not specified, use the default payload clazz
|
||||
assertThat(tableSource.getConf().getString(FlinkOptions.PAYLOAD_CLASS_NAME), is(FlinkOptions.PAYLOAD_CLASS_NAME.defaultValue()));
|
||||
assertThat(tableSink.getConf().getString(FlinkOptions.PAYLOAD_CLASS_NAME), is(FlinkOptions.PAYLOAD_CLASS_NAME.defaultValue()));
|
||||
|
||||
// given pk but miss the pre combine key with DefaultHoodieRecordPayload should throw
|
||||
this.conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, DefaultHoodieRecordPayload.class.getName());
|
||||
final MockContext sourceContext4 = MockContext.getInstance(this.conf, schema3, "f2");
|
||||
|
||||
assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSource(sourceContext4));
|
||||
assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSink(sourceContext4));
|
||||
this.conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, FlinkOptions.PAYLOAD_CLASS_NAME.defaultValue());
|
||||
|
||||
// given pk and pre combine key will be ok
|
||||
ResolvedSchema schema4 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.field("ts", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0")
|
||||
.build();
|
||||
final MockContext sourceContext5 = MockContext.getInstance(this.conf, schema4, "f2");
|
||||
|
||||
assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext5));
|
||||
assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSink(sourceContext5));
|
||||
// precombine field specified(default ts), use DefaultHoodieRecordPayload as payload clazz
|
||||
HoodieTableSource tableSource5 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext5);
|
||||
HoodieTableSink tableSink5 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sourceContext5);
|
||||
assertThat(tableSource5.getConf().getString(FlinkOptions.PAYLOAD_CLASS_NAME), is(EventTimeAvroPayload.class.getName()));
|
||||
assertThat(tableSink5.getConf().getString(FlinkOptions.PAYLOAD_CLASS_NAME), is(EventTimeAvroPayload.class.getName()));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testInferAvroSchemaForSource() {
|
||||
// infer the schema if not specified
|
||||
final HoodieTableSource tableSource1 =
|
||||
(HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(MockContext.getInstance(this.conf));
|
||||
final Configuration conf1 = tableSource1.getConf();
|
||||
assertThat(conf1.get(FlinkOptions.SOURCE_AVRO_SCHEMA), is(INFERRED_SCHEMA));
|
||||
|
||||
// set up the explicit schema using the file path
|
||||
this.conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH, AVRO_SCHEMA_FILE_PATH);
|
||||
HoodieTableSource tableSource2 =
|
||||
(HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(MockContext.getInstance(this.conf));
|
||||
Configuration conf2 = tableSource2.getConf();
|
||||
assertNull(conf2.get(FlinkOptions.SOURCE_AVRO_SCHEMA), "expect schema string as null");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSetupHoodieKeyOptionsForSource() {
|
||||
this.conf.setString(FlinkOptions.RECORD_KEY_FIELD, "dummyField");
|
||||
this.conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, "dummyKeyGenClass");
|
||||
// definition with simple primary key and partition path
|
||||
ResolvedSchema schema1 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.BIGINT())
|
||||
.field("ts", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0")
|
||||
.build();
|
||||
final MockContext sourceContext1 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext1);
|
||||
final Configuration conf1 = tableSource1.getConf();
|
||||
assertThat(conf1.get(FlinkOptions.RECORD_KEY_FIELD), is("f0"));
|
||||
assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS_NAME), is("dummyKeyGenClass"));
|
||||
|
||||
// definition with complex primary keys and partition paths
|
||||
this.conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, FlinkOptions.KEYGEN_CLASS_NAME.defaultValue());
|
||||
ResolvedSchema schema2 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20).notNull())
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.field("ts", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0", "f1")
|
||||
.build();
|
||||
final MockContext sourceContext2 = MockContext.getInstance(this.conf, schema2, "f2");
|
||||
final HoodieTableSource tableSource2 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext2);
|
||||
final Configuration conf2 = tableSource2.getConf();
|
||||
assertThat(conf2.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
|
||||
assertThat(conf2.get(FlinkOptions.KEYGEN_CLASS_NAME), is(ComplexAvroKeyGenerator.class.getName()));
|
||||
|
||||
// definition with complex primary keys and empty partition paths
|
||||
this.conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, FlinkOptions.KEYGEN_CLASS_NAME.defaultValue());
|
||||
final MockContext sourceContext3 = MockContext.getInstance(this.conf, schema2, "");
|
||||
final HoodieTableSource tableSource3 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext3);
|
||||
final Configuration conf3 = tableSource3.getConf();
|
||||
assertThat(conf3.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
|
||||
assertThat(conf3.get(FlinkOptions.KEYGEN_CLASS_NAME), is(NonpartitionedAvroKeyGenerator.class.getName()));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSetupHiveOptionsForSource() {
|
||||
// definition with simple primary key and partition path
|
||||
ResolvedSchema schema1 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.field("ts", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0")
|
||||
.build();
|
||||
|
||||
final MockContext sourceContext1 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext1);
|
||||
final Configuration conf1 = tableSource1.getConf();
|
||||
assertThat(conf1.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME), is(MultiPartKeysValueExtractor.class.getName()));
|
||||
|
||||
// set up hive style partitioning is true.
|
||||
this.conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, true);
|
||||
|
||||
final MockContext sourceContext2 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSource tableSource2 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext2);
|
||||
final Configuration conf2 = tableSource2.getConf();
|
||||
assertThat(conf2.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME), is(SlashEncodedDayPartitionValueExtractor.class.getName()));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSetupCleaningOptionsForSource() {
|
||||
// definition with simple primary key and partition path
|
||||
ResolvedSchema schema1 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.field("ts", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0")
|
||||
.build();
|
||||
// set up new retains commits that is less than min archive commits
|
||||
this.conf.setString(FlinkOptions.CLEAN_RETAIN_COMMITS.key(), "11");
|
||||
|
||||
final MockContext sourceContext1 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext1);
|
||||
final Configuration conf1 = tableSource1.getConf();
|
||||
assertThat(conf1.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), is(FlinkOptions.ARCHIVE_MIN_COMMITS.defaultValue()));
|
||||
assertThat(conf1.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(FlinkOptions.ARCHIVE_MAX_COMMITS.defaultValue()));
|
||||
|
||||
// set up new retains commits that is greater than min archive commits
|
||||
final int retainCommits = FlinkOptions.ARCHIVE_MIN_COMMITS.defaultValue() + 5;
|
||||
this.conf.setInteger(FlinkOptions.CLEAN_RETAIN_COMMITS.key(), retainCommits);
|
||||
|
||||
final MockContext sourceContext2 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSource tableSource2 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext2);
|
||||
final Configuration conf2 = tableSource2.getConf();
|
||||
assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), is(retainCommits + 10));
|
||||
assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(retainCommits + 20));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSetupReadOptionsForSource() {
|
||||
// definition with simple primary key and partition path
|
||||
ResolvedSchema schema1 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.field("ts", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0")
|
||||
.build();
|
||||
// set up new retains commits that is less than min archive commits
|
||||
this.conf.setString(FlinkOptions.READ_END_COMMIT, "123");
|
||||
|
||||
final MockContext sourceContext1 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext1);
|
||||
final Configuration conf1 = tableSource1.getConf();
|
||||
assertThat(conf1.getString(FlinkOptions.QUERY_TYPE), is(FlinkOptions.QUERY_TYPE_INCREMENTAL));
|
||||
|
||||
this.conf.removeConfig(FlinkOptions.READ_END_COMMIT);
|
||||
this.conf.setString(FlinkOptions.READ_START_COMMIT, "123");
|
||||
final MockContext sourceContext2 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSource tableSource2 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext2);
|
||||
final Configuration conf2 = tableSource2.getConf();
|
||||
assertThat(conf2.getString(FlinkOptions.QUERY_TYPE), is(FlinkOptions.QUERY_TYPE_INCREMENTAL));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testInferAvroSchemaForSink() {
|
||||
// infer the schema if not specified
|
||||
final HoodieTableSink tableSink1 =
|
||||
(HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(MockContext.getInstance(this.conf));
|
||||
final Configuration conf1 = tableSink1.getConf();
|
||||
assertThat(conf1.get(FlinkOptions.SOURCE_AVRO_SCHEMA), is(INFERRED_SCHEMA));
|
||||
|
||||
// set up the explicit schema using the file path
|
||||
this.conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH, AVRO_SCHEMA_FILE_PATH);
|
||||
HoodieTableSink tableSink2 =
|
||||
(HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(MockContext.getInstance(this.conf));
|
||||
Configuration conf2 = tableSink2.getConf();
|
||||
assertNull(conf2.get(FlinkOptions.SOURCE_AVRO_SCHEMA), "expect schema string as null");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSetupHoodieKeyOptionsForSink() {
|
||||
this.conf.setString(FlinkOptions.RECORD_KEY_FIELD, "dummyField");
|
||||
this.conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, "dummyKeyGenClass");
|
||||
// definition with simple primary key and partition path
|
||||
ResolvedSchema schema1 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.BIGINT())
|
||||
.field("ts", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0")
|
||||
.build();
|
||||
final MockContext sinkContext1 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSink tableSink1 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext1);
|
||||
final Configuration conf1 = tableSink1.getConf();
|
||||
assertThat(conf1.get(FlinkOptions.RECORD_KEY_FIELD), is("f0"));
|
||||
assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS_NAME), is("dummyKeyGenClass"));
|
||||
|
||||
// definition with complex primary keys and partition paths
|
||||
this.conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, FlinkOptions.KEYGEN_CLASS_NAME.defaultValue());
|
||||
ResolvedSchema schema2 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20).notNull())
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.field("ts", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0", "f1")
|
||||
.build();
|
||||
final MockContext sinkContext2 = MockContext.getInstance(this.conf, schema2, "f2");
|
||||
final HoodieTableSink tableSink2 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext2);
|
||||
final Configuration conf2 = tableSink2.getConf();
|
||||
assertThat(conf2.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
|
||||
assertThat(conf2.get(FlinkOptions.KEYGEN_CLASS_NAME), is(ComplexAvroKeyGenerator.class.getName()));
|
||||
|
||||
// definition with complex primary keys and empty partition paths
|
||||
this.conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, FlinkOptions.KEYGEN_CLASS_NAME.defaultValue());
|
||||
final MockContext sinkContext3 = MockContext.getInstance(this.conf, schema2, "");
|
||||
final HoodieTableSink tableSink3 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext3);
|
||||
final Configuration conf3 = tableSink3.getConf();
|
||||
assertThat(conf3.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
|
||||
assertThat(conf3.get(FlinkOptions.KEYGEN_CLASS_NAME), is(NonpartitionedAvroKeyGenerator.class.getName()));
|
||||
|
||||
// definition of bucket index
|
||||
this.conf.setString(FlinkOptions.INDEX_TYPE, HoodieIndex.IndexType.BUCKET.name());
|
||||
final MockContext sinkContext4 = MockContext.getInstance(this.conf, schema2, "");
|
||||
final HoodieTableSink tableSink4 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext4);
|
||||
final Configuration conf4 = tableSink4.getConf();
|
||||
assertThat(conf4.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
|
||||
assertThat(conf4.get(FlinkOptions.INDEX_KEY_FIELD), is("f0,f1"));
|
||||
assertThat(conf4.get(FlinkOptions.INDEX_TYPE), is(HoodieIndex.IndexType.BUCKET.name()));
|
||||
assertThat(conf4.get(FlinkOptions.KEYGEN_CLASS_NAME), is(NonpartitionedAvroKeyGenerator.class.getName()));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSetupHiveOptionsForSink() {
|
||||
// definition with simple primary key and partition path
|
||||
ResolvedSchema schema1 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.field("ts", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0")
|
||||
.build();
|
||||
|
||||
final MockContext sinkContext1 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSink tableSink1 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext1);
|
||||
final Configuration conf1 = tableSink1.getConf();
|
||||
assertThat(conf1.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME), is(MultiPartKeysValueExtractor.class.getName()));
|
||||
|
||||
// set up hive style partitioning is true.
|
||||
this.conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, true);
|
||||
|
||||
final MockContext sinkContext2 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSink tableSink2 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext2);
|
||||
final Configuration conf2 = tableSink2.getConf();
|
||||
assertThat(conf2.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME), is(SlashEncodedDayPartitionValueExtractor.class.getName()));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSetupCleaningOptionsForSink() {
|
||||
// definition with simple primary key and partition path
|
||||
ResolvedSchema schema1 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.field("ts", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0")
|
||||
.build();
|
||||
// set up new retains commits that is less than min archive commits
|
||||
this.conf.setString(FlinkOptions.CLEAN_RETAIN_COMMITS.key(), "11");
|
||||
|
||||
final MockContext sinkContext1 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSink tableSink1 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext1);
|
||||
final Configuration conf1 = tableSink1.getConf();
|
||||
assertThat(conf1.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), is(FlinkOptions.ARCHIVE_MIN_COMMITS.defaultValue()));
|
||||
assertThat(conf1.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(FlinkOptions.ARCHIVE_MAX_COMMITS.defaultValue()));
|
||||
|
||||
// set up new retains commits that is greater than min archive commits
|
||||
final int retainCommits = FlinkOptions.ARCHIVE_MIN_COMMITS.defaultValue() + 5;
|
||||
this.conf.setInteger(FlinkOptions.CLEAN_RETAIN_COMMITS.key(), retainCommits);
|
||||
|
||||
final MockContext sinkContext2 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSink tableSink2 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext2);
|
||||
final Configuration conf2 = tableSink2.getConf();
|
||||
assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), is(retainCommits + 10));
|
||||
assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(retainCommits + 20));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSetupTimestampBasedKeyGenForSink() {
|
||||
this.conf.setString(FlinkOptions.RECORD_KEY_FIELD, "dummyField");
|
||||
this.conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, "dummyKeyGenClass");
|
||||
// definition with simple primary key and partition path
|
||||
ResolvedSchema schema1 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.field("ts", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0")
|
||||
.build();
|
||||
final MockContext sourceContext1 = MockContext.getInstance(this.conf, schema1, "ts");
|
||||
final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext1);
|
||||
final Configuration conf1 = tableSource1.getConf();
|
||||
assertThat(conf1.get(FlinkOptions.RECORD_KEY_FIELD), is("f0"));
|
||||
assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS_NAME), is(TimestampBasedAvroKeyGenerator.class.getName()));
|
||||
assertThat(conf1.getString(KeyGeneratorOptions.Config.TIMESTAMP_TYPE_FIELD_PROP, "dummy"),
|
||||
is("EPOCHMILLISECONDS"));
|
||||
assertThat(conf1.getString(KeyGeneratorOptions.Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "dummy"),
|
||||
is(FlinkOptions.PARTITION_FORMAT_HOUR));
|
||||
assertThat(conf1.getString(KeyGeneratorOptions.Config.TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP, "dummy"),
|
||||
is("UTC"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSetupWriteOptionsForSink() {
|
||||
final HoodieTableSink tableSink1 =
|
||||
(HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(MockContext.getInstance(this.conf));
|
||||
final Configuration conf1 = tableSink1.getConf();
|
||||
assertThat(conf1.get(FlinkOptions.PRE_COMBINE), is(true));
|
||||
|
||||
// set up operation as 'insert'
|
||||
this.conf.setString(FlinkOptions.OPERATION, "insert");
|
||||
HoodieTableSink tableSink2 =
|
||||
(HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(MockContext.getInstance(this.conf));
|
||||
Configuration conf2 = tableSink2.getConf();
|
||||
assertThat(conf2.get(FlinkOptions.PRE_COMBINE), is(false));
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Inner Class
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Mock dynamic table factory context.
|
||||
*/
|
||||
private static class MockContext implements DynamicTableFactory.Context {
|
||||
private final Configuration conf;
|
||||
private final ResolvedSchema schema;
|
||||
private final List<String> partitions;
|
||||
|
||||
private MockContext(Configuration conf, ResolvedSchema schema, List<String> partitions) {
|
||||
this.conf = conf;
|
||||
this.schema = schema;
|
||||
this.partitions = partitions;
|
||||
}
|
||||
|
||||
static MockContext getInstance(Configuration conf) {
|
||||
return getInstance(conf, TestConfigurations.TABLE_SCHEMA, Collections.singletonList("partition"));
|
||||
}
|
||||
|
||||
static MockContext getInstance(Configuration conf, ResolvedSchema schema, String partition) {
|
||||
return getInstance(conf, schema, Collections.singletonList(partition));
|
||||
}
|
||||
|
||||
static MockContext getInstance(Configuration conf, ResolvedSchema schema, List<String> partitions) {
|
||||
return new MockContext(conf, schema, partitions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ObjectIdentifier getObjectIdentifier() {
|
||||
return ObjectIdentifier.of("hudi", "default", "t1");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResolvedCatalogTable getCatalogTable() {
|
||||
CatalogTable catalogTable = CatalogTable.of(Schema.newBuilder().fromResolvedSchema(schema).build(),
|
||||
"mock source table", partitions, conf.toMap());
|
||||
return new ResolvedCatalogTable(catalogTable, schema);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableConfig getConfiguration() {
|
||||
return conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClassLoader getClassLoader() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isTemporary() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,144 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputFormat;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
import org.apache.hudi.utils.TestData;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.flink.api.common.io.FileInputFormat;
|
||||
import org.apache.flink.api.common.io.InputFormat;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.function.ThrowingSupplier;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.instanceOf;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.core.Is.is;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
|
||||
/**
|
||||
* Test cases for HoodieTableSource.
|
||||
*/
|
||||
public class TestHoodieTableSource {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TestHoodieTableSource.class);
|
||||
|
||||
private Configuration conf;
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
void beforeEach() throws Exception {
|
||||
final String path = tempFile.getAbsolutePath();
|
||||
conf = TestConfigurations.getDefaultConf(path);
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGetReadPaths() throws Exception {
|
||||
beforeEach();
|
||||
HoodieTableSource tableSource = new HoodieTableSource(
|
||||
TestConfigurations.TABLE_SCHEMA,
|
||||
new Path(tempFile.getPath()),
|
||||
Arrays.asList(conf.getString(FlinkOptions.PARTITION_PATH_FIELD).split(",")),
|
||||
"default-par",
|
||||
conf);
|
||||
Path[] paths = tableSource.getReadPaths();
|
||||
assertNotNull(paths);
|
||||
String[] names = Arrays.stream(paths).map(Path::getName)
|
||||
.sorted(Comparator.naturalOrder()).toArray(String[]::new);
|
||||
assertThat(Arrays.toString(names), is("[par1, par2, par3, par4]"));
|
||||
// apply partition pruning
|
||||
Map<String, String> partitions = new HashMap<>();
|
||||
partitions.put("partition", "par1");
|
||||
|
||||
tableSource.applyPartitions(Collections.singletonList(partitions));
|
||||
|
||||
Path[] paths2 = tableSource.getReadPaths();
|
||||
assertNotNull(paths2);
|
||||
String[] names2 = Arrays.stream(paths2).map(Path::getName)
|
||||
.sorted(Comparator.naturalOrder()).toArray(String[]::new);
|
||||
assertThat(Arrays.toString(names2), is("[par1]"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGetInputFormat() throws Exception {
|
||||
beforeEach();
|
||||
// write some data to let the TableSchemaResolver get the right instant
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
|
||||
HoodieTableSource tableSource = new HoodieTableSource(
|
||||
TestConfigurations.TABLE_SCHEMA,
|
||||
new Path(tempFile.getPath()),
|
||||
Arrays.asList(conf.getString(FlinkOptions.PARTITION_PATH_FIELD).split(",")),
|
||||
"default-par",
|
||||
conf);
|
||||
InputFormat<RowData, ?> inputFormat = tableSource.getInputFormat();
|
||||
assertThat(inputFormat, is(instanceOf(FileInputFormat.class)));
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
inputFormat = tableSource.getInputFormat();
|
||||
assertThat(inputFormat, is(instanceOf(MergeOnReadInputFormat.class)));
|
||||
conf.setString(FlinkOptions.QUERY_TYPE.key(), FlinkOptions.QUERY_TYPE_INCREMENTAL);
|
||||
assertDoesNotThrow(
|
||||
(ThrowingSupplier<? extends InputFormat<RowData, ?>>) tableSource::getInputFormat,
|
||||
"Query type: 'incremental' should be supported");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGetTableAvroSchema() {
|
||||
final String path = tempFile.getAbsolutePath();
|
||||
conf = TestConfigurations.getDefaultConf(path);
|
||||
conf.setBoolean(FlinkOptions.READ_AS_STREAMING, true);
|
||||
|
||||
HoodieTableSource tableSource = new HoodieTableSource(
|
||||
TestConfigurations.TABLE_SCHEMA,
|
||||
new Path(tempFile.getPath()),
|
||||
Arrays.asList(conf.getString(FlinkOptions.PARTITION_PATH_FIELD).split(",")),
|
||||
"default-par",
|
||||
conf);
|
||||
assertNull(tableSource.getMetaClient(), "Streaming source with empty table path is allowed");
|
||||
final String schemaFields = tableSource.getTableAvroSchema().getFields().stream()
|
||||
.map(Schema.Field::name)
|
||||
.collect(Collectors.joining(","));
|
||||
final String expected = "_hoodie_commit_time,"
|
||||
+ "_hoodie_commit_seqno,"
|
||||
+ "_hoodie_record_key,"
|
||||
+ "_hoodie_partition_path,"
|
||||
+ "_hoodie_file_name,"
|
||||
+ "uuid,name,age,ts,partition";
|
||||
assertThat(schemaFields, is(expected));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,264 @@
|
||||
/*
|
||||
* 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.table.catalog;
|
||||
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.api.EnvironmentSettings;
|
||||
import org.apache.flink.table.api.Schema;
|
||||
import org.apache.flink.table.api.TableEnvironment;
|
||||
import org.apache.flink.table.api.config.ExecutionConfigOptions;
|
||||
import org.apache.flink.table.api.internal.TableEnvironmentImpl;
|
||||
import org.apache.flink.table.catalog.CatalogBaseTable;
|
||||
import org.apache.flink.table.catalog.CatalogDatabase;
|
||||
import org.apache.flink.table.catalog.CatalogDatabaseImpl;
|
||||
import org.apache.flink.table.catalog.CatalogTable;
|
||||
import org.apache.flink.table.catalog.Column;
|
||||
import org.apache.flink.table.catalog.ObjectPath;
|
||||
import org.apache.flink.table.catalog.ResolvedCatalogTable;
|
||||
import org.apache.flink.table.catalog.ResolvedSchema;
|
||||
import org.apache.flink.table.catalog.UniqueConstraint;
|
||||
import org.apache.flink.table.catalog.exceptions.CatalogException;
|
||||
import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
|
||||
import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
|
||||
import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
|
||||
import org.apache.flink.table.catalog.exceptions.TableNotExistException;
|
||||
import org.apache.flink.table.types.logical.LogicalTypeRoot;
|
||||
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.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.table.catalog.CatalogOptions.CATALOG_PATH;
|
||||
import static org.apache.hudi.table.catalog.CatalogOptions.DEFAULT_DATABASE;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Test cases for {@link HoodieCatalog}.
|
||||
*/
|
||||
public class TestHoodieCatalog {
|
||||
|
||||
private static final String TEST_DEFAULT_DATABASE = "test_db";
|
||||
private static final String NONE_EXIST_DATABASE = "none_exist_database";
|
||||
private static final List<Column> CREATE_COLUMNS = Arrays.asList(
|
||||
Column.physical("uuid", DataTypes.VARCHAR(20)),
|
||||
Column.physical("name", DataTypes.VARCHAR(20)),
|
||||
Column.physical("age", DataTypes.INT()),
|
||||
Column.physical("tss", DataTypes.TIMESTAMP(3)),
|
||||
Column.physical("partition", DataTypes.VARCHAR(10))
|
||||
);
|
||||
private static final UniqueConstraint CONSTRAINTS = UniqueConstraint.primaryKey("uuid", Arrays.asList("uuid"));
|
||||
private static final ResolvedSchema CREATE_TABLE_SCHEMA =
|
||||
new ResolvedSchema(
|
||||
CREATE_COLUMNS,
|
||||
Collections.emptyList(),
|
||||
CONSTRAINTS);
|
||||
|
||||
private static final List<Column> EXPECTED_TABLE_COLUMNS =
|
||||
CREATE_COLUMNS.stream()
|
||||
.map(
|
||||
col -> {
|
||||
// Flink char/varchar is transform to string in avro.
|
||||
if (col.getDataType()
|
||||
.getLogicalType()
|
||||
.getTypeRoot()
|
||||
.equals(LogicalTypeRoot.VARCHAR)) {
|
||||
return Column.physical(col.getName(), DataTypes.STRING());
|
||||
} else {
|
||||
return col;
|
||||
}
|
||||
})
|
||||
.collect(Collectors.toList());
|
||||
private static final ResolvedSchema EXPECTED_TABLE_SCHEMA =
|
||||
new ResolvedSchema(EXPECTED_TABLE_COLUMNS, Collections.emptyList(), CONSTRAINTS);
|
||||
|
||||
private static final Map<String, String> EXPECTED_OPTIONS = new HashMap<>();
|
||||
|
||||
static {
|
||||
EXPECTED_OPTIONS.put(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
EXPECTED_OPTIONS.put(FlinkOptions.INDEX_GLOBAL_ENABLED.key(), "false");
|
||||
EXPECTED_OPTIONS.put(FlinkOptions.PRE_COMBINE.key(), "true");
|
||||
}
|
||||
|
||||
private static final ResolvedCatalogTable EXPECTED_CATALOG_TABLE = new ResolvedCatalogTable(
|
||||
CatalogTable.of(
|
||||
Schema.newBuilder().fromResolvedSchema(CREATE_TABLE_SCHEMA).build(),
|
||||
"test",
|
||||
Arrays.asList("partition"),
|
||||
EXPECTED_OPTIONS),
|
||||
CREATE_TABLE_SCHEMA
|
||||
);
|
||||
|
||||
private TableEnvironment streamTableEnv;
|
||||
private HoodieCatalog catalog;
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@BeforeEach
|
||||
void beforeEach() {
|
||||
EnvironmentSettings settings = EnvironmentSettings.newInstance().build();
|
||||
streamTableEnv = TableEnvironmentImpl.create(settings);
|
||||
streamTableEnv.getConfig().getConfiguration()
|
||||
.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2);
|
||||
File testDb = new File(tempFile, TEST_DEFAULT_DATABASE);
|
||||
testDb.mkdir();
|
||||
Map<String, String> catalogOptions = new HashMap<>();
|
||||
catalogOptions.put(CATALOG_PATH.key(), tempFile.getAbsolutePath());
|
||||
catalogOptions.put(DEFAULT_DATABASE.key(), TEST_DEFAULT_DATABASE);
|
||||
catalog = new HoodieCatalog("hudi", Configuration.fromMap(catalogOptions));
|
||||
catalog.open();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testListDatabases() {
|
||||
List<String> actual = catalog.listDatabases();
|
||||
assertTrue(actual.contains(TEST_DEFAULT_DATABASE));
|
||||
assertFalse(actual.contains(NONE_EXIST_DATABASE));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDatabaseExists() {
|
||||
assertTrue(catalog.databaseExists(TEST_DEFAULT_DATABASE));
|
||||
assertFalse(catalog.databaseExists(NONE_EXIST_DATABASE));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateAndDropDatabase() throws Exception {
|
||||
CatalogDatabase expected = new CatalogDatabaseImpl(Collections.emptyMap(), null);
|
||||
catalog.createDatabase("db1", expected, true);
|
||||
|
||||
CatalogDatabase actual = catalog.getDatabase("db1");
|
||||
assertTrue(catalog.listDatabases().contains("db1"));
|
||||
assertEquals(expected.getProperties(), actual.getProperties());
|
||||
|
||||
// create exist database
|
||||
assertThrows(DatabaseAlreadyExistException.class,
|
||||
() -> catalog.createDatabase("db1", expected, false));
|
||||
|
||||
// drop exist database
|
||||
catalog.dropDatabase("db1", true);
|
||||
assertFalse(catalog.listDatabases().contains("db1"));
|
||||
|
||||
// drop non-exist database
|
||||
assertThrows(DatabaseNotExistException.class,
|
||||
() -> catalog.dropDatabase(NONE_EXIST_DATABASE, false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateDatabaseWithOptions() {
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put("k1", "v1");
|
||||
options.put("k2", "v2");
|
||||
|
||||
assertThrows(
|
||||
CatalogException.class,
|
||||
() -> catalog.createDatabase("db1", new CatalogDatabaseImpl(options, null), true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateTable() throws Exception {
|
||||
ObjectPath tablePath = new ObjectPath(TEST_DEFAULT_DATABASE, "tb1");
|
||||
// test create table
|
||||
catalog.createTable(tablePath, EXPECTED_CATALOG_TABLE, true);
|
||||
|
||||
// test table exist
|
||||
assertTrue(catalog.tableExists(tablePath));
|
||||
|
||||
// test create exist table
|
||||
assertThrows(TableAlreadyExistException.class,
|
||||
() -> catalog.createTable(tablePath, EXPECTED_CATALOG_TABLE, false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testListTable() throws Exception {
|
||||
ObjectPath tablePath1 = new ObjectPath(TEST_DEFAULT_DATABASE, "tb1");
|
||||
ObjectPath tablePath2 = new ObjectPath(TEST_DEFAULT_DATABASE, "tb2");
|
||||
|
||||
// create table
|
||||
catalog.createTable(tablePath1, EXPECTED_CATALOG_TABLE, true);
|
||||
catalog.createTable(tablePath2, EXPECTED_CATALOG_TABLE, true);
|
||||
|
||||
// test list table
|
||||
List<String> tables = catalog.listTables(TEST_DEFAULT_DATABASE);
|
||||
assertTrue(tables.contains(tablePath1.getObjectName()));
|
||||
assertTrue(tables.contains(tablePath2.getObjectName()));
|
||||
|
||||
// test list non-exist database table
|
||||
assertThrows(DatabaseNotExistException.class,
|
||||
() -> catalog.listTables(NONE_EXIST_DATABASE));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetTable() throws Exception {
|
||||
ObjectPath tablePath = new ObjectPath(TEST_DEFAULT_DATABASE, "tb1");
|
||||
// create table
|
||||
catalog.createTable(tablePath, EXPECTED_CATALOG_TABLE, true);
|
||||
|
||||
Map<String, String> expectedOptions = new HashMap<>(EXPECTED_OPTIONS);
|
||||
expectedOptions.put(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
expectedOptions.put(FlinkOptions.INDEX_GLOBAL_ENABLED.key(), "false");
|
||||
expectedOptions.put(FlinkOptions.PRE_COMBINE.key(), "true");
|
||||
expectedOptions.put("connector", "hudi");
|
||||
expectedOptions.put(
|
||||
FlinkOptions.PATH.key(),
|
||||
String.format("%s/%s/%s", tempFile.getAbsolutePath(), tablePath.getDatabaseName(), tablePath.getObjectName()));
|
||||
|
||||
// test get table
|
||||
CatalogBaseTable actualTable = catalog.getTable(tablePath);
|
||||
// validate schema
|
||||
Schema actualSchema = actualTable.getUnresolvedSchema();
|
||||
Schema expectedSchema = Schema.newBuilder().fromResolvedSchema(EXPECTED_TABLE_SCHEMA).build();
|
||||
assertEquals(expectedSchema, actualSchema);
|
||||
// validate options
|
||||
Map<String, String> actualOptions = actualTable.getOptions();
|
||||
assertEquals(expectedOptions, actualOptions);
|
||||
// validate comment
|
||||
assertEquals(EXPECTED_CATALOG_TABLE.getComment(), actualTable.getComment());
|
||||
// validate partition key
|
||||
assertEquals(EXPECTED_CATALOG_TABLE.getPartitionKeys(), ((CatalogTable) actualTable).getPartitionKeys());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void dropTable() throws Exception {
|
||||
ObjectPath tablePath = new ObjectPath(TEST_DEFAULT_DATABASE, "tb1");
|
||||
// create table
|
||||
catalog.createTable(tablePath, EXPECTED_CATALOG_TABLE, true);
|
||||
|
||||
// test drop table
|
||||
catalog.dropTable(tablePath, true);
|
||||
assertFalse(catalog.tableExists(tablePath));
|
||||
|
||||
// drop non-exist table
|
||||
assertThrows(TableNotExistException.class,
|
||||
() -> catalog.dropTable(new ObjectPath(TEST_DEFAULT_DATABASE, "non_exist"), false));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,520 @@
|
||||
/*
|
||||
* 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.table.format;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.table.HoodieTableSource;
|
||||
import org.apache.hudi.table.format.cow.CopyOnWriteInputFormat;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputFormat;
|
||||
import org.apache.hudi.util.AvroSchemaConverter;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
import org.apache.hudi.utils.TestData;
|
||||
|
||||
import org.apache.flink.api.common.io.InputFormat;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.core.io.InputSplit;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.EnumSource;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.instanceOf;
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
|
||||
/**
|
||||
* Test cases for MergeOnReadInputFormat and ParquetInputFormat.
|
||||
*/
|
||||
public class TestInputFormat {
|
||||
|
||||
private HoodieTableSource tableSource;
|
||||
private Configuration conf;
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
void beforeEach(HoodieTableType tableType) throws IOException {
|
||||
beforeEach(tableType, Collections.emptyMap());
|
||||
}
|
||||
|
||||
void beforeEach(HoodieTableType tableType, Map<String, String> options) throws IOException {
|
||||
conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, tableType.name());
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false); // close the async compaction
|
||||
options.forEach((key, value) -> conf.setString(key, value));
|
||||
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
this.tableSource = getTableSource(conf);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = HoodieTableType.class)
|
||||
void testRead(HoodieTableType tableType) throws Exception {
|
||||
beforeEach(tableType);
|
||||
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
|
||||
InputFormat<RowData, ?> inputFormat = this.tableSource.getInputFormat();
|
||||
|
||||
List<RowData> result = readData(inputFormat);
|
||||
|
||||
String actual = TestData.rowDataToString(result);
|
||||
String expected = TestData.rowDataToString(TestData.DATA_SET_INSERT);
|
||||
assertThat(actual, is(expected));
|
||||
|
||||
// write another commit to read again
|
||||
TestData.writeData(TestData.DATA_SET_UPDATE_INSERT, conf);
|
||||
|
||||
// refresh the input format
|
||||
this.tableSource.reset();
|
||||
inputFormat = this.tableSource.getInputFormat();
|
||||
|
||||
result = readData(inputFormat);
|
||||
|
||||
actual = TestData.rowDataToString(result);
|
||||
expected = "["
|
||||
+ "+I[id1, Danny, 24, 1970-01-01T00:00:00.001, par1], "
|
||||
+ "+I[id2, Stephen, 34, 1970-01-01T00:00:00.002, par1], "
|
||||
+ "+I[id3, Julian, 54, 1970-01-01T00:00:00.003, par2], "
|
||||
+ "+I[id4, Fabian, 32, 1970-01-01T00:00:00.004, par2], "
|
||||
+ "+I[id5, Sophia, 18, 1970-01-01T00:00:00.005, par3], "
|
||||
+ "+I[id6, Emma, 20, 1970-01-01T00:00:00.006, par3], "
|
||||
+ "+I[id7, Bob, 44, 1970-01-01T00:00:00.007, par4], "
|
||||
+ "+I[id8, Han, 56, 1970-01-01T00:00:00.008, par4], "
|
||||
+ "+I[id9, Jane, 19, 1970-01-01T00:00:00.006, par3], "
|
||||
+ "+I[id10, Ella, 38, 1970-01-01T00:00:00.007, par4], "
|
||||
+ "+I[id11, Phoebe, 52, 1970-01-01T00:00:00.008, par4]]";
|
||||
assertThat(actual, is(expected));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReadBaseAndLogFiles() throws Exception {
|
||||
beforeEach(HoodieTableType.MERGE_ON_READ);
|
||||
|
||||
// write base first with compaction
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, true);
|
||||
conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, 1);
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
|
||||
InputFormat<RowData, ?> inputFormat = this.tableSource.getInputFormat();
|
||||
|
||||
List<RowData> result = readData(inputFormat);
|
||||
|
||||
String actual = TestData.rowDataToString(result);
|
||||
String expected = TestData.rowDataToString(TestData.DATA_SET_INSERT);
|
||||
assertThat(actual, is(expected));
|
||||
|
||||
// write another commit using logs and read again
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false);
|
||||
TestData.writeData(TestData.DATA_SET_UPDATE_INSERT, conf);
|
||||
|
||||
// write another commit using logs with separate partition
|
||||
// so the file group has only logs
|
||||
TestData.writeData(TestData.DATA_SET_INSERT_SEPARATE_PARTITION, conf);
|
||||
|
||||
// refresh the input format
|
||||
this.tableSource.reset();
|
||||
inputFormat = this.tableSource.getInputFormat();
|
||||
|
||||
result = readData(inputFormat);
|
||||
|
||||
actual = TestData.rowDataToString(result);
|
||||
expected = "["
|
||||
+ "+I[id1, Danny, 24, 1970-01-01T00:00:00.001, par1], "
|
||||
+ "+I[id2, Stephen, 34, 1970-01-01T00:00:00.002, par1], "
|
||||
+ "+I[id3, Julian, 54, 1970-01-01T00:00:00.003, par2], "
|
||||
+ "+I[id4, Fabian, 32, 1970-01-01T00:00:00.004, par2], "
|
||||
+ "+I[id5, Sophia, 18, 1970-01-01T00:00:00.005, par3], "
|
||||
+ "+I[id6, Emma, 20, 1970-01-01T00:00:00.006, par3], "
|
||||
+ "+I[id7, Bob, 44, 1970-01-01T00:00:00.007, par4], "
|
||||
+ "+I[id8, Han, 56, 1970-01-01T00:00:00.008, par4], "
|
||||
+ "+I[id9, Jane, 19, 1970-01-01T00:00:00.006, par3], "
|
||||
+ "+I[id10, Ella, 38, 1970-01-01T00:00:00.007, par4], "
|
||||
+ "+I[id11, Phoebe, 52, 1970-01-01T00:00:00.008, par4], "
|
||||
+ "+I[id12, Monica, 27, 1970-01-01T00:00:00.009, par5], "
|
||||
+ "+I[id13, Phoebe, 31, 1970-01-01T00:00:00.010, par5], "
|
||||
+ "+I[id14, Rachel, 52, 1970-01-01T00:00:00.011, par6], "
|
||||
+ "+I[id15, Ross, 29, 1970-01-01T00:00:00.012, par6]]";
|
||||
assertThat(actual, is(expected));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReadBaseAndLogFilesWithDeletes() throws Exception {
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.CHANGELOG_ENABLED.key(), "true");
|
||||
beforeEach(HoodieTableType.MERGE_ON_READ, options);
|
||||
|
||||
// write base first with compaction.
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, true);
|
||||
conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, 1);
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
|
||||
// write another commit using logs and read again.
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false);
|
||||
TestData.writeData(TestData.DATA_SET_UPDATE_DELETE, conf);
|
||||
|
||||
InputFormat<RowData, ?> inputFormat = this.tableSource.getInputFormat();
|
||||
assertThat(inputFormat, instanceOf(MergeOnReadInputFormat.class));
|
||||
|
||||
// when isEmitDelete is false.
|
||||
List<RowData> result1 = readData(inputFormat);
|
||||
|
||||
final String actual1 = TestData.rowDataToString(result1);
|
||||
final String expected1 = "["
|
||||
+ "+I[id1, Danny, 24, 1970-01-01T00:00:00.001, par1], "
|
||||
+ "+I[id2, Stephen, 34, 1970-01-01T00:00:00.002, par1], "
|
||||
+ "+I[id4, Fabian, 31, 1970-01-01T00:00:00.004, par2], "
|
||||
+ "+I[id6, Emma, 20, 1970-01-01T00:00:00.006, par3], "
|
||||
+ "+I[id7, Bob, 44, 1970-01-01T00:00:00.007, par4], "
|
||||
+ "+I[id8, Han, 56, 1970-01-01T00:00:00.008, par4]]";
|
||||
assertThat(actual1, is(expected1));
|
||||
|
||||
// refresh the input format and set isEmitDelete to true.
|
||||
this.tableSource.reset();
|
||||
inputFormat = this.tableSource.getInputFormat();
|
||||
((MergeOnReadInputFormat) inputFormat).isEmitDelete(true);
|
||||
|
||||
List<RowData> result2 = readData(inputFormat);
|
||||
|
||||
final String actual2 = TestData.rowDataToString(result2);
|
||||
final String expected2 = "["
|
||||
+ "+I[id1, Danny, 24, 1970-01-01T00:00:00.001, par1], "
|
||||
+ "+I[id2, Stephen, 34, 1970-01-01T00:00:00.002, par1], "
|
||||
+ "-D[id3, Julian, 53, 1970-01-01T00:00:00.003, par2], "
|
||||
+ "+I[id4, Fabian, 31, 1970-01-01T00:00:00.004, par2], "
|
||||
+ "-D[id5, Sophia, 18, 1970-01-01T00:00:00.005, par3], "
|
||||
+ "+I[id6, Emma, 20, 1970-01-01T00:00:00.006, par3], "
|
||||
+ "+I[id7, Bob, 44, 1970-01-01T00:00:00.007, par4], "
|
||||
+ "+I[id8, Han, 56, 1970-01-01T00:00:00.008, par4], "
|
||||
+ "-D[id9, Jane, 19, 1970-01-01T00:00:00.006, par3]]";
|
||||
assertThat(actual2, is(expected2));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
void testReadBaseAndLogFilesWithDisorderUpdateDelete(boolean compact) throws Exception {
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.CHANGELOG_ENABLED.key(), "true");
|
||||
beforeEach(HoodieTableType.MERGE_ON_READ, options);
|
||||
|
||||
// write base first with compaction.
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, true);
|
||||
conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, 1);
|
||||
TestData.writeData(TestData.DATA_SET_SINGLE_INSERT, conf);
|
||||
|
||||
// write another commit using logs and read again.
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, compact);
|
||||
TestData.writeData(TestData.DATA_SET_DISORDER_UPDATE_DELETE, conf);
|
||||
|
||||
InputFormat<RowData, ?> inputFormat = this.tableSource.getInputFormat();
|
||||
assertThat(inputFormat, instanceOf(MergeOnReadInputFormat.class));
|
||||
|
||||
// when isEmitDelete is false.
|
||||
List<RowData> result1 = readData(inputFormat);
|
||||
|
||||
final String rowKind = compact ? "I" : "U";
|
||||
final String expected = "[+" + rowKind + "[id1, Danny, 22, 1970-01-01T00:00:00.004, par1]]";
|
||||
|
||||
final String actual1 = TestData.rowDataToString(result1);
|
||||
assertThat(actual1, is(expected));
|
||||
|
||||
// refresh the input format and set isEmitDelete to true.
|
||||
this.tableSource.reset();
|
||||
inputFormat = this.tableSource.getInputFormat();
|
||||
((MergeOnReadInputFormat) inputFormat).isEmitDelete(true);
|
||||
|
||||
List<RowData> result2 = readData(inputFormat);
|
||||
|
||||
final String actual2 = TestData.rowDataToString(result2);
|
||||
assertThat(actual2, is(expected));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReadWithDeletesMOR() throws Exception {
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.CHANGELOG_ENABLED.key(), "true");
|
||||
beforeEach(HoodieTableType.MERGE_ON_READ, options);
|
||||
|
||||
// write another commit to read again
|
||||
TestData.writeData(TestData.DATA_SET_UPDATE_DELETE, conf);
|
||||
|
||||
InputFormat<RowData, ?> inputFormat = this.tableSource.getInputFormat();
|
||||
assertThat(inputFormat, instanceOf(MergeOnReadInputFormat.class));
|
||||
((MergeOnReadInputFormat) inputFormat).isEmitDelete(true);
|
||||
|
||||
List<RowData> result = readData(inputFormat);
|
||||
|
||||
final String actual = TestData.rowDataToString(result);
|
||||
final String expected = "["
|
||||
+ "+I[id1, Danny, 24, 1970-01-01T00:00:00.001, par1], "
|
||||
+ "+I[id2, Stephen, 34, 1970-01-01T00:00:00.002, par1], "
|
||||
+ "-D[id3, Julian, 53, 1970-01-01T00:00:00.003, par2], "
|
||||
+ "-D[id5, Sophia, 18, 1970-01-01T00:00:00.005, par3], "
|
||||
+ "-D[id9, Jane, 19, 1970-01-01T00:00:00.006, par3]]";
|
||||
assertThat(actual, is(expected));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReadWithDeletesCOW() throws Exception {
|
||||
beforeEach(HoodieTableType.COPY_ON_WRITE);
|
||||
|
||||
// write another commit to read again
|
||||
TestData.writeData(TestData.DATA_SET_UPDATE_DELETE, conf);
|
||||
|
||||
InputFormat<RowData, ?> inputFormat = this.tableSource.getInputFormat();
|
||||
assertThat(inputFormat, instanceOf(CopyOnWriteInputFormat.class));
|
||||
|
||||
List<RowData> result = readData(inputFormat);
|
||||
|
||||
final String actual = TestData.rowDataToString(result);
|
||||
final String expected = "["
|
||||
+ "+I[id1, Danny, 24, 1970-01-01T00:00:00.001, par1], "
|
||||
+ "+I[id2, Stephen, 34, 1970-01-01T00:00:00.002, par1]]";
|
||||
assertThat(actual, is(expected));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = HoodieTableType.class)
|
||||
void testReadWithPartitionPrune(HoodieTableType tableType) throws Exception {
|
||||
beforeEach(tableType);
|
||||
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
|
||||
Map<String, String> prunedPartitions = new HashMap<>();
|
||||
prunedPartitions.put("partition", "par1");
|
||||
// prune to only be with partition 'par1'
|
||||
tableSource.applyPartitions(Collections.singletonList(prunedPartitions));
|
||||
InputFormat<RowData, ?> inputFormat = tableSource.getInputFormat();
|
||||
|
||||
List<RowData> result = readData(inputFormat);
|
||||
|
||||
String actual = TestData.rowDataToString(result);
|
||||
String expected = "["
|
||||
+ "+I[id1, Danny, 23, 1970-01-01T00:00:00.001, par1], "
|
||||
+ "+I[id2, Stephen, 33, 1970-01-01T00:00:00.002, par1]]";
|
||||
assertThat(actual, is(expected));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReadChangesMergedMOR() throws Exception {
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.CHANGELOG_ENABLED.key(), "true");
|
||||
beforeEach(HoodieTableType.MERGE_ON_READ, options);
|
||||
|
||||
// write another commit to read again
|
||||
TestData.writeData(TestData.DATA_SET_INSERT_UPDATE_DELETE, conf);
|
||||
|
||||
InputFormat<RowData, ?> inputFormat = this.tableSource.getInputFormat();
|
||||
assertThat(inputFormat, instanceOf(MergeOnReadInputFormat.class));
|
||||
|
||||
List<RowData> result1 = readData(inputFormat);
|
||||
|
||||
final String actual1 = TestData.rowDataToString(result1);
|
||||
// the data set is merged when the data source is bounded.
|
||||
final String expected1 = "[]";
|
||||
assertThat(actual1, is(expected1));
|
||||
|
||||
// refresh the input format and set isEmitDelete to true.
|
||||
this.tableSource.reset();
|
||||
inputFormat = this.tableSource.getInputFormat();
|
||||
((MergeOnReadInputFormat) inputFormat).isEmitDelete(true);
|
||||
|
||||
List<RowData> result2 = readData(inputFormat);
|
||||
|
||||
final String actual2 = TestData.rowDataToString(result2);
|
||||
final String expected2 = "[-D[id1, Danny, 22, 1970-01-01T00:00:00.005, par1]]";
|
||||
assertThat(actual2, is(expected2));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testReadChangesUnMergedMOR() throws Exception {
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.CHANGELOG_ENABLED.key(), "true");
|
||||
options.put(FlinkOptions.READ_AS_STREAMING.key(), "true");
|
||||
beforeEach(HoodieTableType.MERGE_ON_READ, options);
|
||||
|
||||
// write another commit to read again
|
||||
TestData.writeData(TestData.DATA_SET_INSERT_UPDATE_DELETE, conf);
|
||||
|
||||
InputFormat<RowData, ?> inputFormat = this.tableSource.getInputFormat();
|
||||
assertThat(inputFormat, instanceOf(MergeOnReadInputFormat.class));
|
||||
|
||||
List<RowData> result = readData(inputFormat);
|
||||
|
||||
final String actual = TestData.rowDataToString(result);
|
||||
// the data set is merged when the data source is bounded.
|
||||
final String expected = "["
|
||||
+ "+I[id1, Danny, 19, 1970-01-01T00:00:00.001, par1], "
|
||||
+ "-U[id1, Danny, 19, 1970-01-01T00:00:00.001, par1], "
|
||||
+ "+U[id1, Danny, 20, 1970-01-01T00:00:00.002, par1], "
|
||||
+ "-U[id1, Danny, 20, 1970-01-01T00:00:00.002, par1], "
|
||||
+ "+U[id1, Danny, 21, 1970-01-01T00:00:00.003, par1], "
|
||||
+ "-U[id1, Danny, 21, 1970-01-01T00:00:00.003, par1], "
|
||||
+ "+U[id1, Danny, 22, 1970-01-01T00:00:00.004, par1], "
|
||||
+ "-D[id1, Danny, 22, 1970-01-01T00:00:00.005, par1]]";
|
||||
assertThat(actual, is(expected));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = HoodieTableType.class)
|
||||
void testReadIncrementally(HoodieTableType tableType) throws Exception {
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.QUERY_TYPE.key(), FlinkOptions.QUERY_TYPE_INCREMENTAL);
|
||||
beforeEach(tableType, options);
|
||||
|
||||
// write another commit to read again
|
||||
for (int i = 0; i < 6; i += 2) {
|
||||
List<RowData> dataset = TestData.dataSetInsert(i + 1, i + 2);
|
||||
TestData.writeData(dataset, conf);
|
||||
}
|
||||
|
||||
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(tempFile.getAbsolutePath());
|
||||
List<String> commits = metaClient.getCommitsTimeline().filterCompletedInstants().getInstants()
|
||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
|
||||
assertThat(commits.size(), is(3));
|
||||
|
||||
// only the start commit
|
||||
conf.setString(FlinkOptions.READ_START_COMMIT, commits.get(1));
|
||||
this.tableSource = getTableSource(conf);
|
||||
InputFormat<RowData, ?> inputFormat1 = this.tableSource.getInputFormat();
|
||||
assertThat(inputFormat1, instanceOf(MergeOnReadInputFormat.class));
|
||||
|
||||
List<RowData> actual1 = readData(inputFormat1);
|
||||
final List<RowData> expected1 = TestData.dataSetInsert(3, 4, 5, 6);
|
||||
TestData.assertRowDataEquals(actual1, expected1);
|
||||
|
||||
// only the start commit: earliest
|
||||
conf.setString(FlinkOptions.READ_START_COMMIT, FlinkOptions.START_COMMIT_EARLIEST);
|
||||
this.tableSource = getTableSource(conf);
|
||||
InputFormat<RowData, ?> inputFormat2 = this.tableSource.getInputFormat();
|
||||
assertThat(inputFormat2, instanceOf(MergeOnReadInputFormat.class));
|
||||
|
||||
List<RowData> actual2 = readData(inputFormat2);
|
||||
final List<RowData> expected2 = TestData.dataSetInsert(1, 2, 3, 4, 5, 6);
|
||||
TestData.assertRowDataEquals(actual2, expected2);
|
||||
|
||||
// start and end commit: [start commit, end commit]
|
||||
conf.setString(FlinkOptions.READ_START_COMMIT, commits.get(0));
|
||||
conf.setString(FlinkOptions.READ_END_COMMIT, commits.get(1));
|
||||
this.tableSource = getTableSource(conf);
|
||||
InputFormat<RowData, ?> inputFormat3 = this.tableSource.getInputFormat();
|
||||
assertThat(inputFormat3, instanceOf(MergeOnReadInputFormat.class));
|
||||
|
||||
List<RowData> actual3 = readData(inputFormat3);
|
||||
final List<RowData> expected3 = TestData.dataSetInsert(1, 2, 3, 4);
|
||||
TestData.assertRowDataEquals(actual3, expected3);
|
||||
|
||||
// only the end commit: point in time query
|
||||
conf.removeConfig(FlinkOptions.READ_START_COMMIT);
|
||||
conf.setString(FlinkOptions.READ_END_COMMIT, commits.get(1));
|
||||
this.tableSource = getTableSource(conf);
|
||||
InputFormat<RowData, ?> inputFormat4 = this.tableSource.getInputFormat();
|
||||
assertThat(inputFormat4, instanceOf(MergeOnReadInputFormat.class));
|
||||
|
||||
List<RowData> actual4 = readData(inputFormat4);
|
||||
final List<RowData> expected4 = TestData.dataSetInsert(3, 4);
|
||||
TestData.assertRowDataEquals(actual4, expected4);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = HoodieTableType.class)
|
||||
void testReadWithWiderSchema(HoodieTableType tableType) throws Exception {
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.SOURCE_AVRO_SCHEMA.key(),
|
||||
AvroSchemaConverter.convertToSchema(TestConfigurations.ROW_TYPE_WIDER).toString());
|
||||
beforeEach(tableType, options);
|
||||
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
InputFormat<RowData, ?> inputFormat = this.tableSource.getInputFormat();
|
||||
List<RowData> result = readData(inputFormat);
|
||||
TestData.assertRowDataEquals(result, TestData.DATA_SET_INSERT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test reading file groups with compaction plan scheduled and delta logs.
|
||||
* File-slice after pending compaction-requested instant-time should also be considered valid.
|
||||
*/
|
||||
@Test
|
||||
void testReadMORWithCompactionPlanScheduled() throws Exception {
|
||||
Map<String, String> options = new HashMap<>();
|
||||
// compact for each commit
|
||||
options.put(FlinkOptions.COMPACTION_DELTA_COMMITS.key(), "1");
|
||||
options.put(FlinkOptions.COMPACTION_ASYNC_ENABLED.key(), "false");
|
||||
beforeEach(HoodieTableType.MERGE_ON_READ, options);
|
||||
|
||||
// write three commits
|
||||
for (int i = 0; i < 6; i += 2) {
|
||||
List<RowData> dataset = TestData.dataSetInsert(i + 1, i + 2);
|
||||
TestData.writeData(dataset, conf);
|
||||
}
|
||||
|
||||
InputFormat<RowData, ?> inputFormat1 = this.tableSource.getInputFormat();
|
||||
assertThat(inputFormat1, instanceOf(MergeOnReadInputFormat.class));
|
||||
|
||||
List<RowData> actual = readData(inputFormat1);
|
||||
final List<RowData> expected = TestData.dataSetInsert(1, 2, 3, 4, 5, 6);
|
||||
TestData.assertRowDataEquals(actual, expected);
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
private HoodieTableSource getTableSource(Configuration conf) {
|
||||
return new HoodieTableSource(
|
||||
TestConfigurations.TABLE_SCHEMA,
|
||||
new Path(tempFile.getAbsolutePath()),
|
||||
Collections.singletonList("partition"),
|
||||
"default",
|
||||
conf);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked, rawtypes")
|
||||
private static List<RowData> readData(InputFormat inputFormat) throws IOException {
|
||||
InputSplit[] inputSplits = inputFormat.createInputSplits(1);
|
||||
|
||||
List<RowData> result = new ArrayList<>();
|
||||
|
||||
for (InputSplit inputSplit : inputSplits) {
|
||||
inputFormat.open(inputSplit);
|
||||
while (!inputFormat.reachedEnd()) {
|
||||
result.add(TestConfigurations.SERIALIZER.copy((RowData) inputFormat.nextRecord(null))); // no reuse
|
||||
}
|
||||
inputFormat.close();
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,71 @@
|
||||
/*
|
||||
* 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.utils;
|
||||
|
||||
import org.apache.flink.table.catalog.Column;
|
||||
import org.apache.flink.table.catalog.ResolvedSchema;
|
||||
import org.apache.flink.table.catalog.UniqueConstraint;
|
||||
import org.apache.flink.table.catalog.WatermarkSpec;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
/**
|
||||
* Builder for {@link ResolvedSchema}.
|
||||
*/
|
||||
public class SchemaBuilder {
|
||||
private List<Column> columns;
|
||||
private List<WatermarkSpec> watermarkSpecs;
|
||||
private UniqueConstraint constraint;
|
||||
|
||||
public static SchemaBuilder instance() {
|
||||
return new SchemaBuilder();
|
||||
}
|
||||
|
||||
private SchemaBuilder() {
|
||||
this.columns = new ArrayList<>();
|
||||
this.watermarkSpecs = new ArrayList<>();
|
||||
}
|
||||
|
||||
public SchemaBuilder field(String name, DataType type) {
|
||||
this.columns.add(Column.physical(name, type));
|
||||
return this;
|
||||
}
|
||||
|
||||
public SchemaBuilder fields(List<String> names, List<DataType> types) {
|
||||
List<Column> columns = IntStream.range(0, names.size())
|
||||
.mapToObj(idx -> Column.physical(names.get(idx), types.get(idx)))
|
||||
.collect(Collectors.toList());
|
||||
this.columns.addAll(columns);
|
||||
return this;
|
||||
}
|
||||
|
||||
public SchemaBuilder primaryKey(String... columns) {
|
||||
this.constraint = UniqueConstraint.primaryKey("pk", Arrays.asList(columns));
|
||||
return this;
|
||||
}
|
||||
|
||||
public ResolvedSchema build() {
|
||||
return new ResolvedSchema(columns, watermarkSpecs, constraint);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,172 @@
|
||||
/*
|
||||
* 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.utils;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
import org.apache.hudi.util.CompactionUtil;
|
||||
import org.apache.hudi.util.FlinkTables;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
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.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Test cases for {@link org.apache.hudi.util.CompactionUtil}.
|
||||
*/
|
||||
public class TestCompactionUtil {
|
||||
|
||||
private HoodieFlinkTable<?> table;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private Configuration conf;
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
void beforeEach() throws IOException {
|
||||
beforeEach(Collections.emptyMap());
|
||||
}
|
||||
|
||||
void beforeEach(Map<String, String> options) throws IOException {
|
||||
this.conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
options.forEach((k, v) -> conf.setString(k, v));
|
||||
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
|
||||
this.table = FlinkTables.createTable(conf);
|
||||
this.metaClient = table.getMetaClient();
|
||||
}
|
||||
|
||||
@Test
|
||||
void rollbackCompaction() throws Exception {
|
||||
beforeEach();
|
||||
List<String> oriInstants = IntStream.range(0, 3)
|
||||
.mapToObj(i -> generateCompactionPlan()).collect(Collectors.toList());
|
||||
List<HoodieInstant> instants = metaClient.getActiveTimeline()
|
||||
.filterPendingCompactionTimeline()
|
||||
.filter(instant -> instant.getState() == HoodieInstant.State.INFLIGHT)
|
||||
.getInstants()
|
||||
.collect(Collectors.toList());
|
||||
assertThat("all the instants should be in pending state", instants.size(), is(3));
|
||||
CompactionUtil.rollbackCompaction(table);
|
||||
boolean allRolledBack = metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants()
|
||||
.allMatch(instant -> instant.getState() == HoodieInstant.State.REQUESTED);
|
||||
assertTrue(allRolledBack, "all the instants should be rolled back");
|
||||
List<String> actualInstants = metaClient.getActiveTimeline()
|
||||
.filterPendingCompactionTimeline().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
assertThat(actualInstants, is(oriInstants));
|
||||
}
|
||||
|
||||
@Test
|
||||
void rollbackEarliestCompaction() throws Exception {
|
||||
beforeEach();
|
||||
conf.setInteger(FlinkOptions.COMPACTION_TIMEOUT_SECONDS, 0);
|
||||
List<String> oriInstants = IntStream.range(0, 3)
|
||||
.mapToObj(i -> generateCompactionPlan()).collect(Collectors.toList());
|
||||
List<HoodieInstant> instants = metaClient.getActiveTimeline()
|
||||
.filterPendingCompactionTimeline()
|
||||
.filter(instant -> instant.getState() == HoodieInstant.State.INFLIGHT)
|
||||
.getInstants()
|
||||
.collect(Collectors.toList());
|
||||
assertThat("all the instants should be in pending state", instants.size(), is(3));
|
||||
CompactionUtil.rollbackEarliestCompaction(table, conf);
|
||||
long requestedCnt = metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants()
|
||||
.filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).count();
|
||||
assertThat("Only the first instant expects to be rolled back", requestedCnt, is(1L));
|
||||
|
||||
String instantTime = metaClient.getActiveTimeline()
|
||||
.filterPendingCompactionTimeline().filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED)
|
||||
.firstInstant().get().getTimestamp();
|
||||
assertThat(instantTime, is(oriInstants.get(0)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testScheduleCompaction() throws Exception {
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.COMPACTION_SCHEDULE_ENABLED.key(), "false");
|
||||
options.put(FlinkOptions.COMPACTION_TRIGGER_STRATEGY.key(), FlinkOptions.TIME_ELAPSED);
|
||||
options.put(FlinkOptions.COMPACTION_DELTA_SECONDS.key(), "0");
|
||||
beforeEach(options);
|
||||
|
||||
// write a commit with data first
|
||||
TestData.writeDataAsBatch(TestData.DATA_SET_SINGLE_INSERT, conf);
|
||||
|
||||
HoodieFlinkWriteClient<?> writeClient = StreamerUtil.createWriteClient(conf);
|
||||
CompactionUtil.scheduleCompaction(metaClient, writeClient, true, true);
|
||||
|
||||
Option<HoodieInstant> pendingCompactionInstant = metaClient.reloadActiveTimeline().filterPendingCompactionTimeline().lastInstant();
|
||||
assertTrue(pendingCompactionInstant.isPresent(), "A compaction plan expects to be scheduled");
|
||||
|
||||
// write another commit with data and start a new instant
|
||||
TestData.writeDataAsBatch(TestData.DATA_SET_INSERT, conf);
|
||||
TimeUnit.SECONDS.sleep(3); // in case the instant time interval is too close
|
||||
writeClient.startCommit();
|
||||
|
||||
CompactionUtil.scheduleCompaction(metaClient, writeClient, true, false);
|
||||
int numCompactionCommits = metaClient.reloadActiveTimeline().filterPendingCompactionTimeline().countInstants();
|
||||
assertThat("Two compaction plan expects to be scheduled", numCompactionCommits, is(2));
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates a compaction plan on the timeline and returns its instant time.
|
||||
*/
|
||||
private String generateCompactionPlan() {
|
||||
HoodieCompactionOperation operation = new HoodieCompactionOperation();
|
||||
HoodieCompactionPlan plan = new HoodieCompactionPlan(Collections.singletonList(operation), Collections.emptyMap(), 1);
|
||||
String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
HoodieInstant compactionInstant =
|
||||
new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime);
|
||||
try {
|
||||
metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant,
|
||||
TimelineMetadataUtils.serializeCompactionPlan(plan));
|
||||
table.getActiveTimeline().transitionCompactionRequestedToInflight(compactionInstant);
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Exception scheduling compaction", ioe);
|
||||
}
|
||||
metaClient.reloadActiveTimeline();
|
||||
return instantTime;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,319 @@
|
||||
/*
|
||||
* 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.utils;
|
||||
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.streamer.FlinkStreamerConfig;
|
||||
import org.apache.hudi.utils.factory.CollectSinkTableFactory;
|
||||
import org.apache.hudi.utils.factory.ContinuousFileSourceFactory;
|
||||
|
||||
import org.apache.flink.configuration.ConfigOption;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.api.TableSchema;
|
||||
import org.apache.flink.table.catalog.ResolvedSchema;
|
||||
import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Configurations for the test.
|
||||
*/
|
||||
public class TestConfigurations {
|
||||
private TestConfigurations() {
|
||||
}
|
||||
|
||||
public static final DataType ROW_DATA_TYPE = 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();
|
||||
|
||||
public static final RowType ROW_TYPE = (RowType) ROW_DATA_TYPE.getLogicalType();
|
||||
|
||||
public static final ResolvedSchema TABLE_SCHEMA = SchemaBuilder.instance()
|
||||
.fields(ROW_TYPE.getFieldNames(), ROW_DATA_TYPE.getChildren())
|
||||
.build();
|
||||
|
||||
private static final List<String> FIELDS = ROW_TYPE.getFields().stream()
|
||||
.map(RowType.RowField::asSummaryString).collect(Collectors.toList());
|
||||
|
||||
public static final DataType ROW_DATA_TYPE_WIDER = DataTypes.ROW(
|
||||
DataTypes.FIELD("uuid", DataTypes.VARCHAR(20)),// record key
|
||||
DataTypes.FIELD("name", DataTypes.VARCHAR(10)),
|
||||
DataTypes.FIELD("age", DataTypes.INT()),
|
||||
DataTypes.FIELD("salary", DataTypes.DOUBLE()),
|
||||
DataTypes.FIELD("ts", DataTypes.TIMESTAMP(3)), // precombine field
|
||||
DataTypes.FIELD("partition", DataTypes.VARCHAR(10)))
|
||||
.notNull();
|
||||
|
||||
public static final RowType ROW_TYPE_WIDER = (RowType) ROW_DATA_TYPE_WIDER.getLogicalType();
|
||||
|
||||
public static String getCreateHoodieTableDDL(String tableName, Map<String, String> options) {
|
||||
return getCreateHoodieTableDDL(tableName, options, true, "partition");
|
||||
}
|
||||
|
||||
public static String getCreateHoodieTableDDL(
|
||||
String tableName,
|
||||
Map<String, String> options,
|
||||
boolean havePartition,
|
||||
String partitionField) {
|
||||
return getCreateHoodieTableDDL(tableName, FIELDS, options, havePartition, "uuid", partitionField);
|
||||
}
|
||||
|
||||
public static String getCreateHoodieTableDDL(
|
||||
String tableName,
|
||||
List<String> fields,
|
||||
Map<String, String> options,
|
||||
boolean havePartition,
|
||||
String pkField,
|
||||
String partitionField) {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("create table ").append(tableName).append("(\n");
|
||||
for (String field : fields) {
|
||||
builder.append(" ").append(field).append(",\n");
|
||||
}
|
||||
builder.append(" PRIMARY KEY(").append(pkField).append(") NOT ENFORCED\n")
|
||||
.append(")\n");
|
||||
if (havePartition) {
|
||||
builder.append("PARTITIONED BY (`").append(partitionField).append("`)\n");
|
||||
}
|
||||
final String connector = options.computeIfAbsent("connector", k -> "hudi");
|
||||
builder.append("with (\n"
|
||||
+ " 'connector' = '").append(connector).append("'");
|
||||
options.forEach((k, v) -> builder.append(",\n")
|
||||
.append(" '").append(k).append("' = '").append(v).append("'"));
|
||||
builder.append("\n)");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
public static String getCreateHudiCatalogDDL(final String catalogName, final String catalogPath) {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("create catalog ").append(catalogName).append(" with (\n");
|
||||
builder.append(" 'type' = 'hudi',\n"
|
||||
+ " 'catalog.path' = '").append(catalogPath).append("'");
|
||||
builder.append("\n)");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
public static String getFileSourceDDL(String tableName) {
|
||||
return getFileSourceDDL(tableName, "test_source.data");
|
||||
}
|
||||
|
||||
public static String getFileSourceDDL(String tableName, int checkpoints) {
|
||||
return getFileSourceDDL(tableName, "test_source.data", checkpoints);
|
||||
}
|
||||
|
||||
public static String getFileSourceDDL(String tableName, String fileName) {
|
||||
return getFileSourceDDL(tableName, fileName, 2);
|
||||
}
|
||||
|
||||
public static String getFileSourceDDL(String tableName, String fileName, int checkpoints) {
|
||||
String sourcePath = Objects.requireNonNull(Thread.currentThread()
|
||||
.getContextClassLoader().getResource(fileName)).toString();
|
||||
return "create table " + tableName + "(\n"
|
||||
+ " uuid varchar(20),\n"
|
||||
+ " name varchar(10),\n"
|
||||
+ " age int,\n"
|
||||
+ " ts timestamp(3),\n"
|
||||
+ " `partition` varchar(20)\n"
|
||||
+ ") with (\n"
|
||||
+ " 'connector' = '" + ContinuousFileSourceFactory.FACTORY_ID + "',\n"
|
||||
+ " 'path' = '" + sourcePath + "',\n"
|
||||
+ " 'checkpoints' = '" + checkpoints + "'\n"
|
||||
+ ")";
|
||||
}
|
||||
|
||||
public static String getCollectSinkDDL(String tableName) {
|
||||
return "create table " + tableName + "(\n"
|
||||
+ " uuid varchar(20),\n"
|
||||
+ " name varchar(10),\n"
|
||||
+ " age int,\n"
|
||||
+ " ts timestamp(3),\n"
|
||||
+ " `partition` varchar(20)\n"
|
||||
+ ") with (\n"
|
||||
+ " 'connector' = '" + CollectSinkTableFactory.FACTORY_ID + "'"
|
||||
+ ")";
|
||||
}
|
||||
|
||||
public static String getCollectSinkDDL(String tableName, TableSchema tableSchema) {
|
||||
final StringBuilder builder = new StringBuilder("create table " + tableName + "(\n");
|
||||
String[] fieldNames = tableSchema.getFieldNames();
|
||||
DataType[] fieldTypes = tableSchema.getFieldDataTypes();
|
||||
for (int i = 0; i < fieldNames.length; i++) {
|
||||
builder.append(" `")
|
||||
.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"
|
||||
+ ")";
|
||||
builder.append(withProps);
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
public static String getCsvSourceDDL(String tableName, String fileName) {
|
||||
String sourcePath = Objects.requireNonNull(Thread.currentThread()
|
||||
.getContextClassLoader().getResource(fileName)).toString();
|
||||
return "create table " + tableName + "(\n"
|
||||
+ " uuid varchar(20),\n"
|
||||
+ " name varchar(10),\n"
|
||||
+ " age int,\n"
|
||||
+ " ts timestamp(3),\n"
|
||||
+ " `partition` varchar(20)\n"
|
||||
+ ") with (\n"
|
||||
+ " 'connector' = 'filesystem',\n"
|
||||
+ " 'path' = '" + sourcePath + "',\n"
|
||||
+ " 'format' = 'csv'\n"
|
||||
+ ")";
|
||||
}
|
||||
|
||||
public static final RowDataSerializer SERIALIZER = new RowDataSerializer(ROW_TYPE);
|
||||
|
||||
public static Configuration getDefaultConf(String tablePath) {
|
||||
Configuration conf = new Configuration();
|
||||
conf.setString(FlinkOptions.PATH, tablePath);
|
||||
conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA_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;
|
||||
}
|
||||
|
||||
public static FlinkStreamerConfig getDefaultStreamerConf(String tablePath) {
|
||||
FlinkStreamerConfig streamerConf = new FlinkStreamerConfig();
|
||||
streamerConf.targetBasePath = tablePath;
|
||||
streamerConf.sourceAvroSchemaPath = Objects.requireNonNull(Thread.currentThread()
|
||||
.getContextClassLoader().getResource("test_read_schema.avsc")).toString();
|
||||
streamerConf.targetTableName = "TestHoodieTable";
|
||||
streamerConf.partitionPathField = "partition";
|
||||
streamerConf.tableType = "COPY_ON_WRITE";
|
||||
streamerConf.checkpointInterval = 4000L;
|
||||
return streamerConf;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the tool to build hoodie table DDL.
|
||||
*/
|
||||
public static Sql sql(String tableName) {
|
||||
return new Sql(tableName);
|
||||
}
|
||||
|
||||
public static Catalog catalog(String catalogName) {
|
||||
return new Catalog(catalogName);
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Tool to build hoodie table DDL with schema {@link #TABLE_SCHEMA}.
|
||||
*/
|
||||
public static class Sql {
|
||||
private final Map<String, String> options;
|
||||
private final String tableName;
|
||||
private List<String> fields = new ArrayList<>();
|
||||
private boolean withPartition = true;
|
||||
private String pkField = "uuid";
|
||||
private String partitionField = "partition";
|
||||
|
||||
public Sql(String tableName) {
|
||||
options = new HashMap<>();
|
||||
this.tableName = tableName;
|
||||
}
|
||||
|
||||
public Sql option(ConfigOption<?> option, Object val) {
|
||||
this.options.put(option.key(), val.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Sql option(String key, Object val) {
|
||||
this.options.put(key, val.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Sql options(Map<String, String> options) {
|
||||
this.options.putAll(options);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Sql noPartition() {
|
||||
this.withPartition = false;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Sql pkField(String pkField) {
|
||||
this.pkField = pkField;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Sql partitionField(String partitionField) {
|
||||
this.partitionField = partitionField;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Sql field(String fieldSchema) {
|
||||
fields.add(fieldSchema);
|
||||
return this;
|
||||
}
|
||||
|
||||
public String end() {
|
||||
if (this.fields.size() == 0) {
|
||||
this.fields = FIELDS;
|
||||
}
|
||||
return TestConfigurations.getCreateHoodieTableDDL(this.tableName, this.fields, options,
|
||||
this.withPartition, this.pkField, this.partitionField);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Catalog {
|
||||
private final String catalogName;
|
||||
private String catalogPath = ".";
|
||||
|
||||
public Catalog(String catalogName) {
|
||||
this.catalogName = catalogName;
|
||||
}
|
||||
|
||||
public Catalog catalogPath(String catalogPath) {
|
||||
this.catalogPath = catalogPath;
|
||||
return this;
|
||||
}
|
||||
|
||||
public String end() {
|
||||
return TestConfigurations.getCreateHudiCatalogDDL(catalogName, catalogPath);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,738 @@
|
||||
/*
|
||||
* 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.utils;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.config.HoodieCommonConfig;
|
||||
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.configuration.FlinkOptions;
|
||||
import org.apache.hudi.sink.utils.StreamWriteFunctionWrapper;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
|
||||
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.conversion.DataStructureConverter;
|
||||
import org.apache.flink.table.data.conversion.DataStructureConverters;
|
||||
import org.apache.flink.table.data.writer.BinaryRowWriter;
|
||||
import org.apache.flink.table.data.writer.BinaryWriter;
|
||||
import org.apache.flink.table.runtime.typeutils.InternalSerializers;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.flink.types.Row;
|
||||
import org.apache.flink.types.RowKind;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
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.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Properties;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
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;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Data set for testing, also some utilities to check the results.
|
||||
*/
|
||||
public class TestData {
|
||||
public static List<RowData> DATA_SET_INSERT = Arrays.asList(
|
||||
insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23,
|
||||
TimestampData.fromEpochMillis(1), StringData.fromString("par1")),
|
||||
insertRow(StringData.fromString("id2"), StringData.fromString("Stephen"), 33,
|
||||
TimestampData.fromEpochMillis(2), StringData.fromString("par1")),
|
||||
insertRow(StringData.fromString("id3"), StringData.fromString("Julian"), 53,
|
||||
TimestampData.fromEpochMillis(3), StringData.fromString("par2")),
|
||||
insertRow(StringData.fromString("id4"), StringData.fromString("Fabian"), 31,
|
||||
TimestampData.fromEpochMillis(4), StringData.fromString("par2")),
|
||||
insertRow(StringData.fromString("id5"), StringData.fromString("Sophia"), 18,
|
||||
TimestampData.fromEpochMillis(5), StringData.fromString("par3")),
|
||||
insertRow(StringData.fromString("id6"), StringData.fromString("Emma"), 20,
|
||||
TimestampData.fromEpochMillis(6), StringData.fromString("par3")),
|
||||
insertRow(StringData.fromString("id7"), StringData.fromString("Bob"), 44,
|
||||
TimestampData.fromEpochMillis(7), StringData.fromString("par4")),
|
||||
insertRow(StringData.fromString("id8"), StringData.fromString("Han"), 56,
|
||||
TimestampData.fromEpochMillis(8), StringData.fromString("par4"))
|
||||
);
|
||||
|
||||
public static List<RowData> DATA_SET_UPDATE_INSERT = Arrays.asList(
|
||||
// advance the age by 1
|
||||
insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 24,
|
||||
TimestampData.fromEpochMillis(1), StringData.fromString("par1")),
|
||||
insertRow(StringData.fromString("id2"), StringData.fromString("Stephen"), 34,
|
||||
TimestampData.fromEpochMillis(2), StringData.fromString("par1")),
|
||||
insertRow(StringData.fromString("id3"), StringData.fromString("Julian"), 54,
|
||||
TimestampData.fromEpochMillis(3), StringData.fromString("par2")),
|
||||
insertRow(StringData.fromString("id4"), StringData.fromString("Fabian"), 32,
|
||||
TimestampData.fromEpochMillis(4), StringData.fromString("par2")),
|
||||
// same with before
|
||||
insertRow(StringData.fromString("id5"), StringData.fromString("Sophia"), 18,
|
||||
TimestampData.fromEpochMillis(5), StringData.fromString("par3")),
|
||||
// new data
|
||||
insertRow(StringData.fromString("id9"), StringData.fromString("Jane"), 19,
|
||||
TimestampData.fromEpochMillis(6), StringData.fromString("par3")),
|
||||
insertRow(StringData.fromString("id10"), StringData.fromString("Ella"), 38,
|
||||
TimestampData.fromEpochMillis(7), StringData.fromString("par4")),
|
||||
insertRow(StringData.fromString("id11"), StringData.fromString("Phoebe"), 52,
|
||||
TimestampData.fromEpochMillis(8), StringData.fromString("par4"))
|
||||
);
|
||||
|
||||
public static List<RowData> DATA_SET_INSERT_SEPARATE_PARTITION = Arrays.asList(
|
||||
insertRow(StringData.fromString("id12"), StringData.fromString("Monica"), 27,
|
||||
TimestampData.fromEpochMillis(9), StringData.fromString("par5")),
|
||||
insertRow(StringData.fromString("id13"), StringData.fromString("Phoebe"), 31,
|
||||
TimestampData.fromEpochMillis(10), StringData.fromString("par5")),
|
||||
insertRow(StringData.fromString("id14"), StringData.fromString("Rachel"), 52,
|
||||
TimestampData.fromEpochMillis(11), StringData.fromString("par6")),
|
||||
insertRow(StringData.fromString("id15"), StringData.fromString("Ross"), 29,
|
||||
TimestampData.fromEpochMillis(12), StringData.fromString("par6"))
|
||||
);
|
||||
|
||||
public static List<RowData> DATA_SET_INSERT_DUPLICATES = new ArrayList<>();
|
||||
|
||||
static {
|
||||
IntStream.range(0, 5).forEach(i -> DATA_SET_INSERT_DUPLICATES.add(
|
||||
insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23,
|
||||
TimestampData.fromEpochMillis(1), StringData.fromString("par1"))));
|
||||
}
|
||||
|
||||
public static List<RowData> DATA_SET_INSERT_SAME_KEY = new ArrayList<>();
|
||||
|
||||
static {
|
||||
IntStream.range(0, 5).forEach(i -> DATA_SET_INSERT_SAME_KEY.add(
|
||||
insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23,
|
||||
TimestampData.fromEpochMillis(i), StringData.fromString("par1"))));
|
||||
}
|
||||
|
||||
// data set of test_source.data
|
||||
public static List<RowData> DATA_SET_SOURCE_INSERT = Arrays.asList(
|
||||
insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23,
|
||||
TimestampData.fromEpochMillis(1000), StringData.fromString("par1")),
|
||||
insertRow(StringData.fromString("id2"), StringData.fromString("Stephen"), 33,
|
||||
TimestampData.fromEpochMillis(2000), StringData.fromString("par1")),
|
||||
insertRow(StringData.fromString("id3"), StringData.fromString("Julian"), 53,
|
||||
TimestampData.fromEpochMillis(3000), StringData.fromString("par2")),
|
||||
insertRow(StringData.fromString("id4"), StringData.fromString("Fabian"), 31,
|
||||
TimestampData.fromEpochMillis(4000), StringData.fromString("par2")),
|
||||
insertRow(StringData.fromString("id5"), StringData.fromString("Sophia"), 18,
|
||||
TimestampData.fromEpochMillis(5000), StringData.fromString("par3")),
|
||||
insertRow(StringData.fromString("id6"), StringData.fromString("Emma"), 20,
|
||||
TimestampData.fromEpochMillis(6000), StringData.fromString("par3")),
|
||||
insertRow(StringData.fromString("id7"), StringData.fromString("Bob"), 44,
|
||||
TimestampData.fromEpochMillis(7000), StringData.fromString("par4")),
|
||||
insertRow(StringData.fromString("id8"), StringData.fromString("Han"), 56,
|
||||
TimestampData.fromEpochMillis(8000), StringData.fromString("par4"))
|
||||
);
|
||||
|
||||
// data set of test_source.data latest commit.
|
||||
public static List<RowData> DATA_SET_SOURCE_INSERT_LATEST_COMMIT = Arrays.asList(
|
||||
insertRow(StringData.fromString("id5"), StringData.fromString("Sophia"), 18,
|
||||
TimestampData.fromEpochMillis(5000), StringData.fromString("par3")),
|
||||
insertRow(StringData.fromString("id6"), StringData.fromString("Emma"), 20,
|
||||
TimestampData.fromEpochMillis(6000), StringData.fromString("par3")),
|
||||
insertRow(StringData.fromString("id7"), StringData.fromString("Bob"), 44,
|
||||
TimestampData.fromEpochMillis(7000), StringData.fromString("par4")),
|
||||
insertRow(StringData.fromString("id8"), StringData.fromString("Han"), 56,
|
||||
TimestampData.fromEpochMillis(8000), StringData.fromString("par4"))
|
||||
);
|
||||
|
||||
// merged data set of test_source.data and test_source_2.data
|
||||
public static List<RowData> DATA_SET_SOURCE_MERGED = Arrays.asList(
|
||||
insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 24,
|
||||
TimestampData.fromEpochMillis(1000), StringData.fromString("par1")),
|
||||
insertRow(StringData.fromString("id2"), StringData.fromString("Stephen"), 34,
|
||||
TimestampData.fromEpochMillis(2000), StringData.fromString("par1")),
|
||||
insertRow(StringData.fromString("id3"), StringData.fromString("Julian"), 54,
|
||||
TimestampData.fromEpochMillis(3000), StringData.fromString("par2")),
|
||||
insertRow(StringData.fromString("id4"), StringData.fromString("Fabian"), 32,
|
||||
TimestampData.fromEpochMillis(4000), StringData.fromString("par2")),
|
||||
insertRow(StringData.fromString("id5"), StringData.fromString("Sophia"), 18,
|
||||
TimestampData.fromEpochMillis(5000), StringData.fromString("par3")),
|
||||
insertRow(StringData.fromString("id6"), StringData.fromString("Emma"), 20,
|
||||
TimestampData.fromEpochMillis(6000), StringData.fromString("par3")),
|
||||
insertRow(StringData.fromString("id7"), StringData.fromString("Bob"), 44,
|
||||
TimestampData.fromEpochMillis(7000), StringData.fromString("par4")),
|
||||
insertRow(StringData.fromString("id8"), StringData.fromString("Han"), 56,
|
||||
TimestampData.fromEpochMillis(8000), StringData.fromString("par4")),
|
||||
insertRow(StringData.fromString("id9"), StringData.fromString("Jane"), 19,
|
||||
TimestampData.fromEpochMillis(6000), StringData.fromString("par3")),
|
||||
insertRow(StringData.fromString("id10"), StringData.fromString("Ella"), 38,
|
||||
TimestampData.fromEpochMillis(7000), StringData.fromString("par4")),
|
||||
insertRow(StringData.fromString("id11"), StringData.fromString("Phoebe"), 52,
|
||||
TimestampData.fromEpochMillis(8000), StringData.fromString("par4"))
|
||||
);
|
||||
|
||||
// data set of test_source.data with partition 'par1' overwrite
|
||||
public static List<RowData> DATA_SET_SOURCE_INSERT_OVERWRITE = Arrays.asList(
|
||||
insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 24,
|
||||
TimestampData.fromEpochMillis(1000), StringData.fromString("par1")),
|
||||
insertRow(StringData.fromString("id2"), StringData.fromString("Stephen"), 34,
|
||||
TimestampData.fromEpochMillis(2000), StringData.fromString("par1")),
|
||||
insertRow(StringData.fromString("id3"), StringData.fromString("Julian"), 53,
|
||||
TimestampData.fromEpochMillis(3000), StringData.fromString("par2")),
|
||||
insertRow(StringData.fromString("id4"), StringData.fromString("Fabian"), 31,
|
||||
TimestampData.fromEpochMillis(4000), StringData.fromString("par2")),
|
||||
insertRow(StringData.fromString("id5"), StringData.fromString("Sophia"), 18,
|
||||
TimestampData.fromEpochMillis(5000), StringData.fromString("par3")),
|
||||
insertRow(StringData.fromString("id6"), StringData.fromString("Emma"), 20,
|
||||
TimestampData.fromEpochMillis(6000), StringData.fromString("par3")),
|
||||
insertRow(StringData.fromString("id7"), StringData.fromString("Bob"), 44,
|
||||
TimestampData.fromEpochMillis(7000), StringData.fromString("par4")),
|
||||
insertRow(StringData.fromString("id8"), StringData.fromString("Han"), 56,
|
||||
TimestampData.fromEpochMillis(8000), StringData.fromString("par4"))
|
||||
);
|
||||
|
||||
public static List<RowData> DATA_SET_UPDATE_DELETE = Arrays.asList(
|
||||
// this is update
|
||||
insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 24,
|
||||
TimestampData.fromEpochMillis(1), StringData.fromString("par1")),
|
||||
insertRow(StringData.fromString("id2"), StringData.fromString("Stephen"), 34,
|
||||
TimestampData.fromEpochMillis(2), StringData.fromString("par1")),
|
||||
// this is delete
|
||||
deleteRow(StringData.fromString("id3"), StringData.fromString("Julian"), 53,
|
||||
TimestampData.fromEpochMillis(3), StringData.fromString("par2")),
|
||||
deleteRow(StringData.fromString("id5"), StringData.fromString("Sophia"), 18,
|
||||
TimestampData.fromEpochMillis(5), StringData.fromString("par3")),
|
||||
// delete a record that has no inserts
|
||||
deleteRow(StringData.fromString("id9"), StringData.fromString("Jane"), 19,
|
||||
TimestampData.fromEpochMillis(6), StringData.fromString("par3"))
|
||||
);
|
||||
|
||||
public static List<RowData> DATA_SET_INSERT_UPDATE_DELETE = Arrays.asList(
|
||||
// INSERT
|
||||
insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 19,
|
||||
TimestampData.fromEpochMillis(1), StringData.fromString("par1")),
|
||||
// UPDATE
|
||||
updateBeforeRow(StringData.fromString("id1"), StringData.fromString("Danny"), 19,
|
||||
TimestampData.fromEpochMillis(1), StringData.fromString("par1")),
|
||||
updateAfterRow(StringData.fromString("id1"), StringData.fromString("Danny"), 20,
|
||||
TimestampData.fromEpochMillis(2), StringData.fromString("par1")),
|
||||
updateBeforeRow(StringData.fromString("id1"), StringData.fromString("Danny"), 20,
|
||||
TimestampData.fromEpochMillis(2), StringData.fromString("par1")),
|
||||
updateAfterRow(StringData.fromString("id1"), StringData.fromString("Danny"), 21,
|
||||
TimestampData.fromEpochMillis(3), StringData.fromString("par1")),
|
||||
updateBeforeRow(StringData.fromString("id1"), StringData.fromString("Danny"), 21,
|
||||
TimestampData.fromEpochMillis(3), StringData.fromString("par1")),
|
||||
updateAfterRow(StringData.fromString("id1"), StringData.fromString("Danny"), 22,
|
||||
TimestampData.fromEpochMillis(4), StringData.fromString("par1")),
|
||||
// DELETE
|
||||
deleteRow(StringData.fromString("id1"), StringData.fromString("Danny"), 22,
|
||||
TimestampData.fromEpochMillis(5), StringData.fromString("par1"))
|
||||
);
|
||||
|
||||
public static List<RowData> DATA_SET_SINGLE_INSERT = Collections.singletonList(
|
||||
insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23,
|
||||
TimestampData.fromEpochMillis(1), StringData.fromString("par1")));
|
||||
|
||||
public static List<RowData> DATA_SET_DISORDER_UPDATE_DELETE = Arrays.asList(
|
||||
// DISORDER UPDATE
|
||||
updateAfterRow(StringData.fromString("id1"), StringData.fromString("Danny"), 21,
|
||||
TimestampData.fromEpochMillis(3), StringData.fromString("par1")),
|
||||
updateAfterRow(StringData.fromString("id1"), StringData.fromString("Danny"), 20,
|
||||
TimestampData.fromEpochMillis(2), StringData.fromString("par1")),
|
||||
updateBeforeRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23,
|
||||
TimestampData.fromEpochMillis(1), StringData.fromString("par1")),
|
||||
updateBeforeRow(StringData.fromString("id1"), StringData.fromString("Danny"), 20,
|
||||
TimestampData.fromEpochMillis(2), StringData.fromString("par1")),
|
||||
updateAfterRow(StringData.fromString("id1"), StringData.fromString("Danny"), 22,
|
||||
TimestampData.fromEpochMillis(4), StringData.fromString("par1")),
|
||||
updateBeforeRow(StringData.fromString("id1"), StringData.fromString("Danny"), 21,
|
||||
TimestampData.fromEpochMillis(3), StringData.fromString("par1")),
|
||||
// DISORDER DELETE
|
||||
deleteRow(StringData.fromString("id1"), StringData.fromString("Danny"), 22,
|
||||
TimestampData.fromEpochMillis(2), StringData.fromString("par1"))
|
||||
);
|
||||
|
||||
public static List<RowData> dataSetInsert(int... ids) {
|
||||
List<RowData> inserts = new ArrayList<>();
|
||||
Arrays.stream(ids).forEach(i -> inserts.add(
|
||||
insertRow(StringData.fromString("id" + i), StringData.fromString("Danny"), 23,
|
||||
TimestampData.fromEpochMillis(i), StringData.fromString("par1"))));
|
||||
return inserts;
|
||||
}
|
||||
|
||||
private static Integer toIdSafely(Object id) {
|
||||
if (id == null) {
|
||||
return -1;
|
||||
}
|
||||
final String idStr = id.toString();
|
||||
if (idStr.startsWith("id")) {
|
||||
return Integer.parseInt(idStr.substring(2));
|
||||
}
|
||||
return -1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns string format of a list of RowData.
|
||||
*/
|
||||
public static String rowDataToString(List<RowData> rows) {
|
||||
DataStructureConverter<Object, Object> converter =
|
||||
DataStructureConverters.getConverter(TestConfigurations.ROW_DATA_TYPE);
|
||||
return rows.stream()
|
||||
.sorted(Comparator.comparing(o -> toIdSafely(o.getString(0))))
|
||||
.map(row -> converter.toExternal(row).toString())
|
||||
.collect(Collectors.toList()).toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Write a list of row data with Hoodie format base on the given configuration.
|
||||
*
|
||||
* @param dataBuffer The data buffer to write
|
||||
* @param conf The flink configuration
|
||||
* @throws Exception if error occurs
|
||||
*/
|
||||
public static void writeData(
|
||||
List<RowData> dataBuffer,
|
||||
Configuration conf) throws Exception {
|
||||
StreamWriteFunctionWrapper<RowData> funcWrapper = new StreamWriteFunctionWrapper<>(
|
||||
conf.getString(FlinkOptions.PATH),
|
||||
conf);
|
||||
funcWrapper.openFunction();
|
||||
|
||||
for (RowData rowData : dataBuffer) {
|
||||
funcWrapper.invoke(rowData);
|
||||
}
|
||||
|
||||
// this triggers the data write and event send
|
||||
funcWrapper.checkpointFunction(1);
|
||||
|
||||
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
funcWrapper.checkpointComplete(1);
|
||||
|
||||
funcWrapper.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Write a list of row data with Hoodie format base on the given configuration.
|
||||
*
|
||||
* <p>The difference with {@link #writeData} is that it flush data using #endInput, and it
|
||||
* does not generate inflight instant.
|
||||
*
|
||||
* @param dataBuffer The data buffer to write
|
||||
* @param conf The flink configuration
|
||||
* @throws Exception if error occurs
|
||||
*/
|
||||
public static void writeDataAsBatch(
|
||||
List<RowData> dataBuffer,
|
||||
Configuration conf) throws Exception {
|
||||
StreamWriteFunctionWrapper<RowData> funcWrapper = new StreamWriteFunctionWrapper<>(
|
||||
conf.getString(FlinkOptions.PATH),
|
||||
conf);
|
||||
funcWrapper.openFunction();
|
||||
|
||||
for (RowData rowData : dataBuffer) {
|
||||
funcWrapper.invoke(rowData);
|
||||
}
|
||||
|
||||
// this triggers the data write and event send
|
||||
funcWrapper.endInput();
|
||||
|
||||
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
|
||||
funcWrapper.close();
|
||||
}
|
||||
|
||||
private static String toStringSafely(Object obj) {
|
||||
return obj == null ? "null" : obj.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Sort the {@code rows} using field at index 0 and asserts
|
||||
* it equals with the expected string {@code expected}.
|
||||
*
|
||||
* @param rows Actual result rows
|
||||
* @param expected Expected string of the sorted rows
|
||||
*/
|
||||
public static void assertRowsEquals(List<Row> rows, String expected) {
|
||||
assertRowsEquals(rows, expected, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sort the {@code rows} using field at index 0 and asserts
|
||||
* it equals with the expected string {@code expected}.
|
||||
*
|
||||
* @param rows Actual result rows
|
||||
* @param expected Expected string of the sorted rows
|
||||
* @param withChangeFlag Whether compares with change flags
|
||||
*/
|
||||
public static void assertRowsEquals(List<Row> rows, String expected, boolean withChangeFlag) {
|
||||
String rowsString = rows.stream()
|
||||
.sorted(Comparator.comparing(o -> toStringSafely(o.getField(0))))
|
||||
.map(row -> {
|
||||
final String rowStr = row.toString();
|
||||
if (withChangeFlag) {
|
||||
return row.getKind().shortString() + "(" + rowStr + ")";
|
||||
} else {
|
||||
return rowStr;
|
||||
}
|
||||
})
|
||||
.collect(Collectors.toList()).toString();
|
||||
assertThat(rowsString, is(expected));
|
||||
}
|
||||
|
||||
/**
|
||||
* Sort the {@code rows} using field at index {@code orderingPos} and asserts
|
||||
* it equals with the expected string {@code expected}.
|
||||
*
|
||||
* @param rows Actual result rows
|
||||
* @param expected Expected string of the sorted rows
|
||||
* @param orderingPos Field position for ordering
|
||||
*/
|
||||
public static void assertRowsEquals(List<Row> rows, String expected, int orderingPos) {
|
||||
String rowsString = rows.stream()
|
||||
.sorted(Comparator.comparing(o -> toStringSafely(o.getField(orderingPos))))
|
||||
.collect(Collectors.toList()).toString();
|
||||
assertThat(rowsString, is(expected));
|
||||
}
|
||||
|
||||
/**
|
||||
* Sort the {@code rows} using field at index 0 and asserts
|
||||
* it equals with the expected row data list {@code expected}.
|
||||
*
|
||||
* @param rows Actual result rows
|
||||
* @param expected Expected row data list
|
||||
*/
|
||||
public static void assertRowsEquals(List<Row> rows, List<RowData> expected) {
|
||||
String rowsString = rows.stream()
|
||||
.sorted(Comparator.comparing(o -> toIdSafely(o.getField(0))))
|
||||
.collect(Collectors.toList()).toString();
|
||||
assertThat(rowsString, is(rowDataToString(expected)));
|
||||
}
|
||||
|
||||
/**
|
||||
* Sort the {@code rows} using field at index 0 and asserts
|
||||
* it equals with the expected string {@code expected}.
|
||||
*
|
||||
* @param rows Actual result rows
|
||||
* @param expected Expected string of the sorted rows
|
||||
*/
|
||||
public static void assertRowDataEquals(List<RowData> rows, String expected) {
|
||||
String rowsString = rowDataToString(rows);
|
||||
assertThat(rowsString, is(expected));
|
||||
}
|
||||
|
||||
/**
|
||||
* Sort the {@code rows} using field at index 0 and asserts
|
||||
* it equals with the expected row data list {@code expected}.
|
||||
*
|
||||
* @param rows Actual result rows
|
||||
* @param expected Expected row data list
|
||||
*/
|
||||
public static void assertRowDataEquals(List<RowData> rows, List<RowData> expected) {
|
||||
String rowsString = rowDataToString(rows);
|
||||
assertThat(rowsString, is(rowDataToString(expected)));
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the source data set 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 directory
|
||||
* @param expected The expected results mapping, the key should be the partition path
|
||||
* and value should be values list with the key partition
|
||||
*/
|
||||
public static void checkWrittenData(File baseFile, Map<String, String> expected) throws IOException {
|
||||
checkWrittenData(baseFile, expected, 4);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the source data set 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 directory
|
||||
* @param expected The expected results mapping, the key should be the partition path
|
||||
* and value should be values list with the key partition
|
||||
* @param partitions The expected partition number
|
||||
*/
|
||||
public static void checkWrittenData(
|
||||
File baseFile,
|
||||
Map<String, String> expected,
|
||||
int partitions) throws IOException {
|
||||
assert baseFile.isDirectory();
|
||||
FileFilter filter = file -> !file.getName().startsWith(".");
|
||||
File[] partitionDirs = baseFile.listFiles(filter);
|
||||
assertNotNull(partitionDirs);
|
||||
assertThat(partitionDirs.length, is(partitions));
|
||||
for (File partitionDir : partitionDirs) {
|
||||
File[] dataFiles = partitionDir.listFiles(filter);
|
||||
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())));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the source data set are written as expected.
|
||||
* Different with {@link #checkWrittenData}, it reads all the data files.
|
||||
*
|
||||
* <p>Note: Replace it with the Flink reader when it is supported.
|
||||
*
|
||||
* @param baseFile The file base to check, should be a directory
|
||||
* @param expected The expected results mapping, the key should be the partition path
|
||||
* and value should be values list with the key partition
|
||||
* @param partitions The expected partition number
|
||||
*/
|
||||
public static void checkWrittenAllData(
|
||||
File baseFile,
|
||||
Map<String, String> expected,
|
||||
int partitions) throws IOException {
|
||||
assert baseFile.isDirectory();
|
||||
FileFilter filter = file -> !file.getName().startsWith(".");
|
||||
File[] partitionDirs = baseFile.listFiles(filter);
|
||||
|
||||
assertNotNull(partitionDirs);
|
||||
assertThat(partitionDirs.length, is(partitions));
|
||||
|
||||
for (File partitionDir : partitionDirs) {
|
||||
File[] dataFiles = partitionDir.listFiles(filter);
|
||||
assertNotNull(dataFiles);
|
||||
|
||||
List<String> readBuffer = new ArrayList<>();
|
||||
for (File dataFile : dataFiles) {
|
||||
ParquetReader<GenericRecord> reader = AvroParquetReader
|
||||
.<GenericRecord>builder(new Path(dataFile.getAbsolutePath())).build();
|
||||
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())));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the source data are written as expected.
|
||||
*
|
||||
* <p>Note: Replace it with the Flink reader when it is supported.
|
||||
*
|
||||
* @param basePath The file base to check, should be a directory
|
||||
* @param expected The expected results mapping, the key should be the partition path
|
||||
*/
|
||||
public static void checkWrittenFullData(
|
||||
File basePath,
|
||||
Map<String, List<String>> expected) throws IOException {
|
||||
|
||||
// 1. init flink table
|
||||
HoodieTableMetaClient metaClient = HoodieTestUtils.init(basePath.getAbsolutePath());
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath.getAbsolutePath()).build();
|
||||
HoodieFlinkTable table = HoodieFlinkTable.create(config, HoodieFlinkEngineContext.DEFAULT, metaClient);
|
||||
|
||||
// 2. check each partition data
|
||||
expected.forEach((partition, partitionDataSet) -> {
|
||||
|
||||
List<String> readBuffer = new ArrayList<>();
|
||||
|
||||
table.getBaseFileOnlyView().getLatestBaseFiles(partition)
|
||||
.forEach(baseFile -> {
|
||||
String path = baseFile.getPath();
|
||||
try {
|
||||
ParquetReader<GenericRecord> reader = AvroParquetReader.<GenericRecord>builder(new Path(path)).build();
|
||||
GenericRecord nextRecord = reader.read();
|
||||
while (nextRecord != null) {
|
||||
readBuffer.add(filterOutVariables(nextRecord));
|
||||
nextRecord = reader.read();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
|
||||
assertTrue(partitionDataSet.size() == readBuffer.size() && partitionDataSet.containsAll(readBuffer));
|
||||
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 {
|
||||
// in case it is a delete
|
||||
GenericRecord record = (GenericRecord) hoodieRecord.getData()
|
||||
.getInsertValue(schema, new Properties())
|
||||
.orElse(null);
|
||||
return record == null ? (String) null : filterOutVariables(record);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
})
|
||||
.filter(Objects::nonNull)
|
||||
.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/")
|
||||
.withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue())
|
||||
.withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* 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, ",");
|
||||
}
|
||||
|
||||
public static BinaryRowData insertRow(Object... fields) {
|
||||
return insertRow(TestConfigurations.ROW_TYPE, fields);
|
||||
}
|
||||
|
||||
public static BinaryRowData insertRow(RowType rowType, Object... fields) {
|
||||
LogicalType[] types = rowType.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;
|
||||
}
|
||||
|
||||
private static BinaryRowData deleteRow(Object... fields) {
|
||||
BinaryRowData rowData = insertRow(fields);
|
||||
rowData.setRowKind(RowKind.DELETE);
|
||||
return rowData;
|
||||
}
|
||||
|
||||
private static BinaryRowData updateBeforeRow(Object... fields) {
|
||||
BinaryRowData rowData = insertRow(fields);
|
||||
rowData.setRowKind(RowKind.UPDATE_BEFORE);
|
||||
return rowData;
|
||||
}
|
||||
|
||||
private static BinaryRowData updateAfterRow(Object... fields) {
|
||||
BinaryRowData rowData = insertRow(fields);
|
||||
rowData.setRowKind(RowKind.UPDATE_AFTER);
|
||||
return rowData;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,136 @@
|
||||
/*
|
||||
* 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.utils;
|
||||
|
||||
import org.apache.hudi.client.model.HoodieRowData;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.data.DecimalData;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.data.StringData;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.math.BigDecimal;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
/**
|
||||
* Unit tests {@link HoodieRowData}.
|
||||
*/
|
||||
public class TestHoodieRowData {
|
||||
private final int metaColumnsNum = HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.size();
|
||||
private static final Random RANDOM = new Random();
|
||||
private static final int INTEGER_INDEX = 0;
|
||||
private static final int STRING_INDEX = 1;
|
||||
private static final int BOOLEAN_INDEX = 2;
|
||||
private static final int SHORT_INDEX = 3;
|
||||
private static final int BYTE_INDEX = 4;
|
||||
private static final int LONG_INDEX = 5;
|
||||
private static final int FLOAT_INDEX = 6;
|
||||
private static final int DOUBLE_INDEX = 7;
|
||||
private static final int DECIMAL_INDEX = 8;
|
||||
private static final int BINARY_INDEX = 9;
|
||||
private static final int ROW_INDEX = 10;
|
||||
|
||||
private static final DataType BASIC_DATA_TYPE = DataTypes.ROW(
|
||||
DataTypes.FIELD("integer", DataTypes.INT()),
|
||||
DataTypes.FIELD("string", DataTypes.STRING()),
|
||||
DataTypes.FIELD("boolean", DataTypes.BOOLEAN()),
|
||||
DataTypes.FIELD("short", DataTypes.SMALLINT()),
|
||||
DataTypes.FIELD("byte", DataTypes.TINYINT()),
|
||||
DataTypes.FIELD("long", DataTypes.BIGINT()),
|
||||
DataTypes.FIELD("float", DataTypes.FLOAT()),
|
||||
DataTypes.FIELD("double", DataTypes.DOUBLE()),
|
||||
DataTypes.FIELD("decimal", DataTypes.DECIMAL(10, 4)),
|
||||
DataTypes.FIELD("binary", DataTypes.BYTES()),
|
||||
DataTypes.FIELD("row", DataTypes.ROW()))
|
||||
.notNull();
|
||||
private static final RowType ROW_TYPE = (RowType) BASIC_DATA_TYPE.getLogicalType();
|
||||
|
||||
@Test
|
||||
public void testGet() {
|
||||
Object[] values = getRandomValue(true);
|
||||
RowData rowData = TestData.insertRow(ROW_TYPE, values);
|
||||
|
||||
HoodieRowData hoodieRowData = new HoodieRowData("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName",
|
||||
rowData, true);
|
||||
assertValues(hoodieRowData, "commitTime", "commitSeqNo", "recordKey", "partitionPath",
|
||||
"fileName", values);
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches a random Object[] of values for testing.
|
||||
*
|
||||
* @param haveRowType true if rowType need to be added as one of the elements in the Object[]
|
||||
* @return the random Object[] thus generated
|
||||
*/
|
||||
private Object[] getRandomValue(boolean haveRowType) {
|
||||
Object[] values = new Object[11];
|
||||
values[INTEGER_INDEX] = RANDOM.nextInt();
|
||||
values[STRING_INDEX] = StringData.fromString(UUID.randomUUID().toString());
|
||||
values[BOOLEAN_INDEX] = RANDOM.nextBoolean();
|
||||
values[SHORT_INDEX] = (short) RANDOM.nextInt(2);
|
||||
byte[] bytes = new byte[1];
|
||||
RANDOM.nextBytes(bytes);
|
||||
values[BYTE_INDEX] = bytes[0];
|
||||
values[LONG_INDEX] = RANDOM.nextLong();
|
||||
values[FLOAT_INDEX] = RANDOM.nextFloat();
|
||||
values[DOUBLE_INDEX] = RANDOM.nextDouble();
|
||||
values[DECIMAL_INDEX] = DecimalData.fromBigDecimal(new BigDecimal("1005.12313"), 10, 4);
|
||||
bytes = new byte[20];
|
||||
RANDOM.nextBytes(bytes);
|
||||
values[BINARY_INDEX] = bytes;
|
||||
if (haveRowType) {
|
||||
Object[] rowField = getRandomValue(false);
|
||||
values[ROW_INDEX] = TestData.insertRow(ROW_TYPE, rowField);
|
||||
}
|
||||
return values;
|
||||
}
|
||||
|
||||
private void assertValues(HoodieRowData hoodieRowData, String commitTime, String commitSeqNo, String recordKey, String partitionPath,
|
||||
String filename, Object[] values) {
|
||||
assertEquals(commitTime, hoodieRowData.getString(0).toString());
|
||||
assertEquals(commitSeqNo, hoodieRowData.getString(1).toString());
|
||||
assertEquals(recordKey, hoodieRowData.getString(2).toString());
|
||||
assertEquals(partitionPath, hoodieRowData.getString(3).toString());
|
||||
assertEquals(filename, hoodieRowData.getString(4).toString());
|
||||
assertEquals("I", hoodieRowData.getString(5).toString());
|
||||
// row data.
|
||||
assertEquals(values[INTEGER_INDEX], hoodieRowData.getInt(INTEGER_INDEX + metaColumnsNum));
|
||||
assertEquals(values[STRING_INDEX], hoodieRowData.getString(STRING_INDEX + metaColumnsNum));
|
||||
assertEquals(values[BOOLEAN_INDEX], hoodieRowData.getBoolean(BOOLEAN_INDEX + metaColumnsNum));
|
||||
assertEquals(values[SHORT_INDEX], hoodieRowData.getShort(SHORT_INDEX + metaColumnsNum));
|
||||
assertEquals(values[BYTE_INDEX], hoodieRowData.getByte(BYTE_INDEX + metaColumnsNum));
|
||||
assertEquals(values[LONG_INDEX], hoodieRowData.getLong(LONG_INDEX + metaColumnsNum));
|
||||
assertEquals(values[FLOAT_INDEX], hoodieRowData.getFloat(FLOAT_INDEX + metaColumnsNum));
|
||||
assertEquals(values[DOUBLE_INDEX], hoodieRowData.getDouble(DOUBLE_INDEX + metaColumnsNum));
|
||||
assertEquals(values[DECIMAL_INDEX], hoodieRowData.getDecimal(DECIMAL_INDEX + metaColumnsNum, 10, 4));
|
||||
byte[] exceptBinary = (byte[]) values[BINARY_INDEX];
|
||||
byte[] binary = hoodieRowData.getBinary(BINARY_INDEX + metaColumnsNum);
|
||||
for (int i = 0; i < exceptBinary.length; i++) {
|
||||
assertEquals(exceptBinary[i], binary[i]);
|
||||
}
|
||||
assertEquals(values[ROW_INDEX], hoodieRowData.getRow(ROW_INDEX + metaColumnsNum, values.length));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,64 @@
|
||||
/*
|
||||
* 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.utils;
|
||||
|
||||
/**
|
||||
* Test sql statements.
|
||||
*/
|
||||
public class 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"
|
||||
+ "('id2','Stephen',33,TIMESTAMP '1970-01-01 00:00:02','par1'),\n"
|
||||
+ "('id3','Julian',53,TIMESTAMP '1970-01-01 00:00:03','par2'),\n"
|
||||
+ "('id4','Fabian',31,TIMESTAMP '1970-01-01 00:00:04','par2'),\n"
|
||||
+ "('id5','Sophia',18,TIMESTAMP '1970-01-01 00:00:05','par3'),\n"
|
||||
+ "('id6','Emma',20,TIMESTAMP '1970-01-01 00:00:06','par3'),\n"
|
||||
+ "('id7','Bob',44,TIMESTAMP '1970-01-01 00:00:07','par4'),\n"
|
||||
+ "('id8','Han',56,TIMESTAMP '1970-01-01 00:00:08','par4')";
|
||||
|
||||
public static final String INSERT_SAME_KEY_T1 = "insert into t1 values\n"
|
||||
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:05','par1'),\n"
|
||||
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:04','par1'),\n"
|
||||
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:03','par1'),\n"
|
||||
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:02','par1'),\n"
|
||||
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:01','par1')";
|
||||
|
||||
public static final String UPDATE_INSERT_T1 = "insert into t1 values\n"
|
||||
+ "('id1','Danny',24,TIMESTAMP '1970-01-01 00:00:01','par1'),\n"
|
||||
+ "('id2','Stephen',34,TIMESTAMP '1970-01-01 00:00:02','par1'),\n"
|
||||
+ "('id3','Julian',54,TIMESTAMP '1970-01-01 00:00:03','par2'),\n"
|
||||
+ "('id4','Fabian',32,TIMESTAMP '1970-01-01 00:00:04','par2'),\n"
|
||||
+ "('id5','Sophia',18,TIMESTAMP '1970-01-01 00:00:05','par3'),\n"
|
||||
+ "('id9','Jane',19,TIMESTAMP '1970-01-01 00:00:06','par3'),\n"
|
||||
+ "('id10','Ella',38,TIMESTAMP '1970-01-01 00:00:07','par4'),\n"
|
||||
+ "('id11','Phoebe',52,TIMESTAMP '1970-01-01 00:00:08','par4')";
|
||||
|
||||
public static final String COMPLEX_TYPE_INSERT_T1 = "insert into t1 values\n"
|
||||
+ "(1, array['abc1', 'def1'], map['abc1', 1, 'def1', 3], row(1, 'abc1')),\n"
|
||||
+ "(2, array['abc2', 'def2'], map['abc2', 1, 'def2', 3], row(2, 'abc2')),\n"
|
||||
+ "(3, array['abc3', 'def3'], map['abc3', 1, 'def3', 3], row(3, 'abc3'))";
|
||||
|
||||
public static final String COMPLEX_NESTED_ROW_TYPE_INSERT_T1 = "insert into t1 values\n"
|
||||
+ "(1, array['abc1', 'def1'], array[1, 1], map['abc1', 1, 'def1', 3], row(array['abc1', 'def1'], row(1, 'abc1'))),\n"
|
||||
+ "(2, array['abc2', 'def2'], array[2, 2], map['abc2', 1, 'def2', 3], row(array['abc2', 'def2'], row(2, 'abc2'))),\n"
|
||||
+ "(3, array['abc3', 'def3'], array[3, 3], map['abc3', 1, 'def3', 3], row(array['abc3', 'def3'], row(3, 'abc3')))";
|
||||
}
|
||||
@@ -0,0 +1,113 @@
|
||||
/*
|
||||
* 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.utils;
|
||||
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.util.ViewStorageProperties;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Test cases for {@link StreamerUtil}.
|
||||
*/
|
||||
public class TestStreamerUtil {
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@Test
|
||||
void testInitTableIfNotExists() throws IOException {
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
|
||||
// Test for partitioned table.
|
||||
conf.setString(FlinkOptions.PRECOMBINE_FIELD, "ts");
|
||||
conf.setString(FlinkOptions.PARTITION_PATH_FIELD, "p0,p1");
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
|
||||
// Validate the partition fields & preCombineField in hoodie.properties.
|
||||
HoodieTableMetaClient metaClient1 = HoodieTableMetaClient.builder()
|
||||
.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"});
|
||||
assertEquals(metaClient1.getTableConfig().getPreCombineField(), "ts");
|
||||
|
||||
// Test for non-partitioned table.
|
||||
conf.removeConfig(FlinkOptions.PARTITION_PATH_FIELD);
|
||||
FileIOUtils.deleteDirectory(tempFile);
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
HoodieTableMetaClient metaClient2 = HoodieTableMetaClient.builder()
|
||||
.setBasePath(tempFile.getAbsolutePath())
|
||||
.setConf(new org.apache.hadoop.conf.Configuration())
|
||||
.build();
|
||||
assertFalse(metaClient2.getTableConfig().getPartitionFields().isPresent());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testMedianInstantTime() {
|
||||
String higher = "20210705125921";
|
||||
String lower = "20210705125806";
|
||||
String expectedMedianInstant = "20210705125844499";
|
||||
String median1 = StreamerUtil.medianInstantTime(higher, lower).get();
|
||||
assertThat(median1, is(expectedMedianInstant));
|
||||
// test symmetry
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> StreamerUtil.medianInstantTime(lower, higher),
|
||||
"The first argument should have newer instant time");
|
||||
// test very near instant time
|
||||
assertFalse(StreamerUtil.medianInstantTime("20211116115634", "20211116115633").isPresent());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testInstantTimeDiff() {
|
||||
String higher = "20210705125921";
|
||||
String lower = "20210705125806";
|
||||
long diff = StreamerUtil.instantTimeDiffSeconds(higher, lower);
|
||||
assertThat(diff, is(75L));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testDumpRemoteViewStorageConfig() throws IOException {
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
StreamerUtil.createWriteClient(conf);
|
||||
FileSystemViewStorageConfig storageConfig = ViewStorageProperties.loadFromProperties(conf.getString(FlinkOptions.PATH));
|
||||
assertThat(storageConfig.getStorageType(), is(FileSystemViewStorageType.REMOTE_FIRST));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,111 @@
|
||||
/*
|
||||
* 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.utils;
|
||||
|
||||
import org.apache.hudi.keygen.KeyGenUtils;
|
||||
import org.apache.hudi.util.AvroSchemaConverter;
|
||||
import org.apache.hudi.util.RowDataToAvroConverters;
|
||||
import org.apache.hudi.util.StringToRowDataConverter;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.data.DecimalData;
|
||||
import org.apache.flink.table.data.GenericRowData;
|
||||
import org.apache.flink.table.data.TimestampData;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.math.BigDecimal;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDate;
|
||||
import java.time.LocalTime;
|
||||
import java.time.temporal.ChronoField;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
||||
|
||||
/**
|
||||
* Test cases for {@link StringToRowDataConverter}.
|
||||
*/
|
||||
public class TestStringToRowDataConverter {
|
||||
@Test
|
||||
void testConvert() {
|
||||
String[] fields = new String[] {"1.1", "3.4", "2021-03-30", "56669000", "1617119069000", "1617119069666111", "12345.67"};
|
||||
LogicalType[] fieldTypes = new LogicalType[] {
|
||||
DataTypes.FLOAT().getLogicalType(),
|
||||
DataTypes.DOUBLE().getLogicalType(),
|
||||
DataTypes.DATE().getLogicalType(),
|
||||
DataTypes.TIME(3).getLogicalType(),
|
||||
DataTypes.TIMESTAMP(3).getLogicalType(),
|
||||
DataTypes.TIMESTAMP(6).getLogicalType(),
|
||||
DataTypes.DECIMAL(7, 2).getLogicalType()
|
||||
};
|
||||
StringToRowDataConverter converter = new StringToRowDataConverter(fieldTypes);
|
||||
Object[] converted = converter.convert(fields);
|
||||
Object[] expected = new Object[] {
|
||||
1.1f, 3.4D, (int) LocalDate.parse("2021-03-30").toEpochDay(),
|
||||
LocalTime.parse("15:44:29").get(ChronoField.MILLI_OF_DAY),
|
||||
TimestampData.fromInstant(Instant.parse("2021-03-30T15:44:29Z")),
|
||||
TimestampData.fromInstant(Instant.parse("2021-03-30T15:44:29.666111Z")),
|
||||
DecimalData.fromBigDecimal(new BigDecimal("12345.67"), 7, 2)
|
||||
};
|
||||
assertArrayEquals(expected, converted);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRowDataToAvroStringToRowData() {
|
||||
GenericRowData rowData = new GenericRowData(7);
|
||||
rowData.setField(0, 1.1f);
|
||||
rowData.setField(1, 3.4D);
|
||||
rowData.setField(2, (int) LocalDate.parse("2021-03-30").toEpochDay());
|
||||
rowData.setField(3, LocalTime.parse("15:44:29").get(ChronoField.MILLI_OF_DAY));
|
||||
rowData.setField(4, TimestampData.fromInstant(Instant.parse("2021-03-30T15:44:29Z")));
|
||||
rowData.setField(5, TimestampData.fromInstant(Instant.parse("2021-03-30T15:44:29.666111Z")));
|
||||
rowData.setField(6, DecimalData.fromBigDecimal(new BigDecimal("12345.67"), 7, 2));
|
||||
|
||||
DataType dataType = DataTypes.ROW(
|
||||
DataTypes.FIELD("f_float", DataTypes.FLOAT()),
|
||||
DataTypes.FIELD("f_double", DataTypes.DOUBLE()),
|
||||
DataTypes.FIELD("f_date", DataTypes.DATE()),
|
||||
DataTypes.FIELD("f_time", DataTypes.TIME(3)),
|
||||
DataTypes.FIELD("f_timestamp", DataTypes.TIMESTAMP(3)),
|
||||
DataTypes.FIELD("f_timestamp_micros", DataTypes.TIMESTAMP(6)),
|
||||
DataTypes.FIELD("f_decimal", DataTypes.DECIMAL(7, 2))
|
||||
);
|
||||
RowType rowType = (RowType) dataType.getLogicalType();
|
||||
RowDataToAvroConverters.RowDataToAvroConverter converter =
|
||||
RowDataToAvroConverters.createConverter(rowType);
|
||||
GenericRecord avroRecord =
|
||||
(GenericRecord) converter.convert(AvroSchemaConverter.convertToSchema(rowType), rowData);
|
||||
StringToRowDataConverter stringToRowDataConverter =
|
||||
new StringToRowDataConverter(rowType.getChildren().toArray(new LogicalType[0]));
|
||||
final String recordKey = KeyGenUtils.getRecordKey(avroRecord, rowType.getFieldNames(), false);
|
||||
final String[] recordKeys = KeyGenUtils.extractRecordKeys(recordKey);
|
||||
Object[] convertedKeys = stringToRowDataConverter.convert(recordKeys);
|
||||
|
||||
GenericRowData converted = new GenericRowData(7);
|
||||
for (int i = 0; i < 7; i++) {
|
||||
converted.setField(i, convertedKeys[i]);
|
||||
}
|
||||
assertThat(converted, is(rowData));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,91 @@
|
||||
/*
|
||||
* 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.utils;
|
||||
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.source.StreamReadMonitoringFunction;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.core.fs.Path;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Common test utils.
|
||||
*/
|
||||
public class TestUtils {
|
||||
public static String getLastPendingInstant(String basePath) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
return StreamerUtil.getLastPendingInstant(metaClient);
|
||||
}
|
||||
|
||||
public static String getLastCompleteInstant(String basePath) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
return StreamerUtil.getLastCompletedInstant(metaClient);
|
||||
}
|
||||
|
||||
public static String getLastDeltaCompleteInstant(String basePath) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
return metaClient.getCommitsTimeline().filterCompletedInstants()
|
||||
.filter(hoodieInstant -> hoodieInstant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.lastInstant()
|
||||
.map(HoodieInstant::getTimestamp)
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
public static String getFirstCompleteInstant(String basePath) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
return metaClient.getCommitsAndCompactionTimeline().filterCompletedInstants().firstInstant()
|
||||
.map(HoodieInstant::getTimestamp).orElse(null);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public static String getNthCompleteInstant(String basePath, int n, boolean isDelta) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
return metaClient.getActiveTimeline()
|
||||
.filterCompletedInstants()
|
||||
.filter(instant -> isDelta ? HoodieTimeline.DELTA_COMMIT_ACTION.equals(instant.getAction()) : HoodieTimeline.COMMIT_ACTION.equals(instant.getAction()))
|
||||
.nthInstant(n).map(HoodieInstant::getTimestamp)
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
public static String getSplitPartitionPath(MergeOnReadInputSplit split) {
|
||||
assertTrue(split.getLogPaths().isPresent());
|
||||
final String logPath = split.getLogPaths().get().get(0);
|
||||
String[] paths = logPath.split(Path.SEPARATOR);
|
||||
return paths[paths.length - 2];
|
||||
}
|
||||
|
||||
public static StreamReadMonitoringFunction getMonitorFunc(Configuration conf) {
|
||||
final String basePath = conf.getString(FlinkOptions.PATH);
|
||||
return new StreamReadMonitoringFunction(conf, new Path(basePath), 1024 * 1024L, null);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,57 @@
|
||||
/*
|
||||
* 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.utils;
|
||||
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.util.ViewStorageProperties;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
|
||||
/**
|
||||
* Test cases for {@link ViewStorageProperties}.
|
||||
*/
|
||||
public class TestViewStorageProperties {
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@Test
|
||||
void testReadWriteProperties() throws IOException {
|
||||
String basePath = tempFile.getAbsolutePath();
|
||||
FileSystemViewStorageConfig config = FileSystemViewStorageConfig.newBuilder()
|
||||
.withStorageType(FileSystemViewStorageType.SPILLABLE_DISK)
|
||||
.withRemoteServerHost("host1")
|
||||
.withRemoteServerPort(1234).build();
|
||||
ViewStorageProperties.createProperties(basePath, config);
|
||||
ViewStorageProperties.createProperties(basePath, config);
|
||||
ViewStorageProperties.createProperties(basePath, config);
|
||||
|
||||
FileSystemViewStorageConfig readConfig = ViewStorageProperties.loadFromProperties(basePath);
|
||||
assertThat(readConfig.getStorageType(), is(FileSystemViewStorageType.SPILLABLE_DISK));
|
||||
assertThat(readConfig.getRemoteViewServerHost(), is("host1"));
|
||||
assertThat(readConfig.getRemoteViewServerPort(), is(1234));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,179 @@
|
||||
/*
|
||||
* 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.utils.factory;
|
||||
|
||||
import org.apache.flink.api.common.state.ListState;
|
||||
import org.apache.flink.api.common.state.ListStateDescriptor;
|
||||
import org.apache.flink.api.java.typeutils.RowTypeInfo;
|
||||
import org.apache.flink.configuration.ConfigOption;
|
||||
import org.apache.flink.runtime.state.FunctionInitializationContext;
|
||||
import org.apache.flink.runtime.state.FunctionSnapshotContext;
|
||||
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
|
||||
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
|
||||
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
|
||||
import org.apache.flink.table.api.TableSchema;
|
||||
import org.apache.flink.table.connector.ChangelogMode;
|
||||
import org.apache.flink.table.connector.sink.DynamicTableSink;
|
||||
import org.apache.flink.table.connector.sink.SinkFunctionProvider;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.factories.DynamicTableSinkFactory;
|
||||
import org.apache.flink.table.factories.FactoryUtil;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.utils.TypeConversions;
|
||||
import org.apache.flink.types.Row;
|
||||
import org.apache.flink.types.RowKind;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Factory for CollectTableSink.
|
||||
*
|
||||
* <p>Note: The CollectTableSink collects all the data of a table into a global collection {@code RESULT},
|
||||
* so the tests should executed in single thread and the table name should be the same.
|
||||
*/
|
||||
public class CollectSinkTableFactory implements DynamicTableSinkFactory {
|
||||
public static final String FACTORY_ID = "collect";
|
||||
|
||||
// global results to collect and query
|
||||
public static final Map<Integer, List<Row>> RESULT = new HashMap<>();
|
||||
|
||||
@Override
|
||||
public DynamicTableSink createDynamicTableSink(Context context) {
|
||||
FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
|
||||
helper.validate();
|
||||
|
||||
TableSchema schema = context.getCatalogTable().getSchema();
|
||||
RESULT.clear();
|
||||
return new CollectTableSink(schema, context.getObjectIdentifier().getObjectName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String factoryIdentifier() {
|
||||
return FACTORY_ID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<ConfigOption<?>> requiredOptions() {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<ConfigOption<?>> optionalOptions() {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
|
||||
// --------------------------------------------------------------------------------------------
|
||||
// Table sinks
|
||||
// --------------------------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Values {@link DynamicTableSink} for testing.
|
||||
*/
|
||||
private static class CollectTableSink implements DynamicTableSink {
|
||||
|
||||
private final TableSchema schema;
|
||||
private final String tableName;
|
||||
|
||||
private CollectTableSink(
|
||||
TableSchema schema,
|
||||
String tableName) {
|
||||
this.schema = schema;
|
||||
this.tableName = tableName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
|
||||
return ChangelogMode.newBuilder()
|
||||
.addContainedKind(RowKind.INSERT)
|
||||
.addContainedKind(RowKind.DELETE)
|
||||
.addContainedKind(RowKind.UPDATE_AFTER)
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
|
||||
final DataType rowType = schema.toPhysicalRowDataType();
|
||||
final RowTypeInfo rowTypeInfo = (RowTypeInfo) TypeConversions.fromDataTypeToLegacyInfo(rowType);
|
||||
DataStructureConverter converter = context.createDataStructureConverter(schema.toPhysicalRowDataType());
|
||||
return SinkFunctionProvider.of(new CollectSinkFunction(converter, rowTypeInfo));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DynamicTableSink copy() {
|
||||
return new CollectTableSink(schema, tableName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String asSummaryString() {
|
||||
return "CollectSink";
|
||||
}
|
||||
}
|
||||
|
||||
static class CollectSinkFunction extends RichSinkFunction<RowData> implements CheckpointedFunction {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
private final DynamicTableSink.DataStructureConverter converter;
|
||||
private final RowTypeInfo rowTypeInfo;
|
||||
|
||||
protected transient ListState<Row> resultState;
|
||||
protected transient List<Row> localResult;
|
||||
|
||||
private int taskID;
|
||||
|
||||
protected CollectSinkFunction(DynamicTableSink.DataStructureConverter converter, RowTypeInfo rowTypeInfo) {
|
||||
this.converter = converter;
|
||||
this.rowTypeInfo = rowTypeInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void invoke(RowData value, SinkFunction.Context context) {
|
||||
Row row = (Row) converter.toExternal(value);
|
||||
assert row != null;
|
||||
row.setKind(value.getRowKind());
|
||||
RESULT.get(taskID).add(row);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initializeState(FunctionInitializationContext context) throws Exception {
|
||||
this.resultState = context.getOperatorStateStore().getListState(
|
||||
new ListStateDescriptor<>("sink-results", rowTypeInfo));
|
||||
this.localResult = new ArrayList<>();
|
||||
if (context.isRestored()) {
|
||||
for (Row value : resultState.get()) {
|
||||
localResult.add(value);
|
||||
}
|
||||
}
|
||||
this.taskID = getRuntimeContext().getIndexOfThisSubtask();
|
||||
synchronized (CollectSinkTableFactory.class) {
|
||||
RESULT.put(taskID, localResult);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void snapshotState(FunctionSnapshotContext context) throws Exception {
|
||||
resultState.clear();
|
||||
resultState.addAll(RESULT.get(taskID));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,73 @@
|
||||
/*
|
||||
* 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.utils.factory;
|
||||
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.utils.source.ContinuousFileSource;
|
||||
|
||||
import org.apache.flink.configuration.ConfigOption;
|
||||
import org.apache.flink.configuration.ConfigOptions;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.core.fs.Path;
|
||||
import org.apache.flink.table.api.ValidationException;
|
||||
import org.apache.flink.table.connector.source.DynamicTableSource;
|
||||
import org.apache.flink.table.factories.DynamicTableSourceFactory;
|
||||
import org.apache.flink.table.factories.FactoryUtil;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Factory for ContinuousFileSource.
|
||||
*/
|
||||
public class ContinuousFileSourceFactory implements DynamicTableSourceFactory {
|
||||
public static final String FACTORY_ID = "continuous-file-source";
|
||||
|
||||
public static final ConfigOption<Integer> CHECKPOINTS = ConfigOptions
|
||||
.key("checkpoints")
|
||||
.intType()
|
||||
.defaultValue(2)
|
||||
.withDescription("Number of checkpoints to write the data set as, default 2");
|
||||
|
||||
@Override
|
||||
public DynamicTableSource createDynamicTableSource(Context context) {
|
||||
FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
|
||||
helper.validate();
|
||||
|
||||
Configuration conf = (Configuration) helper.getOptions();
|
||||
Path path = new Path(conf.getOptional(FlinkOptions.PATH).orElseThrow(() ->
|
||||
new ValidationException("Option [path] should be not empty.")));
|
||||
return new ContinuousFileSource(context.getCatalogTable().getResolvedSchema(), path, conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String factoryIdentifier() {
|
||||
return FACTORY_ID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<ConfigOption<?>> requiredOptions() {
|
||||
return Collections.singleton(FlinkOptions.PATH);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<ConfigOption<?>> optionalOptions() {
|
||||
return Collections.singleton(CHECKPOINTS);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,185 @@
|
||||
/*
|
||||
* 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.utils.source;
|
||||
|
||||
import org.apache.flink.api.common.state.CheckpointListener;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.core.fs.Path;
|
||||
import org.apache.flink.formats.common.TimestampFormat;
|
||||
import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
|
||||
import org.apache.flink.streaming.api.datastream.DataStream;
|
||||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||||
import org.apache.flink.streaming.api.functions.source.SourceFunction;
|
||||
import org.apache.flink.table.catalog.ResolvedSchema;
|
||||
import org.apache.flink.table.connector.ChangelogMode;
|
||||
import org.apache.flink.table.connector.source.DataStreamScanProvider;
|
||||
import org.apache.flink.table.connector.source.DynamicTableSource;
|
||||
import org.apache.flink.table.connector.source.ScanTableSource;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import static org.apache.hudi.utils.factory.ContinuousFileSourceFactory.CHECKPOINTS;
|
||||
|
||||
/**
|
||||
* A continuous file source that can trigger checkpoints continuously.
|
||||
*
|
||||
* <p>It loads the data in the specified file and split the data into number of checkpoints batches.
|
||||
* Say, if you want 4 checkpoints and there are 8 records in the file, the emit strategy is:
|
||||
*
|
||||
* <pre>
|
||||
* | 2 records | 2 records | 2 records | 2 records |
|
||||
* | cp1 | cp2 |cp3 | cp4 |
|
||||
* </pre>
|
||||
*
|
||||
* <p>If all the data are flushed out, it waits for the next checkpoint to finish and tear down the source.
|
||||
*/
|
||||
public class ContinuousFileSource implements ScanTableSource {
|
||||
|
||||
private final ResolvedSchema tableSchema;
|
||||
private final Path path;
|
||||
private final Configuration conf;
|
||||
|
||||
public ContinuousFileSource(
|
||||
ResolvedSchema tableSchema,
|
||||
Path path,
|
||||
Configuration conf) {
|
||||
this.tableSchema = tableSchema;
|
||||
this.path = path;
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
|
||||
return new DataStreamScanProvider() {
|
||||
|
||||
@Override
|
||||
public boolean isBounded() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataStream<RowData> produceDataStream(StreamExecutionEnvironment execEnv) {
|
||||
final RowType rowType = (RowType) tableSchema.toSourceRowDataType().getLogicalType();
|
||||
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
|
||||
rowType,
|
||||
InternalTypeInfo.of(rowType),
|
||||
false,
|
||||
true,
|
||||
TimestampFormat.ISO_8601);
|
||||
|
||||
return execEnv.addSource(new BoundedSourceFunction(path, conf.getInteger(CHECKPOINTS)))
|
||||
.name("continuous_file_source")
|
||||
.setParallelism(1)
|
||||
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)),
|
||||
InternalTypeInfo.of(rowType));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChangelogMode getChangelogMode() {
|
||||
return ChangelogMode.insertOnly();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DynamicTableSource copy() {
|
||||
return new ContinuousFileSource(this.tableSchema, this.path, this.conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String asSummaryString() {
|
||||
return "ContinuousFileSource";
|
||||
}
|
||||
|
||||
/**
|
||||
* Source function that partition the data into given number checkpoints batches.
|
||||
*/
|
||||
public static class BoundedSourceFunction implements SourceFunction<String>, CheckpointListener {
|
||||
private final Path path;
|
||||
private List<String> dataBuffer;
|
||||
|
||||
private final int checkpoints;
|
||||
private final AtomicInteger currentCP = new AtomicInteger(0);
|
||||
|
||||
private volatile boolean isRunning = true;
|
||||
|
||||
public BoundedSourceFunction(Path path, int checkpoints) {
|
||||
this.path = path;
|
||||
this.checkpoints = checkpoints;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run(SourceContext<String> context) throws Exception {
|
||||
if (this.dataBuffer == null) {
|
||||
loadDataBuffer();
|
||||
}
|
||||
int oldCP = this.currentCP.get();
|
||||
boolean finish = false;
|
||||
while (isRunning) {
|
||||
int batchSize = this.dataBuffer.size() / this.checkpoints;
|
||||
int start = batchSize * oldCP;
|
||||
synchronized (context.getCheckpointLock()) {
|
||||
for (int i = start; i < start + batchSize; i++) {
|
||||
if (i >= this.dataBuffer.size()) {
|
||||
finish = true;
|
||||
break;
|
||||
// wait for the next checkpoint and exit
|
||||
}
|
||||
context.collect(this.dataBuffer.get(i));
|
||||
}
|
||||
}
|
||||
oldCP++;
|
||||
while (this.currentCP.get() < oldCP) {
|
||||
synchronized (context.getCheckpointLock()) {
|
||||
context.getCheckpointLock().wait(10);
|
||||
}
|
||||
}
|
||||
if (finish || !isRunning) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cancel() {
|
||||
this.isRunning = false;
|
||||
}
|
||||
|
||||
private void loadDataBuffer() {
|
||||
try {
|
||||
this.dataBuffer = Files.readAllLines(Paths.get(this.path.toUri()));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("Read file " + this.path + " error", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void notifyCheckpointComplete(long l) {
|
||||
this.currentCP.incrementAndGet();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,18 @@
|
||||
# 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.
|
||||
|
||||
org.apache.hudi.utils.factory.ContinuousFileSourceFactory
|
||||
org.apache.hudi.utils.factory.CollectSinkTableFactory
|
||||
@@ -0,0 +1,16 @@
|
||||
{"before":null,"after":{"id":101,"ts":1000,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606100,"transaction":null}
|
||||
{"before":null,"after":{"id":102,"ts":2000,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
|
||||
{"before":null,"after":{"id":103,"ts":3000,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
|
||||
{"before":null,"after":{"id":104,"ts":4000,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
|
||||
{"before":null,"after":{"id":105,"ts":5000,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
|
||||
{"before":null,"after":{"id":106,"ts":6000,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
|
||||
{"before":null,"after":{"id":107,"ts":7000,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
|
||||
{"before":null,"after":{"id":108,"ts":8000,"name":"jacket","description":"water resistent black wind breaker","weight":0.10000000149011612},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
|
||||
{"before":null,"after":{"id":109,"ts":9000,"name":"spare tire","description":"24 inch spare tire","weight":22.200000762939453},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
|
||||
{"before":{"id":106,"ts":6000,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"after":{"id":106,"ts":10000,"name":"hammer","description":"18oz carpenter hammer","weight":1},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589361987000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":362,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589361987936,"transaction":null}
|
||||
{"before":{"id":107,"ts":7000,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"after":{"id":107,"ts":11000,"name":"rocks","description":"box of assorted rocks","weight":5.099999904632568},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362099000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":717,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362099505,"transaction":null}
|
||||
{"before":null,"after":{"id":110,"ts":12000,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362210000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1068,"row":0,"thread":2,"query":null},"op":"c","ts_ms":1589362210230,"transaction":null}
|
||||
{"before":null,"after":{"id":111,"ts":13000,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362243000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1394,"row":0,"thread":2,"query":null},"op":"c","ts_ms":1589362243428,"transaction":null}
|
||||
{"before":{"id":110,"ts":12000,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"after":{"id":110,"ts":14000,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362293000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1707,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362293539,"transaction":null}
|
||||
{"before":{"id":111,"ts":13000,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"after":{"id":111,"ts":15000,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362330000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2090,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362330904,"transaction":null}
|
||||
{"before":{"id":111,"ts":16000,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"after":null,"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362344000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2443,"row":0,"thread":2,"query":null},"op":"d","ts_ms":1589362344455,"transaction":null}
|
||||
@@ -0,0 +1,29 @@
|
||||
###
|
||||
# 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.
|
||||
###
|
||||
log4j.rootLogger=WARN, CONSOLE
|
||||
log4j.logger.org.apache.hudi=DEBUG
|
||||
log4j.logger.org.apache.hadoop.hbase=ERROR
|
||||
# CONSOLE is set to be a ConsoleAppender.
|
||||
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||
# CONSOLE uses PatternLayout.
|
||||
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n
|
||||
log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
|
||||
log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
|
||||
log4j.appender.CONSOLE.filter.a.LevelMin=WARN
|
||||
log4j.appender.CONSOLE.filter.a.LevelMax=FATAL
|
||||
@@ -0,0 +1,30 @@
|
||||
###
|
||||
# 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.
|
||||
###
|
||||
log4j.rootLogger=INFO, CONSOLE
|
||||
log4j.logger.org.apache=INFO
|
||||
log4j.logger.org.apache.hudi=DEBUG
|
||||
log4j.logger.org.apache.hadoop.hbase=ERROR
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
|
||||
log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
|
||||
log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
|
||||
log4j.appender.CONSOLE.filter.a.LevelMin=INFO
|
||||
log4j.appender.CONSOLE.filter.a.LevelMax=FATAL
|
||||
@@ -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
|
||||
} ]
|
||||
}
|
||||
@@ -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"}
|
||||
@@ -0,0 +1,8 @@
|
||||
{"uuid": "id1", "name": "Danny", "age": 24, "ts": "1970-01-01T00:00:01", "partition": "par1"}
|
||||
{"uuid": "id2", "name": "Stephen", "age": 34, "ts": "1970-01-01T00:00:02", "partition": "par1"}
|
||||
{"uuid": "id3", "name": "Julian", "age": 54, "ts": "1970-01-01T00:00:03", "partition": "par2"}
|
||||
{"uuid": "id4", "name": "Fabian", "age": 32, "ts": "1970-01-01T00:00:04", "partition": "par2"}
|
||||
{"uuid": "id5", "name": "Sophia", "age": 18, "ts": "1970-01-01T00:00:05", "partition": "par3"}
|
||||
{"uuid": "id9", "name": "Jane", "age": 19, "ts": "1970-01-01T00:00:06", "partition": "par3"}
|
||||
{"uuid": "id10", "name": "Ella", "age": 38, "ts": "1970-01-01T00:00:07", "partition": "par4"}
|
||||
{"uuid": "id11", "name": "Phoebe", "age": 52, "ts": "1970-01-01T00:00:08", "partition": "par4"}
|
||||
@@ -0,0 +1,8 @@
|
||||
{"uuid": "id1", "name": "Danny", "age": 24, "ts": "1970-01-01T00:00:01", "partition": "par1"}
|
||||
{"uuid": "id2", "name": "Stephen", "age": 34, "ts": "1970-01-01T00:00:02", "partition": "par1"}
|
||||
{"uuid": "id1", "name": "Danny", "age": 24, "ts": "1970-01-01T00:00:01", "partition": "par1"}
|
||||
{"uuid": "id2", "name": "Stephen", "age": 34, "ts": "1970-01-01T00:00:02", "partition": "par1"}
|
||||
{"uuid": "id1", "name": "Danny", "age": 24, "ts": "1970-01-01T00:00:01", "partition": "par1"}
|
||||
{"uuid": "id2", "name": "Stephen", "age": 34, "ts": "1970-01-01T00:00:02", "partition": "par1"}
|
||||
{"uuid": "id1", "name": "Danny", "age": 24, "ts": "1970-01-01T00:00:01", "partition": "par1"}
|
||||
{"uuid": "id2", "name": "Stephen", "age": 34, "ts": "1970-01-01T00:00:02", "partition": "par1"}
|
||||
@@ -0,0 +1,8 @@
|
||||
{"uuid": "id1", "name": "Danny", "age": 24, "ts": "1970-01-01T00:00:01", "partition": "par1"}
|
||||
{"uuid": "id1", "name": "Stephen", "age": 34, "ts": "1970-01-01T00:00:02", "partition": "par1"}
|
||||
{"uuid": "id1", "name": "Julian", "age": 54, "ts": "1970-01-01T00:00:03", "partition": "par2"}
|
||||
{"uuid": "id1", "name": "Fabian", "age": 32, "ts": "1970-01-01T00:00:04", "partition": "par2"}
|
||||
{"uuid": "id1", "name": "Sophia", "age": 18, "ts": "1970-01-01T00:00:05", "partition": "par3"}
|
||||
{"uuid": "id1", "name": "Jane", "age": 19, "ts": "1970-01-01T00:00:06", "partition": "par3"}
|
||||
{"uuid": "id1", "name": "Ella", "age": 38, "ts": "1970-01-01T00:00:07", "partition": "par4"}
|
||||
{"uuid": "id1", "name": "Phoebe", "age": 52, "ts": "1970-01-01T00:00:08", "partition": "par4"}
|
||||
@@ -0,0 +1,8 @@
|
||||
id1,Danny,23,1970-01-01 00:00:01,par1
|
||||
id2,Stephen,33,1970-01-01 00:00:02,par1
|
||||
id3,Julian,53,1970-01-01 00:00:03,par2
|
||||
id4,Fabian,31,1970-01-01 00:00:04,par2
|
||||
id5,Sophia,18,1970-01-01 00:00:05,par3
|
||||
id6,Emma,20,1970-01-01 00:00:06,par3
|
||||
id7,Bob,44,1970-01-01 00:00:07,par4
|
||||
id8,Han,56,1970-01-01 00:00:08,par4
|
||||
Reference in New Issue
Block a user