[HUDI-2394] Implement Kafka Sink Protocol for Hudi for Ingesting Immutable Data (#3592)
- Fixing packaging, naming of classes - Use of log4j over slf4j for uniformity - More follow-on fixes - Added a version to control/coordinator events. - Eliminated the config added to write config - Fixed fetching of checkpoints based on table type - Clean up of naming, code placement Co-authored-by: Rajesh Mahindra <rmahindra@Rajeshs-MacBook-Pro.local> Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -0,0 +1,253 @@
|
||||
/*
|
||||
* 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.connect;
|
||||
|
||||
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.writers.KafkaConnectConfigs;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.helper.MockConnectTransactionServices;
|
||||
import org.apache.hudi.helper.MockKafkaControlAgent;
|
||||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.connect.sink.SinkRecord;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.EnumSource;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
public class TestConnectTransactionCoordinator {
|
||||
|
||||
private static final String TOPIC_NAME = "kafka-connect-test-topic";
|
||||
private static final int NUM_PARTITIONS = 4;
|
||||
private static final int MAX_COMMIT_ROUNDS = 5;
|
||||
private static final int TEST_TIMEOUT_SECS = 60;
|
||||
|
||||
private KafkaConnectConfigs configs;
|
||||
private MockParticipant participant;
|
||||
private MockKafkaControlAgent kafkaControlAgent;
|
||||
private MockConnectTransactionServices transactionServices;
|
||||
private CountDownLatch latch;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
transactionServices = new MockConnectTransactionServices();
|
||||
configs = KafkaConnectConfigs.newBuilder()
|
||||
.withCommitIntervalSecs(1L)
|
||||
.withCoordinatorWriteTimeoutSecs(1L)
|
||||
.build();
|
||||
latch = new CountDownLatch(1);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = MockParticipant.TestScenarios.class)
|
||||
public void testSingleCommitScenario(MockParticipant.TestScenarios scenario) throws InterruptedException {
|
||||
kafkaControlAgent = new MockKafkaControlAgent();
|
||||
participant = new MockParticipant(kafkaControlAgent, latch, scenario, MAX_COMMIT_ROUNDS);
|
||||
participant.start();
|
||||
|
||||
// Test the coordinator using the mock participant
|
||||
TransactionCoordinator coordinator = new ConnectTransactionCoordinator(
|
||||
configs,
|
||||
new TopicPartition(TOPIC_NAME, 0),
|
||||
kafkaControlAgent,
|
||||
transactionServices,
|
||||
(bootstrapServers, topicName) -> NUM_PARTITIONS);
|
||||
coordinator.start();
|
||||
|
||||
latch.await(TEST_TIMEOUT_SECS, TimeUnit.SECONDS);
|
||||
|
||||
if (latch.getCount() > 0) {
|
||||
throw new HoodieException("Test timedout resulting in failure");
|
||||
}
|
||||
coordinator.stop();
|
||||
participant.stop();
|
||||
}
|
||||
|
||||
/**
|
||||
* A mock Transaction Participant, that exercises all the test scenarios
|
||||
* for the coordinator as mentioned in {@link TestScenarios}.
|
||||
*/
|
||||
private static class MockParticipant implements TransactionParticipant {
|
||||
|
||||
private final MockKafkaControlAgent kafkaControlAgent;
|
||||
private final TopicPartition partition;
|
||||
private final CountDownLatch latch;
|
||||
private final TestScenarios testScenario;
|
||||
private final int maxNumberCommitRounds;
|
||||
private final Map<Integer, Long> kafkaOffsetsCommitted;
|
||||
|
||||
private ControlEvent.MsgType expectedMsgType;
|
||||
private int numberCommitRounds;
|
||||
|
||||
public MockParticipant(MockKafkaControlAgent kafkaControlAgent,
|
||||
CountDownLatch latch,
|
||||
TestScenarios testScenario,
|
||||
int maxNumberCommitRounds) {
|
||||
this.kafkaControlAgent = kafkaControlAgent;
|
||||
this.latch = latch;
|
||||
this.testScenario = testScenario;
|
||||
this.maxNumberCommitRounds = maxNumberCommitRounds;
|
||||
this.partition = new TopicPartition(TOPIC_NAME, (NUM_PARTITIONS - 1));
|
||||
this.kafkaOffsetsCommitted = new HashMap<>();
|
||||
expectedMsgType = ControlEvent.MsgType.START_COMMIT;
|
||||
numberCommitRounds = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
kafkaControlAgent.registerTransactionParticipant(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
kafkaControlAgent.deregisterTransactionParticipant(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void buffer(SinkRecord record) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processRecords() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public TopicPartition getPartition() {
|
||||
return partition;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processControlEvent(ControlEvent message) {
|
||||
assertEquals(message.getSenderType(), ControlEvent.SenderType.COORDINATOR);
|
||||
assertEquals(message.senderPartition().topic(), partition.topic());
|
||||
testScenarios(message);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLastKafkaCommittedOffset() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
private void testScenarios(ControlEvent message) {
|
||||
assertEquals(expectedMsgType, message.getMsgType());
|
||||
|
||||
switch (message.getMsgType()) {
|
||||
case START_COMMIT:
|
||||
expectedMsgType = ControlEvent.MsgType.END_COMMIT;
|
||||
break;
|
||||
case END_COMMIT:
|
||||
assertEquals(kafkaOffsetsCommitted, message.getCoordinatorInfo().getGlobalKafkaCommitOffsets());
|
||||
int numSuccessPartitions;
|
||||
Map<Integer, Long> kafkaOffsets = new HashMap<>();
|
||||
List<ControlEvent> 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(
|
||||
message.getCommitTime(),
|
||||
new TopicPartition(TOPIC_NAME, i),
|
||||
kafkaOffset);
|
||||
controlEvents.add(event);
|
||||
} catch (Exception exception) {
|
||||
throw new HoodieException("Fatal error sending control event to Coordinator");
|
||||
}
|
||||
}
|
||||
|
||||
switch (testScenario) {
|
||||
case ALL_CONNECT_TASKS_SUCCESS:
|
||||
numSuccessPartitions = NUM_PARTITIONS;
|
||||
kafkaOffsetsCommitted.putAll(kafkaOffsets);
|
||||
expectedMsgType = ControlEvent.MsgType.ACK_COMMIT;
|
||||
break;
|
||||
case SUBSET_CONNECT_TASKS_FAILED:
|
||||
numSuccessPartitions = NUM_PARTITIONS / 2;
|
||||
expectedMsgType = ControlEvent.MsgType.START_COMMIT;
|
||||
break;
|
||||
default:
|
||||
throw new HoodieException("Unknown test scenario " + testScenario);
|
||||
}
|
||||
|
||||
// Send events based on test scenario
|
||||
for (int i = 0; i < numSuccessPartitions; i++) {
|
||||
kafkaControlAgent.publishMessage(controlEvents.get(i));
|
||||
}
|
||||
break;
|
||||
case ACK_COMMIT:
|
||||
if (numberCommitRounds >= maxNumberCommitRounds) {
|
||||
latch.countDown();
|
||||
}
|
||||
expectedMsgType = ControlEvent.MsgType.START_COMMIT;
|
||||
break;
|
||||
default:
|
||||
throw new HoodieException("Illegal control message type " + message.getMsgType());
|
||||
}
|
||||
|
||||
if (message.getMsgType().equals(ControlEvent.MsgType.START_COMMIT)) {
|
||||
if (numberCommitRounds >= maxNumberCommitRounds) {
|
||||
latch.countDown();
|
||||
}
|
||||
numberCommitRounds++;
|
||||
expectedMsgType = ControlEvent.MsgType.END_COMMIT;
|
||||
}
|
||||
}
|
||||
|
||||
public enum TestScenarios {
|
||||
SUBSET_CONNECT_TASKS_FAILED,
|
||||
ALL_CONNECT_TASKS_SUCCESS
|
||||
}
|
||||
|
||||
private static ControlEvent 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();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,273 @@
|
||||
/*
|
||||
* 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.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;
|
||||
import org.apache.hudi.helper.MockKafkaControlAgent;
|
||||
import org.apache.hudi.helper.TestHudiWriterProvider;
|
||||
import org.apache.hudi.helper.TestKafkaConnect;
|
||||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.EnumSource;
|
||||
|
||||
import java.util.Collections;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestConnectTransactionParticipant {
|
||||
|
||||
private static final String TOPIC_NAME = "kafka-connect-test-topic";
|
||||
private static final int PARTITION_NUMBER = 4;
|
||||
|
||||
private ConnectTransactionParticipant participant;
|
||||
private MockCoordinator coordinator;
|
||||
private TopicPartition partition;
|
||||
private KafkaConnectConfigs configs;
|
||||
private KafkaControlAgent kafkaControlAgent;
|
||||
private TestHudiWriterProvider testHudiWriterProvider;
|
||||
private TestKafkaConnect testKafkaConnect;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
partition = new TopicPartition(TOPIC_NAME, PARTITION_NUMBER);
|
||||
kafkaControlAgent = new MockKafkaControlAgent();
|
||||
testKafkaConnect = new TestKafkaConnect(partition);
|
||||
coordinator = new MockCoordinator(kafkaControlAgent);
|
||||
coordinator.start();
|
||||
configs = KafkaConnectConfigs.newBuilder()
|
||||
.build();
|
||||
initializeParticipant();
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@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);
|
||||
}
|
||||
|
||||
// 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();
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@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);
|
||||
}
|
||||
}
|
||||
|
||||
private void initializeParticipant() {
|
||||
testHudiWriterProvider = new TestHudiWriterProvider();
|
||||
participant = new ConnectTransactionParticipant(
|
||||
partition,
|
||||
kafkaControlAgent,
|
||||
testKafkaConnect,
|
||||
testHudiWriterProvider);
|
||||
testKafkaConnect.setParticipant(participant);
|
||||
participant.start();
|
||||
}
|
||||
|
||||
private void initializeCoordinator() {
|
||||
coordinator = new MockCoordinator(kafkaControlAgent);
|
||||
coordinator.start();
|
||||
}
|
||||
|
||||
private static class MockCoordinator implements TransactionCoordinator {
|
||||
|
||||
private static int currentCommitTime;
|
||||
|
||||
static {
|
||||
currentCommitTime = 101;
|
||||
}
|
||||
|
||||
private final KafkaControlAgent kafkaControlAgent;
|
||||
private final TopicPartition partition;
|
||||
|
||||
private Option<ControlEvent> lastReceivedWriteStatusEvent;
|
||||
private long committedKafkaOffset;
|
||||
|
||||
public MockCoordinator(KafkaControlAgent kafkaControlAgent) {
|
||||
this.kafkaControlAgent = kafkaControlAgent;
|
||||
partition = new TopicPartition(TOPIC_NAME, 0);
|
||||
lastReceivedWriteStatusEvent = Option.empty();
|
||||
committedKafkaOffset = 0L;
|
||||
}
|
||||
|
||||
public void sendEventFromCoordinator(
|
||||
ControlEvent.MsgType type) {
|
||||
try {
|
||||
if (type.equals(ControlEvent.MsgType.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());
|
||||
} catch (Exception exception) {
|
||||
throw new HoodieException("Fatal error sending control event to Participant");
|
||||
}
|
||||
}
|
||||
|
||||
public Option<ControlEvent> getLastReceivedWriteStatusEvent() {
|
||||
return lastReceivedWriteStatusEvent;
|
||||
}
|
||||
|
||||
public long getCommittedKafkaOffset() {
|
||||
return committedKafkaOffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
kafkaControlAgent.registerTransactionCoordinator(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
kafkaControlAgent.deregisterTransactionCoordinator(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TopicPartition getPartition() {
|
||||
return partition;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processControlEvent(ControlEvent message) {
|
||||
if (message.getMsgType().equals(ControlEvent.MsgType.WRITE_STATUS)) {
|
||||
lastReceivedWriteStatusEvent = Option.of(message);
|
||||
assertTrue(message.getParticipantInfo().getKafkaCommitOffset() >= committedKafkaOffset);
|
||||
committedKafkaOffset = message.getParticipantInfo().getKafkaCommitOffset();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private enum CoordinatorFailureTestScenarios {
|
||||
REGULAR_SCENARIO,
|
||||
COORDINATOR_FAILED_AFTER_START_COMMIT,
|
||||
COORDINATOR_FAILED_AFTER_END_COMMIT,
|
||||
}
|
||||
|
||||
private enum ParticipantFailureTestScenarios {
|
||||
FAILURE_BEFORE_START_COMMIT,
|
||||
FAILURE_AFTER_START_COMMIT,
|
||||
FAILURE_AFTER_END_COMMIT,
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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.helper;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.connect.writers.ConnectTransactionServices;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
/**
|
||||
* Helper class for {@link ConnectTransactionServices} to generate
|
||||
* a unique commit time for testing purposes.
|
||||
*/
|
||||
public class MockConnectTransactionServices implements ConnectTransactionServices {
|
||||
|
||||
private int commitTime;
|
||||
|
||||
public MockConnectTransactionServices() {
|
||||
commitTime = 100;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String startCommit() {
|
||||
commitTime++;
|
||||
return String.valueOf(commitTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void endCommit(String commitTime, List<WriteStatus> writeStatuses, Map<String, String> extraMetadata) {
|
||||
assertEquals(String.valueOf(this.commitTime), commitTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, String> fetchLatestExtraCommitMetadata() {
|
||||
return new HashMap<>();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,87 @@
|
||||
/*
|
||||
* 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.helper;
|
||||
|
||||
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;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A mock Kafka Control Agent that supports the testing
|
||||
* of a {@link TransactionCoordinator} with multiple
|
||||
* instances of {@link TransactionParticipant}.
|
||||
*/
|
||||
public class MockKafkaControlAgent implements KafkaControlAgent {
|
||||
|
||||
private final Map<String, TransactionCoordinator> coordinators;
|
||||
private final Map<String, List<TransactionParticipant>> participants;
|
||||
|
||||
public MockKafkaControlAgent() {
|
||||
coordinators = new HashMap<>();
|
||||
participants = new HashMap<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void registerTransactionCoordinator(TransactionCoordinator coordinator) {
|
||||
coordinators.put(coordinator.getPartition().topic(), coordinator);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void registerTransactionParticipant(TransactionParticipant participant) {
|
||||
if (!participants.containsKey(participant.getPartition().topic())) {
|
||||
participants.put(participant.getPartition().topic(), new ArrayList<>());
|
||||
}
|
||||
participants.get(participant.getPartition().topic()).add(participant);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deregisterTransactionCoordinator(TransactionCoordinator coordinator) {
|
||||
coordinators.remove(coordinator.getPartition().topic());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deregisterTransactionParticipant(TransactionParticipant worker) {
|
||||
if (participants.containsKey(worker.getPartition().topic())) {
|
||||
participants.get(worker.getPartition().topic()).remove(worker);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void publishMessage(ControlEvent message) {
|
||||
try {
|
||||
String topic = message.senderPartition().topic();
|
||||
if (message.getSenderType().equals(ControlEvent.SenderType.COORDINATOR)) {
|
||||
for (TransactionParticipant participant : participants.get(topic)) {
|
||||
participant.processControlEvent(message);
|
||||
}
|
||||
} else {
|
||||
coordinators.get(topic).processControlEvent(message);
|
||||
}
|
||||
} catch (Exception exception) {
|
||||
throw new HoodieException("Fatal error trying to relay Kafka Control Messages for Testing.");
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,83 @@
|
||||
/*
|
||||
* 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.helper;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.connect.writers.ConnectWriter;
|
||||
import org.apache.hudi.connect.writers.ConnectWriterProvider;
|
||||
|
||||
import org.apache.kafka.connect.sink.SinkRecord;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Helper class the provides a Hudi writer and
|
||||
* maintains stats that are used for test validation.
|
||||
*/
|
||||
public class TestHudiWriterProvider implements ConnectWriterProvider<WriteStatus> {
|
||||
|
||||
private TestHudiWriter currentWriter;
|
||||
|
||||
public TestHudiWriterProvider() {
|
||||
}
|
||||
|
||||
public int getLatestNumberWrites() {
|
||||
return (currentWriter != null) ? currentWriter.numberRecords : 0;
|
||||
}
|
||||
|
||||
public boolean isClosed() {
|
||||
return currentWriter == null || currentWriter.isClosed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ConnectWriter<WriteStatus> getWriter(String commitTime) {
|
||||
currentWriter = new TestHudiWriter();
|
||||
return currentWriter;
|
||||
}
|
||||
|
||||
private static class TestHudiWriter implements ConnectWriter<WriteStatus> {
|
||||
|
||||
private int numberRecords;
|
||||
private boolean isClosed;
|
||||
|
||||
public TestHudiWriter() {
|
||||
this.numberRecords = 0;
|
||||
this.isClosed = false;
|
||||
}
|
||||
|
||||
public int getNumberRecords() {
|
||||
return numberRecords;
|
||||
}
|
||||
|
||||
public boolean isClosed() {
|
||||
return isClosed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeRecord(SinkRecord record) {
|
||||
numberRecords++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<WriteStatus> close() {
|
||||
isClosed = false;
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,133 @@
|
||||
/*
|
||||
* 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.helper;
|
||||
|
||||
import org.apache.hudi.connect.transaction.TransactionParticipant;
|
||||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.connect.data.Schema;
|
||||
import org.apache.kafka.connect.sink.SinkRecord;
|
||||
import org.apache.kafka.connect.sink.SinkTaskContext;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Helper class that emulates the Kafka Connect f/w and additionally
|
||||
* implements {@link SinkTaskContext} for testing purposes.
|
||||
*/
|
||||
public class TestKafkaConnect implements SinkTaskContext {
|
||||
|
||||
private static final int NUM_RECORDS_BATCH = 5;
|
||||
private final TopicPartition testPartition;
|
||||
|
||||
private TransactionParticipant participant;
|
||||
private long currentKafkaOffset;
|
||||
private boolean isPaused;
|
||||
|
||||
public TestKafkaConnect(TopicPartition testPartition) {
|
||||
this.testPartition = testPartition;
|
||||
isPaused = false;
|
||||
currentKafkaOffset = 0L;
|
||||
}
|
||||
|
||||
public void setParticipant(TransactionParticipant participant) {
|
||||
this.participant = participant;
|
||||
}
|
||||
|
||||
public boolean isPaused() {
|
||||
return isPaused;
|
||||
}
|
||||
|
||||
public boolean isResumed() {
|
||||
return !isPaused;
|
||||
}
|
||||
|
||||
public int putRecordsToParticipant() {
|
||||
for (int i = 1; i <= NUM_RECORDS_BATCH; i++) {
|
||||
participant.buffer(getNextKafkaRecord());
|
||||
}
|
||||
participant.processRecords();
|
||||
return NUM_RECORDS_BATCH;
|
||||
}
|
||||
|
||||
public SinkRecord getNextKafkaRecord() {
|
||||
return new SinkRecord(testPartition.topic(),
|
||||
testPartition.partition(),
|
||||
Schema.OPTIONAL_BYTES_SCHEMA,
|
||||
("key-" + currentKafkaOffset).getBytes(),
|
||||
Schema.OPTIONAL_BYTES_SCHEMA,
|
||||
"value".getBytes(), currentKafkaOffset++);
|
||||
}
|
||||
|
||||
public long getCurrentKafkaOffset() {
|
||||
return currentKafkaOffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pause(TopicPartition... partitions) {
|
||||
if (Arrays.stream(partitions).allMatch(testPartition::equals)) {
|
||||
isPaused = true;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void resume(TopicPartition... partitions) {
|
||||
if (Arrays.stream(partitions).allMatch(testPartition::equals)) {
|
||||
isPaused = false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void offset(Map<TopicPartition, Long> offsets) {
|
||||
for (TopicPartition tp : offsets.keySet()) {
|
||||
if (tp.equals(testPartition)) {
|
||||
currentKafkaOffset = offsets.get(tp);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void offset(TopicPartition tp, long offset) {
|
||||
if (tp.equals(testPartition)) {
|
||||
currentKafkaOffset = offset;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, String> configs() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void timeout(long timeoutMs) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<TopicPartition> assignment() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void requestCommit() {
|
||||
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,203 @@
|
||||
/*
|
||||
* 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.writers;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.SchemaTestUtil;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.connect.writers.AbstractConnectWriter;
|
||||
import org.apache.hudi.connect.writers.KafkaConnectConfigs;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.keygen.KeyGenerator;
|
||||
import org.apache.hudi.schema.SchemaProvider;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericDatumReader;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.avro.io.DecoderFactory;
|
||||
import org.apache.kafka.connect.sink.SinkRecord;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.EnumSource;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestAbstractConnectWriter {
|
||||
|
||||
private static final String TOPIC_NAME = "kafka-connect-test-topic";
|
||||
private static final int PARTITION_NUMBER = 4;
|
||||
private static final int NUM_RECORDS = 10;
|
||||
private static final int RECORD_KEY_INDEX = 0;
|
||||
|
||||
private KafkaConnectConfigs configs;
|
||||
private TestKeyGenerator keyGenerator;
|
||||
private SchemaProvider schemaProvider;
|
||||
private long currentKafkaOffset;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
keyGenerator = new TestKeyGenerator(new TypedProperties());
|
||||
schemaProvider = new TestSchemaProvider();
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = TestInputFormats.class)
|
||||
public void testAbstractWriterForAllFormats(TestInputFormats inputFormats) throws Exception {
|
||||
Schema schema = schemaProvider.getSourceSchema();
|
||||
List<?> inputRecords;
|
||||
List<HoodieRecord> expectedRecords;
|
||||
|
||||
String formatConverter;
|
||||
switch (inputFormats) {
|
||||
case JSON_STRING:
|
||||
formatConverter = AbstractConnectWriter.KAFKA_STRING_CONVERTER;
|
||||
GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(schema, schema);
|
||||
inputRecords = SchemaTestUtil.generateTestJsonRecords(0, NUM_RECORDS);
|
||||
expectedRecords = ((List<String>) inputRecords).stream().map(s -> {
|
||||
try {
|
||||
return HoodieAvroUtils.rewriteRecord((GenericRecord) reader.read(null, DecoderFactory.get().jsonDecoder(schema, s)),
|
||||
schema);
|
||||
} catch (IOException exception) {
|
||||
throw new HoodieException("Error converting JSON records to AVRO");
|
||||
}
|
||||
}).map(p -> convertToHoodieRecords(p, p.get(RECORD_KEY_INDEX).toString(), "000/00/00")).collect(Collectors.toList());
|
||||
break;
|
||||
case AVRO:
|
||||
formatConverter = AbstractConnectWriter.KAFKA_AVRO_CONVERTER;
|
||||
inputRecords = SchemaTestUtil.generateTestRecords(0, NUM_RECORDS);
|
||||
expectedRecords = inputRecords.stream().map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, schema))
|
||||
.map(p -> convertToHoodieRecords(p, p.get(RECORD_KEY_INDEX).toString(), "000/00/00")).collect(Collectors.toList());
|
||||
break;
|
||||
default:
|
||||
throw new HoodieException("Unknown test scenario " + inputFormats);
|
||||
}
|
||||
|
||||
configs = KafkaConnectConfigs.newBuilder()
|
||||
.withProperties(
|
||||
Collections.singletonMap(KafkaConnectConfigs.KAFKA_VALUE_CONVERTER, formatConverter))
|
||||
.build();
|
||||
AbstractHudiConnectWriterTestWrapper writer = new AbstractHudiConnectWriterTestWrapper(
|
||||
configs,
|
||||
keyGenerator,
|
||||
schemaProvider);
|
||||
|
||||
for (int i = 0; i < NUM_RECORDS; i++) {
|
||||
writer.writeRecord(getNextKafkaRecord(inputRecords.get(i)));
|
||||
}
|
||||
|
||||
validateRecords(writer.getWrittenRecords(), expectedRecords);
|
||||
}
|
||||
|
||||
private static void validateRecords(List<HoodieRecord> actualRecords, List<HoodieRecord> expectedRecords) {
|
||||
assertEquals(actualRecords.size(), expectedRecords.size());
|
||||
|
||||
actualRecords.sort(Comparator.comparing(HoodieRecord::getRecordKey));
|
||||
expectedRecords.sort(Comparator.comparing(HoodieRecord::getRecordKey));
|
||||
|
||||
// iterate through the elements and compare them one by one using
|
||||
// the provided comparator.
|
||||
Iterator<HoodieRecord> it1 = actualRecords.iterator();
|
||||
Iterator<HoodieRecord> it2 = expectedRecords.iterator();
|
||||
while (it1.hasNext()) {
|
||||
HoodieRecord t1 = it1.next();
|
||||
HoodieRecord t2 = it2.next();
|
||||
assertEquals(t1.getRecordKey(), t2.getRecordKey());
|
||||
}
|
||||
}
|
||||
|
||||
private SinkRecord getNextKafkaRecord(Object record) {
|
||||
return new SinkRecord(TOPIC_NAME, PARTITION_NUMBER,
|
||||
org.apache.kafka.connect.data.Schema.OPTIONAL_BYTES_SCHEMA,
|
||||
("key-" + currentKafkaOffset).getBytes(),
|
||||
org.apache.kafka.connect.data.Schema.OPTIONAL_BYTES_SCHEMA,
|
||||
record, currentKafkaOffset++);
|
||||
}
|
||||
|
||||
private static class AbstractHudiConnectWriterTestWrapper extends AbstractConnectWriter {
|
||||
|
||||
private List<HoodieRecord> writtenRecords;
|
||||
|
||||
public AbstractHudiConnectWriterTestWrapper(KafkaConnectConfigs connectConfigs, KeyGenerator keyGenerator, SchemaProvider schemaProvider) {
|
||||
super(connectConfigs, keyGenerator, schemaProvider);
|
||||
writtenRecords = new ArrayList<>();
|
||||
}
|
||||
|
||||
public List<HoodieRecord> getWrittenRecords() {
|
||||
return writtenRecords;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeHudiRecord(HoodieRecord<HoodieAvroPayload> record) {
|
||||
writtenRecords.add(record);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<WriteStatus> flushHudiRecords() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private static HoodieRecord convertToHoodieRecords(IndexedRecord iRecord, String key, String partitionPath) {
|
||||
return new HoodieRecord<>(new HoodieKey(key, partitionPath),
|
||||
new HoodieAvroPayload(Option.of((GenericRecord) iRecord)));
|
||||
}
|
||||
|
||||
private enum TestInputFormats {
|
||||
AVRO,
|
||||
JSON_STRING
|
||||
}
|
||||
|
||||
static class TestKeyGenerator extends KeyGenerator {
|
||||
|
||||
protected TestKeyGenerator(TypedProperties config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
return new HoodieKey(record.get(RECORD_KEY_INDEX).toString(), "000/00/00");
|
||||
}
|
||||
}
|
||||
|
||||
static class TestSchemaProvider extends SchemaProvider {
|
||||
|
||||
@Override
|
||||
public Schema getSourceSchema() {
|
||||
try {
|
||||
return SchemaTestUtil.getSimpleSchema();
|
||||
} catch (IOException exception) {
|
||||
throw new HoodieException("Fatal error parsing schema", exception);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,101 @@
|
||||
/*
|
||||
* 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.writers;
|
||||
|
||||
import org.apache.hudi.client.HoodieJavaWriteClient;
|
||||
import org.apache.hudi.client.common.HoodieJavaEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.connect.writers.BufferedConnectWriter;
|
||||
import org.apache.hudi.connect.writers.KafkaConnectConfigs;
|
||||
import org.apache.hudi.schema.SchemaProvider;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.mockito.ArgumentCaptor;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.mockito.ArgumentMatchers.anyList;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.times;
|
||||
|
||||
public class TestBufferedConnectWriter {
|
||||
|
||||
private static final int NUM_RECORDS = 10;
|
||||
private static final String COMMIT_TIME = "101";
|
||||
|
||||
private HoodieJavaWriteClient mockHoodieJavaWriteClient;
|
||||
private HoodieJavaEngineContext javaEngineContext;
|
||||
private KafkaConnectConfigs configs;
|
||||
private HoodieWriteConfig writeConfig;
|
||||
private SchemaProvider schemaProvider;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
mockHoodieJavaWriteClient = mock(HoodieJavaWriteClient.class);
|
||||
Configuration hadoopConf = new Configuration();
|
||||
javaEngineContext = new HoodieJavaEngineContext(hadoopConf);
|
||||
configs = KafkaConnectConfigs.newBuilder().build();
|
||||
schemaProvider = new TestAbstractConnectWriter.TestSchemaProvider();
|
||||
writeConfig = HoodieWriteConfig.newBuilder()
|
||||
.withPath("/tmp")
|
||||
.withSchema(schemaProvider.getSourceSchema().toString())
|
||||
.build();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleWriteAndFlush() throws Exception {
|
||||
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(new String[] {partitionPath});
|
||||
List<HoodieRecord> records = dataGen.generateInserts(COMMIT_TIME, NUM_RECORDS);
|
||||
|
||||
BufferedConnectWriter writer = new BufferedConnectWriter(
|
||||
javaEngineContext,
|
||||
mockHoodieJavaWriteClient,
|
||||
COMMIT_TIME,
|
||||
configs,
|
||||
writeConfig,
|
||||
null,
|
||||
schemaProvider);
|
||||
|
||||
for (int i = 0; i < NUM_RECORDS; i++) {
|
||||
writer.writeHudiRecord(records.get(i));
|
||||
}
|
||||
Mockito.verify(mockHoodieJavaWriteClient, times(0))
|
||||
.bulkInsertPreppedRecords(anyList(), eq(COMMIT_TIME), eq(Option.empty()));
|
||||
|
||||
writer.flushHudiRecords();
|
||||
final ArgumentCaptor<List<HoodieRecord>> actualRecords = ArgumentCaptor.forClass(List.class);
|
||||
Mockito.verify(mockHoodieJavaWriteClient, times(1))
|
||||
.bulkInsertPreppedRecords(actualRecords.capture(), eq(COMMIT_TIME), eq(Option.empty()));
|
||||
|
||||
actualRecords.getValue().sort(Comparator.comparing(HoodieRecord::getRecordKey));
|
||||
records.sort(Comparator.comparing(HoodieRecord::getRecordKey));
|
||||
|
||||
assertEquals(records, actualRecords.getValue());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,32 @@
|
||||
###
|
||||
# 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.
|
||||
###
|
||||
log4j.rootLogger=WARN, CONSOLE
|
||||
log4j.logger.org.apache=INFO
|
||||
log4j.logger.org.apache.hudi=DEBUG
|
||||
log4j.logger.org.apache.hadoop.hbase=ERROR
|
||||
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
|
||||
log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
|
||||
log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
|
||||
log4j.appender.CONSOLE.filter.a.LevelMin=WARN
|
||||
log4j.appender.CONSOLE.filter.a.LevelMax=FATAL
|
||||
|
||||
Reference in New Issue
Block a user