1
0

[HUDI-2890] Kafka Connect: Fix failed writes and avoid table service concurrent operations (#4211)

* Fix kafka connect readme

* Fix handling of errors in write records for kafka connect

* By default, ensure we skip error records and keep the pipeline alive

* Fix indentation

Co-authored-by: Rajesh Mahindra <rmahindra@Rajeshs-MacBook-Pro.local>
This commit is contained in:
rmahindra123
2021-12-03 21:30:32 -08:00
committed by GitHub
parent 0fd6b2d71e
commit 94f45e928c
11 changed files with 270 additions and 95 deletions

View File

@@ -280,26 +280,56 @@ public class ConnectTransactionCoordinator implements TransactionCoordinator, Ru
private void onReceiveWriteStatus(ControlMessage message) {
ControlMessage.ParticipantInfo participantInfo = message.getParticipantInfo();
int partition = message.getSenderPartition();
partitionsWriteStatusReceived.put(partition, KafkaConnectUtils.getWriteStatuses(participantInfo));
currentConsumedKafkaOffsets.put(partition, participantInfo.getKafkaOffset());
int partitionId = message.getSenderPartition();
partitionsWriteStatusReceived.put(partitionId, KafkaConnectUtils.getWriteStatuses(participantInfo));
currentConsumedKafkaOffsets.put(partitionId, participantInfo.getKafkaOffset());
if (partitionsWriteStatusReceived.size() >= numPartitions
&& currentState.equals(State.ENDED_COMMIT)) {
// Commit the kafka offsets to the commit file
try {
List<WriteStatus> allWriteStatuses = new ArrayList<>();
partitionsWriteStatusReceived.forEach((key, value) -> allWriteStatuses.addAll(value));
// Commit the last write in Hudi, along with the latest kafka offset
if (!allWriteStatuses.isEmpty()) {
transactionServices.endCommit(currentCommitTime,
long totalErrorRecords = (long) allWriteStatuses.stream().mapToDouble(WriteStatus::getTotalErrorRecords).sum();
long totalRecords = (long) allWriteStatuses.stream().mapToDouble(WriteStatus::getTotalRecords).sum();
boolean hasErrors = totalErrorRecords > 0;
if ((!hasErrors || configs.allowCommitOnErrors()) && !allWriteStatuses.isEmpty()) {
boolean success = transactionServices.endCommit(currentCommitTime,
allWriteStatuses,
transformKafkaOffsets(currentConsumedKafkaOffsets));
if (success) {
LOG.info("Commit " + currentCommitTime + " successful!");
currentState = State.WRITE_STATUS_RCVD;
globalCommittedKafkaOffsets.putAll(currentConsumedKafkaOffsets);
submitEvent(new CoordinatorEvent(CoordinatorEvent.CoordinatorEventType.ACK_COMMIT,
message.getTopicName(),
currentCommitTime));
return;
} else {
LOG.error("Commit " + currentCommitTime + " failed!");
}
} else if (hasErrors) {
LOG.error("Coordinator found errors when writing. Errors/Total=" + totalErrorRecords + "/" + totalRecords);
LOG.error("Printing out the top 100 errors");
allWriteStatuses.stream().filter(WriteStatus::hasErrors).limit(100).forEach(ws -> {
LOG.error("Global error :", ws.getGlobalError());
if (ws.getErrors().size() > 0) {
ws.getErrors().forEach((key, value) -> LOG.trace("Error for key:" + key + " is " + value));
}
});
} else {
LOG.warn("Empty write statuses were received from all Participants");
}
currentState = State.WRITE_STATUS_RCVD;
globalCommittedKafkaOffsets.putAll(currentConsumedKafkaOffsets);
submitEvent(new CoordinatorEvent(CoordinatorEvent.CoordinatorEventType.ACK_COMMIT,
message.getTopicName(),
currentCommitTime));
// Submit the next start commit, that will rollback the current commit.
currentState = State.FAILED_COMMIT;
LOG.warn("Current commit " + currentCommitTime + " failed, so starting a new commit after recovery delay");
submitEvent(new CoordinatorEvent(CoordinatorEvent.CoordinatorEventType.START_COMMIT,
partition.topic(),
StringUtils.EMPTY_STRING),
RESTART_COMMIT_DELAY_MS, TimeUnit.MILLISECONDS);
} catch (Exception exception) {
LOG.error("Fatal error while committing file", exception);
}
@@ -310,7 +340,7 @@ public class ConnectTransactionCoordinator implements TransactionCoordinator, Ru
// If we are still stuck in ENDED_STATE
if (currentState.equals(State.ENDED_COMMIT)) {
currentState = State.WRITE_STATUS_TIMEDOUT;
LOG.warn("Did not receive the Write Status from all partitions");
LOG.warn("Current commit " + currentCommitTime + " failed after a write status timeout, so starting a new commit after recovery delay");
// Submit the next start commit
submitEvent(new CoordinatorEvent(CoordinatorEvent.CoordinatorEventType.START_COMMIT,
partition.topic(),
@@ -365,6 +395,7 @@ public class ConnectTransactionCoordinator implements TransactionCoordinator, Ru
INIT,
STARTED_COMMIT,
ENDED_COMMIT,
FAILED_COMMIT,
WRITE_STATUS_RCVD,
WRITE_STATUS_TIMEDOUT,
ACKED_COMMIT,

View File

@@ -32,7 +32,7 @@ public interface ConnectTransactionServices {
String startCommit();
void endCommit(String commitTime, List<WriteStatus> writeStatuses, Map<String, String> extraMetadata);
boolean endCommit(String commitTime, List<WriteStatus> writeStatuses, Map<String, String> extraMetadata);
Map<String, String> fetchLatestExtraCommitMetadata();
}

View File

@@ -88,6 +88,11 @@ public class KafkaConnectConfigs extends HoodieConfig {
.defaultValue(HiveSyncTool.class.getName())
.withDocumentation("Meta sync client tool, using comma to separate multi tools");
public static final ConfigProperty<Boolean> ALLOW_COMMIT_ON_ERRORS = ConfigProperty
.key("hoodie.kafka.allow.commit.on.errors")
.defaultValue(true)
.withDocumentation("Commit even when some records failed to be written");
protected KafkaConnectConfigs() {
super();
}
@@ -136,6 +141,10 @@ public class KafkaConnectConfigs extends HoodieConfig {
return getString(META_SYNC_CLASSES);
}
public Boolean allowCommitOnErrors() {
return getBoolean(ALLOW_COMMIT_ON_ERRORS);
}
public static class Builder {
protected final KafkaConnectConfigs connectConfigs = new KafkaConnectConfigs();
@@ -160,6 +169,11 @@ public class KafkaConnectConfigs extends HoodieConfig {
return this;
}
public Builder withAllowCommitOnErrors(Boolean allowCommitOnErrors) {
connectConfigs.setValue(ALLOW_COMMIT_ON_ERRORS, String.valueOf(allowCommitOnErrors));
return this;
}
// Kafka connect task are passed with props with type Map<>
public Builder withProperties(Map<?, ?> properties) {
connectConfigs.getProps().putAll(properties);

View File

@@ -77,6 +77,7 @@ public class KafkaConnectTransactionServices implements ConnectTransactionServic
public KafkaConnectTransactionServices(KafkaConnectConfigs connectConfigs) throws HoodieException {
this.connectConfigs = connectConfigs;
// This is the writeConfig for the Transaction Coordinator
this.writeConfig = HoodieWriteConfig.newBuilder()
.withEngineType(EngineType.JAVA)
.withProperties(connectConfigs.getProps())
@@ -122,20 +123,23 @@ public class KafkaConnectTransactionServices implements ConnectTransactionServic
}
@Override
public void endCommit(String commitTime, List<WriteStatus> writeStatuses, Map<String, String> extraMetadata) {
javaClient.commit(commitTime, writeStatuses, Option.of(extraMetadata));
LOG.info("Ending Hudi commit " + commitTime);
public boolean endCommit(String commitTime, List<WriteStatus> writeStatuses, Map<String, String> extraMetadata) {
boolean success = javaClient.commit(commitTime, writeStatuses, Option.of(extraMetadata));
if (success) {
LOG.info("Ending Hudi commit " + commitTime);
// Schedule clustering and compaction as needed.
if (writeConfig.isAsyncClusteringEnabled()) {
javaClient.scheduleClustering(Option.empty()).ifPresent(
instantTs -> LOG.info("Scheduled clustering at instant time:" + instantTs));
// Schedule clustering and compaction as needed.
if (writeConfig.isAsyncClusteringEnabled()) {
javaClient.scheduleClustering(Option.empty()).ifPresent(
instantTs -> LOG.info("Scheduled clustering at instant time:" + instantTs));
}
if (isAsyncCompactionEnabled()) {
javaClient.scheduleCompaction(Option.empty()).ifPresent(
instantTs -> LOG.info("Scheduled compaction at instant time:" + instantTs));
}
syncMeta();
}
if (isAsyncCompactionEnabled()) {
javaClient.scheduleCompaction(Option.empty()).ifPresent(
instantTs -> LOG.info("Scheduled compaction at instant time:" + instantTs));
}
syncMeta();
return success;
}
@Override

View File

@@ -27,6 +27,8 @@ import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.config.HoodieClusteringConfig;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.connect.KafkaConnectFileIdPrefixProvider;
@@ -73,7 +75,7 @@ public class KafkaConnectWriterProvider implements ConnectWriterProvider<WriteSt
this.keyGenerator = HoodieAvroKeyGeneratorFactory.createKeyGenerator(
new TypedProperties(connectConfigs.getProps()));
// Create the write client to write some records in
// This is the writeConfig for the writers for the individual Transaction Coordinators
writeConfig = HoodieWriteConfig.newBuilder()
.withEngineType(EngineType.JAVA)
.withProperties(connectConfigs.getProps())
@@ -84,6 +86,14 @@ public class KafkaConnectWriterProvider implements ConnectWriterProvider<WriteSt
.withSchema(schemaProvider.getSourceSchema().toString())
.withAutoCommit(false)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build())
// participants should not trigger table services, and leave it to the coordinator
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withAutoClean(false)
.withAutoArchive(false)
.withInlineCompaction(false).build())
.withClusteringConfig(HoodieClusteringConfig.newBuilder()
.withInlineClustering(false)
.build())
.build();
context = new HoodieJavaEngineContext(hadoopConf);