1
0

[HUDI-2469] [Kafka Connect] Replace json based payload with protobuf for Transaction protocol. (#3694)

* Substitue Control Event with protobuf

* Fix tests

* Fix unit tests

* Add javadocs

* Add javadocs

* Address reviewer comments

Co-authored-by: Rajesh Mahindra <rmahindra@Rajeshs-MacBook-Pro.local>
This commit is contained in:
rmahindra123
2021-10-19 14:29:48 -07:00
committed by GitHub
parent 46f0496a08
commit 3686c25fae
24 changed files with 354 additions and 461 deletions

View File

@@ -22,9 +22,9 @@ import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.connect.transaction.ConnectTransactionCoordinator;
import org.apache.hudi.connect.transaction.ControlEvent;
import org.apache.hudi.connect.transaction.TransactionCoordinator;
import org.apache.hudi.connect.transaction.TransactionParticipant;
import org.apache.hudi.connect.utils.KafkaConnectUtils;
import org.apache.hudi.connect.writers.KafkaConnectConfigs;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.helper.MockConnectTransactionServices;
@@ -108,7 +108,7 @@ public class TestConnectTransactionCoordinator {
private final int maxNumberCommitRounds;
private final Map<Integer, Long> kafkaOffsetsCommitted;
private ControlEvent.MsgType expectedMsgType;
private ControlMessage.EventType expectedMsgType;
private int numberCommitRounds;
public MockParticipant(MockKafkaControlAgent kafkaControlAgent,
@@ -121,7 +121,7 @@ public class TestConnectTransactionCoordinator {
this.maxNumberCommitRounds = maxNumberCommitRounds;
this.partition = new TopicPartition(TOPIC_NAME, (NUM_PARTITIONS - 1));
this.kafkaOffsetsCommitted = new HashMap<>();
expectedMsgType = ControlEvent.MsgType.START_COMMIT;
expectedMsgType = ControlMessage.EventType.START_COMMIT;
numberCommitRounds = 0;
}
@@ -149,9 +149,9 @@ public class TestConnectTransactionCoordinator {
}
@Override
public void processControlEvent(ControlEvent message) {
assertEquals(message.getSenderType(), ControlEvent.SenderType.COORDINATOR);
assertEquals(message.senderPartition().topic(), partition.topic());
public void processControlEvent(ControlMessage message) {
assertEquals(message.getSenderType(), ControlMessage.EntityType.COORDINATOR);
assertEquals(message.getTopicName(), partition.topic());
testScenarios(message);
}
@@ -160,24 +160,24 @@ public class TestConnectTransactionCoordinator {
return 0;
}
private void testScenarios(ControlEvent message) {
assertEquals(expectedMsgType, message.getMsgType());
private void testScenarios(ControlMessage message) {
assertEquals(expectedMsgType, message.getType());
switch (message.getMsgType()) {
switch (message.getType()) {
case START_COMMIT:
expectedMsgType = ControlEvent.MsgType.END_COMMIT;
expectedMsgType = ControlMessage.EventType.END_COMMIT;
break;
case END_COMMIT:
assertEquals(kafkaOffsetsCommitted, message.getCoordinatorInfo().getGlobalKafkaCommitOffsets());
int numSuccessPartitions;
Map<Integer, Long> kafkaOffsets = new HashMap<>();
List<ControlEvent> controlEvents = new ArrayList<>();
List<ControlMessage> controlEvents = new ArrayList<>();
// Prepare the WriteStatuses for all partitions
for (int i = 1; i <= NUM_PARTITIONS; i++) {
try {
long kafkaOffset = (long) (Math.random() * 10000);
kafkaOffsets.put(i, kafkaOffset);
ControlEvent event = successWriteStatus(
ControlMessage event = successWriteStatus(
message.getCommitTime(),
new TopicPartition(TOPIC_NAME, i),
kafkaOffset);
@@ -191,11 +191,11 @@ public class TestConnectTransactionCoordinator {
case ALL_CONNECT_TASKS_SUCCESS:
numSuccessPartitions = NUM_PARTITIONS;
kafkaOffsetsCommitted.putAll(kafkaOffsets);
expectedMsgType = ControlEvent.MsgType.ACK_COMMIT;
expectedMsgType = ControlMessage.EventType.ACK_COMMIT;
break;
case SUBSET_CONNECT_TASKS_FAILED:
numSuccessPartitions = NUM_PARTITIONS / 2;
expectedMsgType = ControlEvent.MsgType.START_COMMIT;
expectedMsgType = ControlMessage.EventType.START_COMMIT;
break;
default:
throw new HoodieException("Unknown test scenario " + testScenario);
@@ -210,18 +210,18 @@ public class TestConnectTransactionCoordinator {
if (numberCommitRounds >= maxNumberCommitRounds) {
latch.countDown();
}
expectedMsgType = ControlEvent.MsgType.START_COMMIT;
expectedMsgType = ControlMessage.EventType.START_COMMIT;
break;
default:
throw new HoodieException("Illegal control message type " + message.getMsgType());
throw new HoodieException("Illegal control message type " + message.getType());
}
if (message.getMsgType().equals(ControlEvent.MsgType.START_COMMIT)) {
if (message.getType().equals(ControlMessage.EventType.START_COMMIT)) {
if (numberCommitRounds >= maxNumberCommitRounds) {
latch.countDown();
}
numberCommitRounds++;
expectedMsgType = ControlEvent.MsgType.END_COMMIT;
expectedMsgType = ControlMessage.EventType.END_COMMIT;
}
}
@@ -230,24 +230,29 @@ public class TestConnectTransactionCoordinator {
ALL_CONNECT_TASKS_SUCCESS
}
private static ControlEvent successWriteStatus(String commitTime,
TopicPartition partition,
long kafkaOffset) throws Exception {
private static ControlMessage successWriteStatus(String commitTime,
TopicPartition partition,
long kafkaOffset) throws Exception {
// send WS
WriteStatus writeStatus = new WriteStatus();
WriteStatus status = new WriteStatus(false, 1.0);
for (int i = 0; i < 1000; i++) {
status.markSuccess(mock(HoodieRecord.class), Option.empty());
}
return new ControlEvent.Builder(ControlEvent.MsgType.WRITE_STATUS,
ControlEvent.SenderType.PARTICIPANT,
commitTime,
partition)
.setParticipantInfo(new ControlEvent.ParticipantInfo(
Collections.singletonList(writeStatus),
kafkaOffset,
ControlEvent.OutcomeType.WRITE_SUCCESS))
.build();
return ControlMessage.newBuilder()
.setType(ControlMessage.EventType.WRITE_STATUS)
.setTopicName(partition.topic())
.setSenderType(ControlMessage.EntityType.PARTICIPANT)
.setSenderPartition(partition.partition())
.setReceiverType(ControlMessage.EntityType.COORDINATOR)
.setReceiverPartition(ConnectTransactionCoordinator.COORDINATOR_KAFKA_PARTITION)
.setCommitTime(commitTime)
.setParticipantInfo(
ControlMessage.ParticipantInfo.newBuilder()
.setWriteStatus(KafkaConnectUtils.buildWriteStatuses(Collections.singletonList(writeStatus)))
.setKafkaOffset(kafkaOffset)
.build()
).build();
}
}
}

View File

@@ -21,7 +21,6 @@ package org.apache.hudi.connect;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.connect.kafka.KafkaControlAgent;
import org.apache.hudi.connect.transaction.ConnectTransactionParticipant;
import org.apache.hudi.connect.transaction.ControlEvent;
import org.apache.hudi.connect.transaction.TransactionCoordinator;
import org.apache.hudi.connect.writers.KafkaConnectConfigs;
import org.apache.hudi.exception.HoodieException;
@@ -76,16 +75,16 @@ public class TestConnectTransactionParticipant {
break;
case COORDINATOR_FAILED_AFTER_START_COMMIT:
testKafkaConnect.putRecordsToParticipant();
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.START_COMMIT);
testKafkaConnect.putRecordsToParticipant();
// Coordinator Failed
initializeCoordinator();
break;
case COORDINATOR_FAILED_AFTER_END_COMMIT:
testKafkaConnect.putRecordsToParticipant();
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.START_COMMIT);
testKafkaConnect.putRecordsToParticipant();
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.END_COMMIT);
expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant();
// Coordinator Failed
initializeCoordinator();
@@ -95,13 +94,13 @@ public class TestConnectTransactionParticipant {
}
// Regular Case or Coordinator Recovery Case
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.START_COMMIT);
expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant();
assertTrue(testKafkaConnect.isResumed());
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.END_COMMIT);
testKafkaConnect.putRecordsToParticipant();
assertTrue(testKafkaConnect.isPaused());
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.ACK_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.ACK_COMMIT);
testKafkaConnect.putRecordsToParticipant();
assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten);
// Ensure Coordinator and participant are in sync in the kafka offsets
@@ -122,13 +121,13 @@ public class TestConnectTransactionParticipant {
testKafkaConnect.putRecordsToParticipant();
// Participant fails
initializeParticipant();
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.START_COMMIT);
expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant();
assertTrue(testKafkaConnect.isResumed());
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.END_COMMIT);
testKafkaConnect.putRecordsToParticipant();
assertTrue(testKafkaConnect.isPaused());
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.ACK_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.ACK_COMMIT);
testKafkaConnect.putRecordsToParticipant();
assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten);
// Ensure Coordinator and participant are in sync in the kafka offsets
@@ -136,15 +135,15 @@ public class TestConnectTransactionParticipant {
break;
case FAILURE_AFTER_START_COMMIT:
testKafkaConnect.putRecordsToParticipant();
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.START_COMMIT);
testKafkaConnect.putRecordsToParticipant();
// Participant fails
initializeParticipant();
testKafkaConnect.putRecordsToParticipant();
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.END_COMMIT);
testKafkaConnect.putRecordsToParticipant();
assertTrue(testKafkaConnect.isPaused());
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.ACK_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.ACK_COMMIT);
testKafkaConnect.putRecordsToParticipant();
assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten);
// Ensure Coordinator and participant are in sync in the kafka offsets
@@ -152,17 +151,17 @@ public class TestConnectTransactionParticipant {
break;
case FAILURE_AFTER_END_COMMIT:
testKafkaConnect.putRecordsToParticipant();
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.START_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.START_COMMIT);
testKafkaConnect.putRecordsToParticipant();
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.END_COMMIT);
testKafkaConnect.putRecordsToParticipant();
// Participant fails
initializeParticipant();
testKafkaConnect.putRecordsToParticipant();
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.END_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.END_COMMIT);
testKafkaConnect.putRecordsToParticipant();
assertTrue(testKafkaConnect.isPaused());
coordinator.sendEventFromCoordinator(ControlEvent.MsgType.ACK_COMMIT);
coordinator.sendEventFromCoordinator(ControlMessage.EventType.ACK_COMMIT);
testKafkaConnect.putRecordsToParticipant();
assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten);
// Ensure Coordinator and participant are in sync in the kafka offsets
@@ -203,7 +202,7 @@ public class TestConnectTransactionParticipant {
private final KafkaControlAgent kafkaControlAgent;
private final TopicPartition partition;
private Option<ControlEvent> lastReceivedWriteStatusEvent;
private Option<ControlMessage> lastReceivedWriteStatusEvent;
private long committedKafkaOffset;
public MockCoordinator(KafkaControlAgent kafkaControlAgent) {
@@ -213,26 +212,30 @@ public class TestConnectTransactionParticipant {
committedKafkaOffset = 0L;
}
public void sendEventFromCoordinator(
ControlEvent.MsgType type) {
public void sendEventFromCoordinator(ControlMessage.EventType type) {
try {
if (type.equals(ControlEvent.MsgType.START_COMMIT)) {
if (type.equals(ControlMessage.EventType.START_COMMIT)) {
++currentCommitTime;
}
kafkaControlAgent.publishMessage(new ControlEvent.Builder(
type,
ControlEvent.SenderType.COORDINATOR,
String.valueOf(currentCommitTime),
partition)
.setCoordinatorInfo(new ControlEvent.CoordinatorInfo(
Collections.singletonMap(PARTITION_NUMBER, committedKafkaOffset)))
.build());
kafkaControlAgent.publishMessage(
ControlMessage.newBuilder()
.setType(type)
.setTopicName(partition.topic())
.setSenderType(ControlMessage.EntityType.COORDINATOR)
.setSenderPartition(partition.partition())
.setReceiverType(ControlMessage.EntityType.PARTICIPANT)
.setCommitTime(String.valueOf(currentCommitTime))
.setCoordinatorInfo(
ControlMessage.CoordinatorInfo.newBuilder()
.putAllGlobalKafkaCommitOffsets(Collections.singletonMap(PARTITION_NUMBER, committedKafkaOffset))
.build()
).build());
} catch (Exception exception) {
throw new HoodieException("Fatal error sending control event to Participant");
}
}
public Option<ControlEvent> getLastReceivedWriteStatusEvent() {
public Option<ControlMessage> getLastReceivedWriteStatusEvent() {
return lastReceivedWriteStatusEvent;
}
@@ -256,11 +259,11 @@ public class TestConnectTransactionParticipant {
}
@Override
public void processControlEvent(ControlEvent message) {
if (message.getMsgType().equals(ControlEvent.MsgType.WRITE_STATUS)) {
public void processControlEvent(ControlMessage message) {
if (message.getType().equals(ControlMessage.EventType.WRITE_STATUS)) {
lastReceivedWriteStatusEvent = Option.of(message);
assertTrue(message.getParticipantInfo().getKafkaCommitOffset() >= committedKafkaOffset);
committedKafkaOffset = message.getParticipantInfo().getKafkaCommitOffset();
assertTrue(message.getParticipantInfo().getKafkaOffset() >= committedKafkaOffset);
committedKafkaOffset = message.getParticipantInfo().getKafkaOffset();
}
}
}

View File

@@ -18,8 +18,8 @@
package org.apache.hudi.helper;
import org.apache.hudi.connect.ControlMessage;
import org.apache.hudi.connect.kafka.KafkaControlAgent;
import org.apache.hudi.connect.transaction.ControlEvent;
import org.apache.hudi.connect.transaction.TransactionCoordinator;
import org.apache.hudi.connect.transaction.TransactionParticipant;
import org.apache.hudi.exception.HoodieException;
@@ -70,10 +70,10 @@ public class MockKafkaControlAgent implements KafkaControlAgent {
}
@Override
public void publishMessage(ControlEvent message) {
public void publishMessage(ControlMessage message) {
try {
String topic = message.senderPartition().topic();
if (message.getSenderType().equals(ControlEvent.SenderType.COORDINATOR)) {
String topic = message.getTopicName();
if (message.getSenderType().equals(ControlMessage.EntityType.COORDINATOR)) {
for (TransactionParticipant participant : participants.get(topic)) {
participant.processControlEvent(message);
}