Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[hotfix] Fix unstable testRandomCdcEventsUnawareBucket #3014

Merged
merged 3 commits into from
Mar 15, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -21,27 +21,30 @@
import org.apache.paimon.CoreOptions;
import org.apache.paimon.catalog.Catalog;
import org.apache.paimon.catalog.CatalogUtils;
import org.apache.paimon.data.InternalRow;
import org.apache.paimon.flink.FlinkCatalogFactory;
import org.apache.paimon.flink.util.AbstractTestBase;
import org.apache.paimon.fs.FileIO;
import org.apache.paimon.fs.Path;
import org.apache.paimon.fs.local.LocalFileIO;
import org.apache.paimon.options.MemorySize;
import org.apache.paimon.options.Options;
import org.apache.paimon.reader.RecordReaderIterator;
import org.apache.paimon.schema.Schema;
import org.apache.paimon.schema.SchemaManager;
import org.apache.paimon.schema.SchemaUtils;
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.FileStoreTableFactory;
import org.apache.paimon.table.source.ReadBuilder;
import org.apache.paimon.table.source.TableScan;
import org.apache.paimon.types.RowType;
import org.apache.paimon.utils.FailingFileIO;
import org.apache.paimon.utils.TraceableFileIO;

import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.api.io.TempDir;
Expand All @@ -64,23 +67,20 @@ public class FlinkCdcSyncDatabaseSinkITCase extends AbstractTestBase {

@TempDir java.nio.file.Path tempDir;

@Disabled
@Test
@Timeout(120)
public void testRandomCdcEvents() throws Exception {
innerTestRandomCdcEvents(() -> ThreadLocalRandom.current().nextInt(5) + 1, false);
}

@Disabled
@Test
@Timeout(180)
@Timeout(120)
public void testRandomCdcEventsDynamicBucket() throws Exception {
innerTestRandomCdcEvents(() -> -1, false);
}

@Disabled
@Test
@Timeout(180)
@Timeout(120)
public void testRandomCdcEventsUnawareBucket() throws Exception {
innerTestRandomCdcEvents(() -> -1, true);
}
Expand Down Expand Up @@ -177,17 +177,21 @@ private void innerTestRandomCdcEvents(Supplier<Integer> bucket, boolean unawareB

// enable failure when running jobs if needed
FailingFileIO.reset(failingName, 2, 10000);
if (unawareBucketMode) {
// there's a compact operator which won't terminate
env.executeAsync();
} else {
env.execute();
}
env.execute();

// no failure when checking results
FailingFileIO.reset(failingName, 0, 1);
for (int i = 0; i < fileStoreTables.size(); i++) {
testTables.get(i).assertResult(fileStoreTables.get(i));
for (int i = 0; i < numTables; i++) {
FileStoreTable table = fileStoreTables.get(i).copyWithLatestSchema();
SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location());
TableSchema schema = schemaManager.latest().get();

ReadBuilder readBuilder = table.newReadBuilder();
TableScan.Plan plan = readBuilder.newScan().plan();
try (RecordReaderIterator<InternalRow> it =
new RecordReaderIterator<>(readBuilder.newRead().createReader(plan))) {
testTables.get(i).assertResult(schema, it);
}
}
}

Expand All @@ -204,6 +208,10 @@ private FileStoreTable createFileStoreTable(
conf.set(CoreOptions.DYNAMIC_BUCKET_TARGET_ROW_NUM, 100L);
conf.set(CoreOptions.WRITE_BUFFER_SIZE, new MemorySize(4096 * 3));
conf.set(CoreOptions.PAGE_SIZE, new MemorySize(4096));
// disable compaction for unaware bucket mode to avoid instability
if (primaryKeys.isEmpty() && numBucket == -1) {
conf.set(CoreOptions.WRITE_ONLY, true);
}

TableSchema tableSchema =
SchemaUtils.forceCommit(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,19 +22,23 @@
import org.apache.paimon.catalog.Catalog;
import org.apache.paimon.catalog.CatalogUtils;
import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.data.InternalRow;
import org.apache.paimon.flink.FlinkCatalogFactory;
import org.apache.paimon.flink.util.AbstractTestBase;
import org.apache.paimon.fs.FileIO;
import org.apache.paimon.fs.Path;
import org.apache.paimon.fs.local.LocalFileIO;
import org.apache.paimon.options.MemorySize;
import org.apache.paimon.options.Options;
import org.apache.paimon.reader.RecordReaderIterator;
import org.apache.paimon.schema.Schema;
import org.apache.paimon.schema.SchemaManager;
import org.apache.paimon.schema.SchemaUtils;
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.FileStoreTableFactory;
import org.apache.paimon.table.source.ReadBuilder;
import org.apache.paimon.table.source.TableScan;
import org.apache.paimon.types.RowType;
import org.apache.paimon.utils.FailingFileIO;
import org.apache.paimon.utils.TraceableFileIO;
Expand Down Expand Up @@ -168,18 +172,21 @@ private void innerTestRandomCdcEvents(

// enable failure when running jobs if needed
FailingFileIO.reset(failingName, 10, 10000);

if (unawareBucketMode) {
// there's a compact operator which won't terminate
env.executeAsync();
} else {
env.execute();
}
env.execute();

// no failure when checking results
FailingFileIO.reset(failingName, 0, 1);

testTable.assertResult(table);
table = table.copyWithLatestSchema();
SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location());
TableSchema schema = schemaManager.latest().get();

ReadBuilder readBuilder = table.newReadBuilder();
TableScan.Plan plan = readBuilder.newScan().plan();
try (RecordReaderIterator<InternalRow> it =
new RecordReaderIterator<>(readBuilder.newRead().createReader(plan))) {
testTable.assertResult(schema, it);
}
}

private FileStoreTable createFileStoreTable(
Expand All @@ -195,6 +202,10 @@ private FileStoreTable createFileStoreTable(
conf.set(CoreOptions.DYNAMIC_BUCKET_TARGET_ROW_NUM, 100L);
conf.set(CoreOptions.WRITE_BUFFER_SIZE, new MemorySize(4096 * 3));
conf.set(CoreOptions.PAGE_SIZE, new MemorySize(4096));
// disable compaction for unaware bucket mode to avoid instability
if (primaryKeys.isEmpty() && numBucket == -1) {
conf.set(CoreOptions.WRITE_ONLY, true);
}

TableSchema tableSchema =
SchemaUtils.forceCommit(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,7 @@
package org.apache.paimon.flink.sink.cdc;

import org.apache.paimon.data.InternalRow;
import org.apache.paimon.reader.RecordReaderIterator;
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.source.ReadBuilder;
import org.apache.paimon.table.source.TableScan;
import org.apache.paimon.types.DataField;
import org.apache.paimon.types.DataType;
import org.apache.paimon.types.DataTypes;
Expand All @@ -33,6 +29,7 @@
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -188,40 +185,23 @@ public Queue<TestCdcEvent> events() {
return events;
}

public void assertResult(FileStoreTable table) throws Exception {
public void assertResult(TableSchema schema, Iterator<InternalRow> it) {
Map<Integer, Map<String, String>> actual = new HashMap<>();
while (true) {
actual.clear();
table = table.copyWithLatestSchema();
TableSchema schema = table.schema();
ReadBuilder readBuilder = table.newReadBuilder();
TableScan.Plan plan = readBuilder.newScan().plan();

try (RecordReaderIterator<InternalRow> it =
new RecordReaderIterator<>(readBuilder.newRead().createReader(plan))) {
while (it.hasNext()) {
InternalRow row = it.next();
Map<String, String> fields = new HashMap<>();
for (int i = 0; i < schema.fieldNames().size(); i++) {
if (!row.isNullAt(i)) {
fields.put(
schema.fieldNames().get(i),
String.valueOf(
schema.fields().get(i).type().equals(DataTypes.BIGINT())
? row.getLong(i)
: row.getInt(i)));
}
}
actual.put(Integer.valueOf(fields.get("k")), fields);
while (it.hasNext()) {
InternalRow row = it.next();
Map<String, String> fields = new HashMap<>();
for (int i = 0; i < schema.fieldNames().size(); i++) {
if (!row.isNullAt(i)) {
fields.put(
schema.fieldNames().get(i),
String.valueOf(
schema.fields().get(i).type().equals(DataTypes.BIGINT())
? row.getLong(i)
: row.getInt(i)));
}
}

if (actual.size() == expected.size()) {
assertThat(actual).isEqualTo(expected);
break;
}

Thread.sleep(500);
actual.put(Integer.valueOf(fields.get("k")), fields);
}
assertThat(actual).isEqualTo(expected);
}
}
Loading