1
0

Revert "[HUDI-2087] Support Append only in Flink stream (#3174)" (#3251)

This reverts commit 371526789d.
This commit is contained in:
vinoth chandar
2021-07-09 11:20:09 -07:00
committed by GitHub
parent 371526789d
commit b4562e86e4
14 changed files with 33 additions and 246 deletions

View File

@@ -23,7 +23,6 @@ import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
@@ -45,7 +44,6 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import java.io.File;
import java.util.Arrays;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
@@ -511,83 +509,6 @@ public class TestWriteCopyOnWrite {
checkWrittenData(tempFile, expected, 1);
}
@Test
public void testAppendOnly() throws Exception {
// reset the config option
conf.setDouble(FlinkOptions.WRITE_BATCH_SIZE, 0.0006); // 630 bytes batch size
conf.setBoolean(FlinkOptions.INSERT_DROP_DUPS, false);
conf.setBoolean(FlinkOptions.APPEND_ONLY_ENABLE, true);
conf.setString(FlinkOptions.OPERATION, WriteOperationType.INSERT.value());
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
// open the function and ingest data
funcWrapper.openFunction();
// Each record is 208 bytes. so 4 records expect to trigger a mini-batch write
for (RowData rowData : TestData.DATA_SET_INSERT_SAME_KEY) {
funcWrapper.invoke(rowData);
}
// this triggers the data write and event send
funcWrapper.checkpointFunction(1);
Map<String, List<HoodieRecord>> dataBuffer = funcWrapper.getDataBuffer();
assertThat("All data should be flushed out", dataBuffer.size(), is(0));
final OperatorEvent event1 = funcWrapper.getNextEvent(); // remove the first event first
final OperatorEvent event2 = funcWrapper.getNextEvent();
assertThat("The operator expect to send an event", event2, instanceOf(WriteMetadataEvent.class));
funcWrapper.getCoordinator().handleEventFromOperator(0, event1);
funcWrapper.getCoordinator().handleEventFromOperator(0, event2);
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
String instant = funcWrapper.getWriteClient()
.getLastPendingInstant(getTableType());
funcWrapper.checkpointComplete(1);
Map<String, List<String>> expected = new HashMap<>();
expected.put("par1", Arrays.asList(
"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"));
TestData.checkWrittenAllData(tempFile, expected, 1);
// started a new instant already
checkInflightInstant(funcWrapper.getWriteClient());
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
// insert duplicates again
for (RowData rowData : TestData.DATA_SET_INSERT_SAME_KEY) {
funcWrapper.invoke(rowData);
}
funcWrapper.checkpointFunction(2);
final OperatorEvent event3 = funcWrapper.getNextEvent(); // remove the first event first
final OperatorEvent event4 = funcWrapper.getNextEvent();
funcWrapper.getCoordinator().handleEventFromOperator(0, event3);
funcWrapper.getCoordinator().handleEventFromOperator(0, event4);
funcWrapper.checkpointComplete(2);
// Same the original base file content.
expected.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"));
TestData.checkWrittenAllData(tempFile, expected, 1);
}
@Test
public void testInsertWithSmallBufferSize() throws Exception {
// reset the config option

View File

@@ -23,14 +23,12 @@ import org.apache.hudi.client.common.HoodieFlinkEngineContext;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.sink.utils.StreamWriteFunctionWrapper;
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.utils.TestData;
@@ -39,7 +37,6 @@ import org.apache.avro.Schema;
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 java.io.File;
import java.util.Comparator;
@@ -47,8 +44,6 @@ import java.util.HashMap;
import java.util.Map;
import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertThrows;
/**
* Test cases for delta stream write.
*/
@@ -91,16 +86,6 @@ public class TestWriteMergeOnRead extends TestWriteCopyOnWrite {
return EXPECTED1;
}
@Test
public void testAppendOnly() throws Exception {
conf.setBoolean(FlinkOptions.APPEND_ONLY_ENABLE, true);
conf.setString(FlinkOptions.OPERATION, WriteOperationType.INSERT.value());
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
assertThrows(IllegalArgumentException.class, () -> {
funcWrapper.openFunction();
}, "APPEND_ONLY mode only support in COPY_ON_WRITE table");
}
protected Map<String, String> getMiniBatchExpected() {
Map<String, String> expected = new HashMap<>();
// MOR mode merges the messages with the same key.

View File

@@ -19,18 +19,15 @@
package org.apache.hudi.sink;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.sink.utils.StreamWriteFunctionWrapper;
import org.apache.flink.configuration.Configuration;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import java.util.HashMap;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertThrows;
/**
* Test cases for delta stream write with compaction.
*/
@@ -42,14 +39,10 @@ public class TestWriteMergeOnReadWithCompact extends TestWriteCopyOnWrite {
conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, 1);
}
@Disabled
@Test
public void testAppendOnly() throws Exception {
conf.setBoolean(FlinkOptions.APPEND_ONLY_ENABLE, true);
conf.setString(FlinkOptions.OPERATION, WriteOperationType.INSERT.value());
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
assertThrows(IllegalArgumentException.class, () -> {
funcWrapper.openFunction();
}, "APPEND_ONLY mode only support in COPY_ON_WRITE table");
public void testIndexStateBootstrap() {
// Ignore the index bootstrap because we only support parquet load now.
}
protected Map<String, String> getMiniBatchExpected() {

View File

@@ -71,9 +71,7 @@ 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.
*/
/** 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,
@@ -128,7 +126,6 @@ public class TestData {
);
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,
@@ -136,7 +133,6 @@ public class TestData {
}
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,
@@ -240,8 +236,8 @@ public class TestData {
/**
* 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
* @param dataBuffer The data buffer to write
* @param conf The flink configuration
* @throws Exception if error occurs
*/
public static void writeData(
@@ -285,8 +281,8 @@ public class TestData {
* 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 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) {
@@ -364,10 +360,8 @@ public class TestData {
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);
@@ -387,44 +381,13 @@ public class TestData {
}
}
public static void checkWrittenAllData(
File baseFile,
Map<String, List<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, 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
* @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,
@@ -468,12 +431,12 @@ public class TestData {
*
* <p>Note: Replace it with the Flink reader when it is supported.
*
* @param fs The file system
* @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
* @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,