[HUDI-2129] StreamerUtil.medianInstantTime should return a valid date time string (#3221)
This commit is contained in:
@@ -24,25 +24,30 @@ import org.apache.hudi.configuration.FlinkOptions;
|
||||
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 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.assertTrue;
|
||||
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
|
||||
public void testInitTableIfNotExists() throws IOException {
|
||||
void testInitTableIfNotExists() throws IOException {
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
|
||||
// Test for partitioned table.
|
||||
@@ -70,5 +75,25 @@ public class TestStreamerUtil {
|
||||
.build();
|
||||
assertFalse(metaClient2.getTableConfig().getPartitionColumns().isPresent());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testMedianInstantTime() {
|
||||
String higher = "20210705125921";
|
||||
String lower = "20210705125806";
|
||||
String median1 = StreamerUtil.medianInstantTime(higher, lower);
|
||||
assertThat(median1, is("20210705125843"));
|
||||
// test symmetry
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> StreamerUtil.medianInstantTime(lower, higher),
|
||||
"The first argument should have newer instant time");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testInstantTimeDiff() {
|
||||
String higher = "20210705125921";
|
||||
String lower = "20210705125806";
|
||||
long diff = StreamerUtil.instantTimeDiffSeconds(higher, lower);
|
||||
assertThat(diff, is(75L));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user