[HUDI-2562] Embedded timeline server on JobManager (#3812)
This commit is contained in:
@@ -20,7 +20,6 @@ package org.apache.hudi.sink;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
@@ -30,6 +29,7 @@ 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;
|
||||
@@ -94,8 +94,8 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
coordinator.handleEventFromOperator(1, event1);
|
||||
|
||||
coordinator.notifyCheckpointComplete(1);
|
||||
String inflight = coordinator.getWriteClient().getLastPendingInstant(HoodieTableType.COPY_ON_WRITE);
|
||||
String lastCompleted = coordinator.getWriteClient().getLastCompletedInstant(HoodieTableType.COPY_ON_WRITE);
|
||||
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");
|
||||
@@ -145,7 +145,7 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
|
||||
assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(1),
|
||||
"Returns early for empty write results");
|
||||
String lastCompleted = coordinator.getWriteClient().getLastCompletedInstant(HoodieTableType.COPY_ON_WRITE);
|
||||
String lastCompleted = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath());
|
||||
assertNull(lastCompleted, "Returns early for empty write results");
|
||||
assertNull(coordinator.getEventBuffer()[0]);
|
||||
|
||||
@@ -153,7 +153,7 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
coordinator.handleEventFromOperator(1, event1);
|
||||
assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(2),
|
||||
"Commits the instant with partial events anyway");
|
||||
lastCompleted = coordinator.getWriteClient().getLastCompletedInstant(HoodieTableType.COPY_ON_WRITE);
|
||||
lastCompleted = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath());
|
||||
assertThat("Commits the instant with partial events anyway", lastCompleted, is(instant));
|
||||
}
|
||||
|
||||
|
||||
@@ -27,13 +27,13 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
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.exception.HoodieException;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
import org.apache.hudi.sink.utils.InsertFunctionWrapper;
|
||||
import org.apache.hudi.sink.utils.StreamWriteFunctionWrapper;
|
||||
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.operators.coordination.OperatorEvent;
|
||||
@@ -45,22 +45,24 @@ import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED;
|
||||
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.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.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertSame;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
@@ -99,8 +101,7 @@ public class TestWriteCopyOnWrite {
|
||||
|
||||
@BeforeEach
|
||||
public void before() throws Exception {
|
||||
final String basePath = tempFile.getAbsolutePath();
|
||||
conf = TestConfigurations.getDefaultConf(basePath);
|
||||
conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, getTableType().name());
|
||||
setUp(conf);
|
||||
this.funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
|
||||
@@ -134,7 +135,7 @@ public class TestWriteCopyOnWrite {
|
||||
// this triggers the data write and event send
|
||||
funcWrapper.checkpointFunction(1);
|
||||
|
||||
String instant = funcWrapper.getWriteClient().getLastPendingInstant(getTableType());
|
||||
String instant = lastPendingInstant();
|
||||
|
||||
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
MatcherAssert.assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
@@ -149,18 +150,17 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, instant);
|
||||
checkInstantState(REQUESTED, instant);
|
||||
funcWrapper.checkpointComplete(1);
|
||||
// the coordinator checkpoint commits the inflight instant.
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, instant);
|
||||
|
||||
// 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
|
||||
funcWrapper.checkpointFunction(2);
|
||||
|
||||
String instant2 = funcWrapper.getWriteClient()
|
||||
.getLastPendingInstant(getTableType());
|
||||
String instant2 = lastPendingInstant();
|
||||
assertNotEquals(instant, instant2);
|
||||
|
||||
final OperatorEvent nextEvent2 = funcWrapper.getNextEvent();
|
||||
@@ -174,12 +174,15 @@ public class TestWriteCopyOnWrite {
|
||||
|
||||
funcWrapper.checkpointComplete(2);
|
||||
// started a new instant already
|
||||
checkInflightInstant(funcWrapper.getWriteClient());
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
|
||||
checkInflightInstant();
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, instant);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckpointFails() throws Exception {
|
||||
// reset the config option
|
||||
conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, 1L);
|
||||
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
|
||||
// open the function and ingest data
|
||||
funcWrapper.openFunction();
|
||||
// no data written and triggers checkpoint fails,
|
||||
@@ -188,8 +191,7 @@ public class TestWriteCopyOnWrite {
|
||||
// this triggers the data write and event send
|
||||
funcWrapper.checkpointFunction(1);
|
||||
|
||||
String instant = funcWrapper.getWriteClient()
|
||||
.getLastPendingInstant(getTableType());
|
||||
String instant = lastPendingInstant();
|
||||
assertNotNull(instant);
|
||||
|
||||
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
@@ -204,18 +206,17 @@ public class TestWriteCopyOnWrite {
|
||||
"The last checkpoint was aborted, ignore the events");
|
||||
|
||||
// the instant metadata should be reused
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, instant);
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, null);
|
||||
checkInstantState(REQUESTED, instant);
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, null);
|
||||
|
||||
for (RowData rowData : TestData.DATA_SET_INSERT) {
|
||||
funcWrapper.invoke(rowData);
|
||||
}
|
||||
|
||||
// this returns early because there is no inflight instant
|
||||
assertThrows(HoodieException.class,
|
||||
() -> funcWrapper.checkpointFunction(2),
|
||||
"Timeout(0ms) while waiting for");
|
||||
// do not sent the write event and fails the checkpoint,
|
||||
assertDoesNotThrow(() -> funcWrapper.checkpointFunction(2),
|
||||
"The stream writer reuse the last instant time when waiting for the last instant commit timeout");
|
||||
// do not send the write event and fails the checkpoint,
|
||||
// behaves like the last checkpoint is successful.
|
||||
funcWrapper.checkpointFails(2);
|
||||
}
|
||||
@@ -231,16 +232,16 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.checkpointFunction(1);
|
||||
funcWrapper.getNextEvent();
|
||||
|
||||
String instant1 = funcWrapper.getWriteClient().getLastPendingInstant(getTableType());
|
||||
String instant1 = lastPendingInstant();
|
||||
assertNotNull(instant1);
|
||||
|
||||
// fails the subtask
|
||||
funcWrapper.subTaskFails(0);
|
||||
|
||||
String instant2 = funcWrapper.getWriteClient().getLastPendingInstant(getTableType());
|
||||
String instant2 = lastPendingInstant();
|
||||
assertNotEquals(instant2, instant1, "The previous instant should be rolled back when starting new instant");
|
||||
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, null);
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -255,8 +256,7 @@ public class TestWriteCopyOnWrite {
|
||||
// this triggers the data write and event send
|
||||
funcWrapper.checkpointFunction(1);
|
||||
|
||||
String instant = funcWrapper.getWriteClient()
|
||||
.getLastPendingInstant(getTableType());
|
||||
String instant = lastPendingInstant();
|
||||
|
||||
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
@@ -264,11 +264,11 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, instant);
|
||||
checkInstantState(REQUESTED, instant);
|
||||
funcWrapper.checkpointComplete(1);
|
||||
checkWrittenData(tempFile, EXPECTED1);
|
||||
// the coordinator checkpoint commits the inflight instant.
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, instant);
|
||||
checkWrittenData(tempFile, EXPECTED1);
|
||||
}
|
||||
|
||||
@@ -341,8 +341,7 @@ public class TestWriteCopyOnWrite {
|
||||
// this triggers the data write and event send
|
||||
funcWrapper.checkpointFunction(2);
|
||||
|
||||
String instant = funcWrapper.getWriteClient()
|
||||
.getLastPendingInstant(getTableType());
|
||||
String instant = lastPendingInstant();
|
||||
|
||||
nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
@@ -350,10 +349,10 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, instant);
|
||||
checkInstantState(REQUESTED, instant);
|
||||
funcWrapper.checkpointComplete(2);
|
||||
// the coordinator checkpoint commits the inflight instant.
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, instant);
|
||||
checkWrittenData(tempFile, EXPECTED2);
|
||||
}
|
||||
|
||||
@@ -386,8 +385,7 @@ public class TestWriteCopyOnWrite {
|
||||
// this triggers the data write and event send
|
||||
funcWrapper.checkpointFunction(2);
|
||||
|
||||
String instant = funcWrapper.getWriteClient()
|
||||
.getLastPendingInstant(getTableType());
|
||||
String instant = lastPendingInstant();
|
||||
|
||||
nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
@@ -395,10 +393,10 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, instant);
|
||||
checkInstantState(REQUESTED, instant);
|
||||
funcWrapper.checkpointComplete(2);
|
||||
// the coordinator checkpoint commits the inflight instant.
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, instant);
|
||||
|
||||
Map<String, String> expected = getUpsertWithDeleteExpected();
|
||||
checkWrittenData(tempFile, expected);
|
||||
@@ -437,8 +435,7 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event2);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
|
||||
String instant = funcWrapper.getWriteClient()
|
||||
.getLastPendingInstant(getTableType());
|
||||
String instant = lastPendingInstant();
|
||||
|
||||
funcWrapper.checkpointComplete(1);
|
||||
|
||||
@@ -446,8 +443,8 @@ public class TestWriteCopyOnWrite {
|
||||
checkWrittenData(tempFile, expected, 1);
|
||||
|
||||
// started a new instant already
|
||||
checkInflightInstant(funcWrapper.getWriteClient());
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
|
||||
checkInflightInstant();
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, instant);
|
||||
|
||||
// insert duplicates again
|
||||
for (RowData rowData : TestData.DATA_SET_INSERT_DUPLICATES) {
|
||||
@@ -500,8 +497,7 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event2);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
|
||||
String instant = funcWrapper.getWriteClient()
|
||||
.getLastPendingInstant(getTableType());
|
||||
String instant = lastPendingInstant();
|
||||
|
||||
funcWrapper.checkpointComplete(1);
|
||||
|
||||
@@ -511,8 +507,8 @@ public class TestWriteCopyOnWrite {
|
||||
checkWrittenData(tempFile, expected, 1);
|
||||
|
||||
// started a new instant already
|
||||
checkInflightInstant(funcWrapper.getWriteClient());
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
|
||||
checkInflightInstant();
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, instant);
|
||||
|
||||
// insert duplicates again
|
||||
for (RowData rowData : TestData.DATA_SET_INSERT_SAME_KEY) {
|
||||
@@ -552,8 +548,7 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event1);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
|
||||
String instant = funcWrapper.getWriteClient()
|
||||
.getLastPendingInstant(getTableType());
|
||||
String instant = lastPendingInstant();
|
||||
|
||||
funcWrapper.checkpointComplete(1);
|
||||
|
||||
@@ -569,8 +564,8 @@ public class TestWriteCopyOnWrite {
|
||||
TestData.checkWrittenAllData(tempFile, expected, 1);
|
||||
|
||||
// started a new instant already
|
||||
checkInflightInstant(funcWrapper.getWriteClient());
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
|
||||
checkInflightInstant();
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, instant);
|
||||
|
||||
// insert duplicates again
|
||||
for (RowData rowData : TestData.DATA_SET_INSERT_SAME_KEY) {
|
||||
@@ -631,8 +626,7 @@ public class TestWriteCopyOnWrite {
|
||||
}
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
|
||||
String instant = funcWrapper.getWriteClient()
|
||||
.getLastPendingInstant(getTableType());
|
||||
String instant = lastPendingInstant();
|
||||
|
||||
funcWrapper.checkpointComplete(1);
|
||||
|
||||
@@ -640,8 +634,8 @@ public class TestWriteCopyOnWrite {
|
||||
checkWrittenData(tempFile, expected, 1);
|
||||
|
||||
// started a new instant already
|
||||
checkInflightInstant(funcWrapper.getWriteClient());
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
|
||||
checkInflightInstant();
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, instant);
|
||||
|
||||
// insert duplicates again
|
||||
for (RowData rowData : TestData.DATA_SET_INSERT_DUPLICATES) {
|
||||
@@ -736,8 +730,7 @@ public class TestWriteCopyOnWrite {
|
||||
|
||||
assertTrue(funcWrapper.isAlreadyBootstrap());
|
||||
|
||||
String instant = funcWrapper.getWriteClient()
|
||||
.getLastPendingInstant(getTableType());
|
||||
String instant = lastPendingInstant();
|
||||
|
||||
nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
@@ -748,18 +741,18 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, instant);
|
||||
checkInstantState(REQUESTED, instant);
|
||||
|
||||
funcWrapper.checkpointComplete(1);
|
||||
// the coordinator checkpoint commits the inflight instant.
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, instant);
|
||||
checkWrittenData(tempFile, EXPECTED2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteExactlyOnce() throws Exception {
|
||||
// reset the config option
|
||||
conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, 3);
|
||||
conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, 1L);
|
||||
conf.setDouble(FlinkOptions.WRITE_TASK_MAX_SIZE, 200.0006); // 630 bytes buffer size
|
||||
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
|
||||
|
||||
@@ -779,7 +772,7 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.checkpointFunction(1);
|
||||
assertTrue(funcWrapper.isConforming(), "The write function should be waiting for the instant to commit");
|
||||
|
||||
for (int i = 0; i < 2; i++) {
|
||||
for (int i = 0; i < 4; i++) {
|
||||
final OperatorEvent event = funcWrapper.getNextEvent(); // remove the first event first
|
||||
assertThat("The operator expect to send an event", event, instanceOf(WriteMetadataEvent.class));
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event);
|
||||
@@ -793,26 +786,25 @@ public class TestWriteCopyOnWrite {
|
||||
|
||||
assertFalse(funcWrapper.isConforming(), "The write function should finish waiting for the instant to commit");
|
||||
|
||||
// checkpoint for the next round, when there is eager flush but the write
|
||||
// task is waiting for the instant commit ack, should throw for timeout.
|
||||
// checkpoint for the next round
|
||||
funcWrapper.checkpointFunction(2);
|
||||
|
||||
assertThrows(HoodieException.class, () -> {
|
||||
assertDoesNotThrow(() -> {
|
||||
for (RowData rowData : TestData.DATA_SET_INSERT) {
|
||||
funcWrapper.invoke(rowData);
|
||||
}
|
||||
}, "Timeout(500ms) while waiting for instant");
|
||||
}, "The stream writer reuse the last instant time when waiting for the last instant commit timeout");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReuseEmbeddedServer() {
|
||||
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf, null);
|
||||
public void testReuseEmbeddedServer() throws IOException {
|
||||
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf);
|
||||
FileSystemViewStorageConfig viewStorageConfig = writeClient.getConfig().getViewStorageConfig();
|
||||
|
||||
assertSame(viewStorageConfig.getStorageType(), FileSystemViewStorageType.REMOTE_FIRST);
|
||||
|
||||
// get another write client
|
||||
writeClient = StreamerUtil.createWriteClient(conf, null);
|
||||
writeClient = StreamerUtil.createWriteClient(conf);
|
||||
assertSame(writeClient.getConfig().getViewStorageConfig().getStorageType(), FileSystemViewStorageType.REMOTE_FIRST);
|
||||
assertEquals(viewStorageConfig.getRemoteViewServerPort(), writeClient.getConfig().getViewStorageConfig().getRemoteViewServerPort());
|
||||
}
|
||||
@@ -821,24 +813,19 @@ public class TestWriteCopyOnWrite {
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
private void checkInflightInstant(HoodieFlinkWriteClient writeClient) {
|
||||
final String instant = writeClient.getLastPendingInstant(getTableType());
|
||||
private void checkInflightInstant() {
|
||||
final String instant = TestUtils.getLastPendingInstant(tempFile.getAbsolutePath());
|
||||
assertNotNull(instant);
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
private void checkInstantState(
|
||||
HoodieFlinkWriteClient writeClient,
|
||||
HoodieInstant.State state,
|
||||
String instantStr) {
|
||||
private void checkInstantState(HoodieInstant.State state, String instantStr) {
|
||||
final String instant;
|
||||
switch (state) {
|
||||
case REQUESTED:
|
||||
instant = writeClient.getLastPendingInstant(getTableType());
|
||||
instant = lastPendingInstant();
|
||||
break;
|
||||
case COMPLETED:
|
||||
instant = writeClient.getLastCompletedInstant(getTableType());
|
||||
instant = lastCompleteInstant();
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError("Unexpected state");
|
||||
@@ -846,6 +833,14 @@ public class TestWriteCopyOnWrite {
|
||||
assertThat(instant, is(instantStr));
|
||||
}
|
||||
|
||||
protected String lastPendingInstant() {
|
||||
return TestUtils.getLastPendingInstant(tempFile.getAbsolutePath());
|
||||
}
|
||||
|
||||
protected String lastCompleteInstant() {
|
||||
return TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath());
|
||||
}
|
||||
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.COPY_ON_WRITE;
|
||||
}
|
||||
|
||||
@@ -25,8 +25,6 @@ import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
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.table.HoodieFlinkTable;
|
||||
@@ -39,10 +37,8 @@ import org.apache.hadoop.fs.FileSystem;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Test cases for delta stream write.
|
||||
@@ -71,13 +67,7 @@ public class TestWriteMergeOnRead extends TestWriteCopyOnWrite {
|
||||
protected void checkWrittenData(File baseFile, Map<String, String> expected, int partitions) throws Exception {
|
||||
HoodieTableMetaClient metaClient = HoodieFlinkTable.create(writeConfig, context).getMetaClient();
|
||||
Schema schema = new TableSchemaResolver(metaClient).getTableAvroSchema();
|
||||
String latestInstant = metaClient.getCommitsTimeline().filterCompletedInstants()
|
||||
.getInstants()
|
||||
.filter(x -> x.getAction().equals(HoodieActiveTimeline.DELTA_COMMIT_ACTION))
|
||||
.map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList()).stream()
|
||||
.max(Comparator.naturalOrder())
|
||||
.orElse(null);
|
||||
String latestInstant = lastCompleteInstant();
|
||||
TestData.checkWrittenDataMOR(fs, latestInstant, baseFile, expected, partitions, schema);
|
||||
}
|
||||
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi.sink;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.utils.TestUtils;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
|
||||
@@ -53,4 +54,9 @@ public class TestWriteMergeOnReadWithCompact extends TestWriteCopyOnWrite {
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.MERGE_ON_READ;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String lastCompleteInstant() {
|
||||
return TestUtils.getLastDeltaCompleteInstant(tempFile.getAbsolutePath());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -41,8 +41,9 @@ 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.Disabled;
|
||||
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;
|
||||
@@ -69,9 +70,8 @@ public class ITTestHoodieFlinkCompactor {
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
//@ParameterizedTest
|
||||
//@ValueSource(booleans = {true, false})
|
||||
@Disabled
|
||||
@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();
|
||||
@@ -112,7 +112,7 @@ public class ITTestHoodieFlinkCompactor {
|
||||
// judge whether have operation
|
||||
// To compute the compaction instant time and do compaction.
|
||||
String compactionInstantTime = CompactionUtil.getCompactionInstantTime(metaClient);
|
||||
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf, null);
|
||||
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf);
|
||||
boolean scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTime, Option.empty());
|
||||
|
||||
assertTrue(scheduled, "The compaction plan should be scheduled");
|
||||
@@ -141,6 +141,7 @@ public class ITTestHoodieFlinkCompactor {
|
||||
.setParallelism(1);
|
||||
|
||||
env.execute("flink_hudi_compaction");
|
||||
writeClient.close();
|
||||
TestData.checkWrittenFullData(tempFile, EXPECTED);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,9 +22,6 @@ 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.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.sink.partitioner.profile.WriteProfile;
|
||||
@@ -51,9 +48,9 @@ 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.assertFalse;
|
||||
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;
|
||||
|
||||
/**
|
||||
@@ -358,11 +355,11 @@ public class TestBucketAssigner {
|
||||
assertTrue(smallFiles1.isEmpty(), "Should have no small files");
|
||||
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
Option<String> instantOption = getLastCompleteInstant(writeProfile);
|
||||
assertFalse(instantOption.isPresent());
|
||||
String instantOption = getLastCompleteInstant(writeProfile);
|
||||
assertNull(instantOption);
|
||||
|
||||
writeProfile.reload(1);
|
||||
String instant1 = getLastCompleteInstant(writeProfile).orElse(null);
|
||||
String instant1 = getLastCompleteInstant(writeProfile);
|
||||
assertNotNull(instant1);
|
||||
List<SmallFile> smallFiles2 = writeProfile.getSmallFiles("par1");
|
||||
assertThat("Should have 1 small file", smallFiles2.size(), is(1));
|
||||
@@ -376,7 +373,7 @@ public class TestBucketAssigner {
|
||||
smallFiles3.get(0).location.getInstantTime(), is(instant1));
|
||||
|
||||
writeProfile.reload(2);
|
||||
String instant2 = getLastCompleteInstant(writeProfile).orElse(null);
|
||||
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));
|
||||
@@ -389,12 +386,11 @@ public class TestBucketAssigner {
|
||||
WriteProfile writeProfile = new WriteProfile(writeConfig, context);
|
||||
assertTrue(writeProfile.getMetadataCache().isEmpty(), "Empty table should no have any instant metadata");
|
||||
|
||||
HoodieTimeline emptyTimeline = writeProfile.getTable().getActiveTimeline();
|
||||
|
||||
// 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));
|
||||
@@ -402,15 +398,10 @@ public class TestBucketAssigner {
|
||||
writeProfile.getSmallFiles("par1");
|
||||
assertThat("The metadata should be reused",
|
||||
writeProfile.getMetadataCache().size(), is(3));
|
||||
|
||||
writeProfile.reload(2);
|
||||
writeProfile.initFSViewIfNecessary(emptyTimeline);
|
||||
assertTrue(writeProfile.getMetadataCache().isEmpty(), "Metadata cache should be all cleaned");
|
||||
}
|
||||
|
||||
private static Option<String> getLastCompleteInstant(WriteProfile profile) {
|
||||
return profile.getTable().getMetaClient().getCommitsTimeline()
|
||||
.filterCompletedInstants().lastInstant().map(HoodieInstant::getTimestamp);
|
||||
private static String getLastCompleteInstant(WriteProfile profile) {
|
||||
return StreamerUtil.getLastCompletedInstant(profile.getTable().getMetaClient());
|
||||
}
|
||||
|
||||
private void assertBucketEquals(
|
||||
|
||||
@@ -18,7 +18,6 @@
|
||||
|
||||
package org.apache.hudi.sink.utils;
|
||||
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
|
||||
import org.apache.hudi.sink.append.AppendWriteFunction;
|
||||
import org.apache.hudi.sink.bulk.BulkInsertWriterHelper;
|
||||
@@ -98,11 +97,6 @@ public class InsertFunctionWrapper<I> {
|
||||
return this.gateway.getNextEvent();
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
public HoodieFlinkWriteClient getWriteClient() {
|
||||
return this.writeFunction.getWriteClient();
|
||||
}
|
||||
|
||||
public void checkpointFunction(long checkpointId) throws Exception {
|
||||
// checkpoint the coordinator first
|
||||
this.coordinator.checkpointCoordinator(checkpointId, new CompletableFuture<>());
|
||||
|
||||
@@ -18,7 +18,6 @@
|
||||
|
||||
package org.apache.hudi.sink.utils;
|
||||
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
@@ -211,11 +210,6 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
return this.writeFunction.getDataBuffer();
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
public HoodieFlinkWriteClient getWriteClient() {
|
||||
return this.writeFunction.getWriteClient();
|
||||
}
|
||||
|
||||
public void checkpointFunction(long checkpointId) throws Exception {
|
||||
// checkpoint the coordinator first
|
||||
this.coordinator.checkpointCoordinator(checkpointId, new CompletableFuture<>());
|
||||
|
||||
@@ -89,7 +89,7 @@ public class TestStreamReadMonitoringFunction {
|
||||
|
||||
assertTrue(sourceContext.splits.stream().allMatch(split -> split.getInstantRange().isPresent()),
|
||||
"All the instants should have range limit");
|
||||
String latestCommit = TestUtils.getLatestCommit(tempFile.getAbsolutePath());
|
||||
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");
|
||||
|
||||
@@ -143,7 +143,7 @@ public class TestStreamReadMonitoringFunction {
|
||||
// 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.getLatestCommit(tempFile.getAbsolutePath());
|
||||
String specifiedCommit = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath());
|
||||
conf.setString(FlinkOptions.READ_START_COMMIT, specifiedCommit);
|
||||
StreamReadMonitoringFunction function = TestUtils.getMonitorFunc(conf);
|
||||
try (AbstractStreamOperatorTestHarness<MergeOnReadInputSplit> harness = createHarness(function)) {
|
||||
@@ -174,7 +174,7 @@ public class TestStreamReadMonitoringFunction {
|
||||
// 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.getLatestCommit(tempFile.getAbsolutePath());
|
||||
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)) {
|
||||
|
||||
@@ -111,7 +111,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
String insertInto = "insert into t1 select * from source";
|
||||
execInsertSql(streamTableEnv, insertInto);
|
||||
|
||||
String firstCommit = TestUtils.getFirstCommit(tempFile.getAbsolutePath());
|
||||
String firstCommit = TestUtils.getFirstCompleteInstant(tempFile.getAbsolutePath());
|
||||
streamTableEnv.executeSql("drop table t1");
|
||||
hoodieTableDDL = sql("t1")
|
||||
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
|
||||
@@ -181,7 +181,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
// execute 2 times
|
||||
execInsertSql(streamTableEnv, insertInto);
|
||||
// remember the commit
|
||||
String specifiedCommit = TestUtils.getFirstCommit(tempFile.getAbsolutePath());
|
||||
String specifiedCommit = TestUtils.getFirstCompleteInstant(tempFile.getAbsolutePath());
|
||||
// another update batch
|
||||
String insertInto2 = "insert into t1 select * from source2";
|
||||
execInsertSql(streamTableEnv, insertInto2);
|
||||
@@ -264,8 +264,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
Map<String, String> options1 = new HashMap<>(defaultConf.toMap());
|
||||
options1.put(FlinkOptions.TABLE_NAME.key(), "t1");
|
||||
Configuration conf = Configuration.fromMap(options1);
|
||||
HoodieTimeline timeline = StreamerUtil.createWriteClient(conf, null)
|
||||
.getHoodieTable().getActiveTimeline();
|
||||
HoodieTimeline timeline = StreamerUtil.createMetaClient(conf).getActiveTimeline();
|
||||
assertTrue(timeline.filterCompletedInstants()
|
||||
.getInstants().anyMatch(instant -> instant.getAction().equals("clean")),
|
||||
"some commits should be cleaned");
|
||||
@@ -285,8 +284,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
// write another commit with deletes
|
||||
TestData.writeData(TestData.DATA_SET_UPDATE_DELETE, conf);
|
||||
|
||||
String latestCommit = StreamerUtil.createWriteClient(conf, null)
|
||||
.getLastCompletedInstant(HoodieTableType.MERGE_ON_READ);
|
||||
String latestCommit = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath());
|
||||
|
||||
String hoodieTableDDL = sql("t1")
|
||||
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
|
||||
@@ -756,19 +754,17 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
+ " 'format' = 'debezium-json'\n"
|
||||
+ ")";
|
||||
streamTableEnv.executeSql(sourceDDL);
|
||||
String hoodieTableDDL = ""
|
||||
+ "CREATE TABLE hoodie_sink(\n"
|
||||
+ " id INT NOT NULL,\n"
|
||||
+ " ts BIGINT,\n"
|
||||
+ " name STRING,"
|
||||
+ " weight DOUBLE,"
|
||||
+ " PRIMARY KEY (id) NOT ENFORCED"
|
||||
+ ") with (\n"
|
||||
+ " 'connector' = 'hudi',\n"
|
||||
+ " 'path' = '" + tempFile.getAbsolutePath() + "',\n"
|
||||
+ " 'read.streaming.enabled' = '" + (execMode == ExecMode.STREAM) + "',\n"
|
||||
+ " 'write.insert.drop.duplicates' = 'true'"
|
||||
+ ")";
|
||||
String hoodieTableDDL = sql("hoodie_sink")
|
||||
.field("id INT NOT NULL")
|
||||
.field("ts BIGINT")
|
||||
.field("name STRING")
|
||||
.field("weight DOUBLE")
|
||||
.pkField("id")
|
||||
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
|
||||
.option(FlinkOptions.READ_AS_STREAMING, execMode == ExecMode.STREAM)
|
||||
.option(FlinkOptions.PRE_COMBINE, true)
|
||||
.noPartition()
|
||||
.end();
|
||||
streamTableEnv.executeSql(hoodieTableDDL);
|
||||
String insertInto = "insert into hoodie_sink select id, ts, name, weight from debezium_source";
|
||||
execInsertSql(streamTableEnv, insertInto);
|
||||
@@ -949,7 +945,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
TestData.writeData(TestData.dataSetInsert(3, 4), conf);
|
||||
TestData.writeData(TestData.dataSetInsert(5, 6), conf);
|
||||
|
||||
String latestCommit = TestUtils.getLatestCommit(tempFile.getAbsolutePath());
|
||||
String latestCommit = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath());
|
||||
|
||||
String hoodieTableDDL = sql("t1")
|
||||
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
|
||||
|
||||
@@ -58,7 +58,7 @@ public class TestCompactionUtil {
|
||||
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
|
||||
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf, null);
|
||||
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf);
|
||||
HoodieFlinkTable table = writeClient.getHoodieTable();
|
||||
HoodieTableMetaClient metaClient = table.getMetaClient();
|
||||
|
||||
|
||||
@@ -19,6 +19,8 @@
|
||||
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;
|
||||
@@ -33,17 +35,33 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
* Common test utils.
|
||||
*/
|
||||
public class TestUtils {
|
||||
|
||||
public static String getLatestCommit(String basePath) {
|
||||
public static String getLastPendingInstant(String basePath) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
return metaClient.getCommitsAndCompactionTimeline().filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
return StreamerUtil.getLastPendingInstant(metaClient);
|
||||
}
|
||||
|
||||
public static String getFirstCommit(String basePath) {
|
||||
public static String getLastCompleteInstant(String basePath) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
return metaClient.getCommitsAndCompactionTimeline().filterCompletedInstants().firstInstant().get().getTimestamp();
|
||||
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);
|
||||
}
|
||||
|
||||
public static String getSplitPartitionPath(MergeOnReadInputSplit split) {
|
||||
|
||||
@@ -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));
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user