[HUDI-2428] Fix protocol and other issues after stress testing Hudi Kafka Connect (#3656)
* Fixes based on tests and some improvements * Fix the issues after running stress tests * Fixing checkstyle issues and updating README Co-authored-by: Rajesh Mahindra <rmahindra@Rajeshs-MacBook-Pro.local> Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -68,44 +68,47 @@ public class TestConnectTransactionParticipant {
|
||||
@EnumSource(value = CoordinatorFailureTestScenarios.class)
|
||||
public void testAllCoordinatorFailureScenarios(CoordinatorFailureTestScenarios testScenario) {
|
||||
int expectedRecordsWritten = 0;
|
||||
switch (testScenario) {
|
||||
case REGULAR_SCENARIO:
|
||||
expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant();
|
||||
assertTrue(testKafkaConnect.isPaused());
|
||||
break;
|
||||
case COORDINATOR_FAILED_AFTER_START_COMMIT:
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
// Coordinator Failed
|
||||
initializeCoordinator();
|
||||
break;
|
||||
case COORDINATOR_FAILED_AFTER_END_COMMIT:
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
|
||||
expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant();
|
||||
// Coordinator Failed
|
||||
initializeCoordinator();
|
||||
break;
|
||||
default:
|
||||
throw new HoodieException("Unknown test scenario " + testScenario);
|
||||
try {
|
||||
switch (testScenario) {
|
||||
case REGULAR_SCENARIO:
|
||||
expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant();
|
||||
assertTrue(testKafkaConnect.isPaused());
|
||||
break;
|
||||
case COORDINATOR_FAILED_AFTER_START_COMMIT:
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
// Coordinator Failed
|
||||
initializeCoordinator();
|
||||
break;
|
||||
case COORDINATOR_FAILED_AFTER_END_COMMIT:
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
|
||||
expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant();
|
||||
// Coordinator Failed
|
||||
initializeCoordinator();
|
||||
break;
|
||||
default:
|
||||
throw new HoodieException("Unknown test scenario " + testScenario);
|
||||
}
|
||||
|
||||
// Regular Case or Coordinator Recovery Case
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
|
||||
expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant();
|
||||
assertTrue(testKafkaConnect.isResumed());
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertTrue(testKafkaConnect.isPaused());
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.ACK_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten);
|
||||
// Ensure Coordinator and participant are in sync in the kafka offsets
|
||||
assertEquals(participant.getLastKafkaCommittedOffset(), coordinator.getCommittedKafkaOffset());
|
||||
} catch (Exception exception) {
|
||||
throw new HoodieException("Unexpected test failure ", exception);
|
||||
}
|
||||
|
||||
// Regular Case or Coordinator Recovery Case
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
|
||||
expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant();
|
||||
assertTrue(testKafkaConnect.isResumed());
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertTrue(testKafkaConnect.isPaused());
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.ACK_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten);
|
||||
// Ensure Coordinator and participant are in sync in the kafka offsets
|
||||
assertEquals(participant.getLastKafkaCommittedOffset(), coordinator.getCommittedKafkaOffset());
|
||||
|
||||
participant.stop();
|
||||
}
|
||||
|
||||
@@ -113,59 +116,63 @@ public class TestConnectTransactionParticipant {
|
||||
@EnumSource(value = ParticipantFailureTestScenarios.class)
|
||||
public void testAllParticipantFailureScenarios(ParticipantFailureTestScenarios testScenario) {
|
||||
int expectedRecordsWritten = 0;
|
||||
switch (testScenario) {
|
||||
case FAILURE_BEFORE_START_COMMIT:
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
// Participant fails
|
||||
initializeParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
|
||||
expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant();
|
||||
assertTrue(testKafkaConnect.isResumed());
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertTrue(testKafkaConnect.isPaused());
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.ACK_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten);
|
||||
// Ensure Coordinator and participant are in sync in the kafka offsets
|
||||
assertEquals(participant.getLastKafkaCommittedOffset(), coordinator.getCommittedKafkaOffset());
|
||||
break;
|
||||
case FAILURE_AFTER_START_COMMIT:
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
// Participant fails
|
||||
initializeParticipant();
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertTrue(testKafkaConnect.isPaused());
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.ACK_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten);
|
||||
// Ensure Coordinator and participant are in sync in the kafka offsets
|
||||
assertEquals(participant.getLastKafkaCommittedOffset(), coordinator.getCommittedKafkaOffset());
|
||||
break;
|
||||
case FAILURE_AFTER_END_COMMIT:
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
// Participant fails
|
||||
initializeParticipant();
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertTrue(testKafkaConnect.isPaused());
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.ACK_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten);
|
||||
// Ensure Coordinator and participant are in sync in the kafka offsets
|
||||
assertEquals(participant.getLastKafkaCommittedOffset(), coordinator.getCommittedKafkaOffset());
|
||||
break;
|
||||
default:
|
||||
throw new HoodieException("Unknown test scenario " + testScenario);
|
||||
try {
|
||||
switch (testScenario) {
|
||||
case FAILURE_BEFORE_START_COMMIT:
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
// Participant fails
|
||||
initializeParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
|
||||
expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant();
|
||||
assertTrue(testKafkaConnect.isResumed());
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertTrue(testKafkaConnect.isPaused());
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.ACK_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten);
|
||||
// Ensure Coordinator and participant are in sync in the kafka offsets
|
||||
assertEquals(participant.getLastKafkaCommittedOffset(), coordinator.getCommittedKafkaOffset());
|
||||
break;
|
||||
case FAILURE_AFTER_START_COMMIT:
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
// Participant fails
|
||||
initializeParticipant();
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertTrue(testKafkaConnect.isPaused());
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.ACK_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten);
|
||||
// Ensure Coordinator and participant are in sync in the kafka offsets
|
||||
assertEquals(participant.getLastKafkaCommittedOffset(), coordinator.getCommittedKafkaOffset());
|
||||
break;
|
||||
case FAILURE_AFTER_END_COMMIT:
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
// Participant fails
|
||||
initializeParticipant();
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertTrue(testKafkaConnect.isPaused());
|
||||
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.ACK_COMMIT);
|
||||
testKafkaConnect.putRecordsToParticipant();
|
||||
assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten);
|
||||
// Ensure Coordinator and participant are in sync in the kafka offsets
|
||||
assertEquals(participant.getLastKafkaCommittedOffset(), coordinator.getCommittedKafkaOffset());
|
||||
break;
|
||||
default:
|
||||
throw new HoodieException("Unknown test scenario " + testScenario);
|
||||
}
|
||||
} catch (Exception exception) {
|
||||
throw new HoodieException("Unexpected test failure ", exception);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -25,6 +25,7 @@ import org.apache.kafka.connect.data.Schema;
|
||||
import org.apache.kafka.connect.sink.SinkRecord;
|
||||
import org.apache.kafka.connect.sink.SinkTaskContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
@@ -60,7 +61,7 @@ public class TestKafkaConnect implements SinkTaskContext {
|
||||
return !isPaused;
|
||||
}
|
||||
|
||||
public int putRecordsToParticipant() {
|
||||
public int putRecordsToParticipant() throws IOException {
|
||||
for (int i = 1; i <= NUM_RECORDS_BATCH; i++) {
|
||||
participant.buffer(getNextKafkaRecord());
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user