[HUDI-2547] Schedule Flink compaction in service (#4254)
Co-authored-by: yuzhaojing <yuzhaojing@bytedance.com>
This commit is contained in:
@@ -44,6 +44,8 @@ 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;
|
||||
@@ -58,13 +60,23 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
* IT cases for {@link org.apache.hudi.common.model.HoodieRecord}.
|
||||
*/
|
||||
public class ITTestHoodieFlinkCompactor {
|
||||
private static final Map<String, List<String>> EXPECTED = new HashMap<>();
|
||||
|
||||
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 {
|
||||
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"));
|
||||
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
|
||||
@@ -148,6 +160,48 @@ public class ITTestHoodieFlinkCompactor {
|
||||
|
||||
env.execute("flink_hudi_compaction");
|
||||
writeClient.close();
|
||||
TestData.checkWrittenFullData(tempFile, EXPECTED);
|
||||
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);
|
||||
tableEnv.executeSql(TestSQL.INSERT_T1).await();
|
||||
|
||||
// wait for the asynchronous commit to finish
|
||||
TimeUnit.SECONDS.sleep(5);
|
||||
|
||||
// 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);
|
||||
|
||||
tableEnv.executeSql(TestSQL.UPDATE_INSERT_T1).await();
|
||||
|
||||
// wait for the asynchronous commit to finish
|
||||
TimeUnit.SECONDS.sleep(5);
|
||||
|
||||
asyncCompactionService.shutDown();
|
||||
|
||||
TestData.checkWrittenFullData(tempFile, EXPECTED2);
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user