1
0

[HUDI-2330][HUDI-2335] Adding support for merge-on-read tables (#3679)

- Inserts go into logs, hashed by Kafka and Hudi partitions
 - Fixed issues with the setupKafka script
 - Bumped up the default commit interval to 300 seconds
 - Minor renaming
This commit is contained in:
vinoth chandar
2021-09-16 15:24:34 -07:00
committed by GitHub
parent b8dad628e5
commit 57d5da68aa
16 changed files with 315 additions and 124 deletions

View File

@@ -18,17 +18,13 @@
package org.apache.hudi.connect;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.connect.utils.KafkaConnectUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.table.FileIdPrefixProvider;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.nio.charset.StandardCharsets;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import java.util.Objects;
import java.util.Properties;
public class KafkaConnectFileIdPrefixProvider extends FileIdPrefixProvider {
@@ -52,18 +48,9 @@ public class KafkaConnectFileIdPrefixProvider extends FileIdPrefixProvider {
// We use a combination of kafka partition and partition path as the file id, and then hash it
// to generate a fixed sized hash.
String rawFileIdPrefix = kafkaPartition + partitionPath;
MessageDigest md;
try {
md = MessageDigest.getInstance("MD5");
} catch (NoSuchAlgorithmException e) {
LOG.error("Fatal error selecting hash algorithm", e);
throw new HoodieException(e);
}
byte[] digest = Objects.requireNonNull(md).digest(rawFileIdPrefix.getBytes(StandardCharsets.UTF_8));
String hashedPrefix = KafkaConnectUtils.hashDigest(rawFileIdPrefix);
LOG.info("CreateFileId for Kafka Partition " + kafkaPartition + " : " + partitionPath + " = " + rawFileIdPrefix
+ " === " + StringUtils.toHexString(digest).toUpperCase());
return StringUtils.toHexString(digest).toUpperCase();
+ " === " + hashedPrefix);
return hashedPrefix;
}
}

View File

@@ -26,6 +26,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
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.common.util.StringUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.keygen.BaseKeyGenerator;
import org.apache.hudi.keygen.CustomAvroKeyGenerator;
@@ -41,8 +42,12 @@ import org.apache.kafka.common.KafkaFuture;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.nio.charset.StandardCharsets;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import java.util.Arrays;
import java.util.Map;
import java.util.Objects;
import java.util.Properties;
import java.util.stream.Collectors;
@@ -137,4 +142,16 @@ public class KafkaConnectUtils {
return Option.empty();
}
}
public static String hashDigest(String stringToHash) {
MessageDigest md;
try {
md = MessageDigest.getInstance("MD5");
} catch (NoSuchAlgorithmException e) {
LOG.error("Fatal error selecting hash algorithm", e);
throw new HoodieException(e);
}
byte[] digest = Objects.requireNonNull(md).digest(stringToHash.getBytes(StandardCharsets.UTF_8));
return StringUtils.toHexString(digest).toUpperCase();
}
}

View File

@@ -21,7 +21,9 @@ package org.apache.hudi.connect.writers;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.connect.utils.KafkaConnectUtils;
import org.apache.hudi.keygen.KeyGenerator;
import org.apache.hudi.schema.SchemaProvider;
import org.apache.hudi.utilities.sources.helpers.AvroConvertor;
@@ -46,17 +48,19 @@ public abstract class AbstractConnectWriter implements ConnectWriter<WriteStatus
public static final String KAFKA_JSON_CONVERTER = "org.apache.kafka.connect.json.JsonConverter";
public static final String KAFKA_STRING_CONVERTER = "org.apache.kafka.connect.storage.StringConverter";
private static final Logger LOG = LogManager.getLogger(AbstractConnectWriter.class);
protected final String instantTime;
private final KafkaConnectConfigs connectConfigs;
private final KeyGenerator keyGenerator;
private final SchemaProvider schemaProvider;
protected final KafkaConnectConfigs connectConfigs;
public AbstractConnectWriter(KafkaConnectConfigs connectConfigs,
KeyGenerator keyGenerator,
SchemaProvider schemaProvider) {
SchemaProvider schemaProvider, String instantTime) {
this.connectConfigs = connectConfigs;
this.keyGenerator = keyGenerator;
this.schemaProvider = schemaProvider;
this.instantTime = instantTime;
}
@Override
@@ -76,16 +80,22 @@ public abstract class AbstractConnectWriter implements ConnectWriter<WriteStatus
throw new IOException("Unsupported Kafka Format type (" + connectConfigs.getKafkaValueConverter() + ")");
}
HoodieRecord hoodieRecord = new HoodieRecord<>(keyGenerator.getKey(avroRecord.get()), new HoodieAvroPayload(avroRecord));
// Tag records with a file ID based on kafka partition and hudi partition.
HoodieRecord<?> hoodieRecord = new HoodieRecord<>(keyGenerator.getKey(avroRecord.get()), new HoodieAvroPayload(avroRecord));
String fileId = KafkaConnectUtils.hashDigest(String.format("%s-%s", record.kafkaPartition(), hoodieRecord.getPartitionPath()));
hoodieRecord.unseal();
hoodieRecord.setCurrentLocation(new HoodieRecordLocation(instantTime, fileId));
hoodieRecord.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
hoodieRecord.seal();
writeHudiRecord(hoodieRecord);
}
@Override
public List<WriteStatus> close() throws IOException {
return flushHudiRecords();
return flushRecords();
}
protected abstract void writeHudiRecord(HoodieRecord<HoodieAvroPayload> record);
protected abstract void writeHudiRecord(HoodieRecord<?> record);
protected abstract List<WriteStatus> flushHudiRecords() throws IOException;
protected abstract List<WriteStatus> flushRecords() throws IOException;
}

View File

@@ -21,8 +21,9 @@ package org.apache.hudi.connect.writers;
import org.apache.hudi.client.HoodieJavaWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.util.DefaultSizeEstimator;
import org.apache.hudi.common.util.HoodieRecordSizeEstimator;
import org.apache.hudi.common.util.Option;
@@ -39,8 +40,8 @@ import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
import java.util.stream.Collectors;
/**
* Specific implementation of a Hudi Writer that buffers all incoming records,
@@ -52,9 +53,8 @@ public class BufferedConnectWriter extends AbstractConnectWriter {
private final HoodieEngineContext context;
private final HoodieJavaWriteClient writeClient;
private final String instantTime;
private final HoodieWriteConfig config;
private ExternalSpillableMap<String, HoodieRecord<HoodieAvroPayload>> bufferedRecords;
private ExternalSpillableMap<String, HoodieRecord<?>> bufferedRecords;
public BufferedConnectWriter(HoodieEngineContext context,
HoodieJavaWriteClient writeClient,
@@ -63,10 +63,9 @@ public class BufferedConnectWriter extends AbstractConnectWriter {
HoodieWriteConfig config,
KeyGenerator keyGenerator,
SchemaProvider schemaProvider) {
super(connectConfigs, keyGenerator, schemaProvider);
super(connectConfigs, keyGenerator, schemaProvider, instantTime);
this.context = context;
this.writeClient = writeClient;
this.instantTime = instantTime;
this.config = config;
init();
}
@@ -88,12 +87,12 @@ public class BufferedConnectWriter extends AbstractConnectWriter {
}
@Override
public void writeHudiRecord(HoodieRecord<HoodieAvroPayload> record) {
public void writeHudiRecord(HoodieRecord<?> record) {
bufferedRecords.put(record.getRecordKey(), record);
}
@Override
public List<WriteStatus> flushHudiRecords() throws IOException {
public List<WriteStatus> flushRecords() throws IOException {
try {
LOG.info("Number of entries in MemoryBasedMap => "
+ bufferedRecords.getInMemoryMapNumEntries()
@@ -102,15 +101,25 @@ public class BufferedConnectWriter extends AbstractConnectWriter {
+ bufferedRecords.getDiskBasedMapNumEntries() + "Size of file spilled to disk => "
+ bufferedRecords.getSizeOfFileOnDiskInBytes());
List<WriteStatus> writeStatuses = new ArrayList<>();
boolean isMorTable = Option.ofNullable(connectConfigs.getString(HoodieTableConfig.TYPE))
.map(t -> t.equals(HoodieTableType.MERGE_ON_READ.name()))
.orElse(false);
// Write out all records if non-empty
if (!bufferedRecords.isEmpty()) {
writeStatuses = writeClient.bulkInsertPreppedRecords(
bufferedRecords.values().stream().collect(Collectors.toList()),
instantTime, Option.empty());
if (isMorTable) {
writeStatuses = writeClient.upsertPreppedRecords(
new LinkedList<>(bufferedRecords.values()),
instantTime);
} else {
writeStatuses = writeClient.bulkInsertPreppedRecords(
new LinkedList<>(bufferedRecords.values()),
instantTime, Option.empty());
}
}
bufferedRecords.close();
LOG.info("Flushed hudi records and got writeStatuses: "
+ writeStatuses);
LOG.info("Flushed hudi records and got writeStatuses: " + writeStatuses);
return writeStatuses;
} catch (Exception e) {
throw new IOException("Write records failed", e);

View File

@@ -67,7 +67,7 @@ public class KafkaConnectConfigs extends HoodieConfig {
public static final ConfigProperty<String> COORDINATOR_WRITE_TIMEOUT_SECS = ConfigProperty
.key("hoodie.kafka.coordinator.write.timeout.secs")
.defaultValue("60")
.defaultValue("300")
.withDocumentation("The timeout after sending an END_COMMIT until when "
+ "the coordinator will wait for the write statuses from all the partitions"
+ "to ignore the current commit and start a new commit.");

View File

@@ -23,12 +23,10 @@ import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieJavaEngineContext;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.connect.transaction.TransactionCoordinator;
@@ -38,7 +36,6 @@ import org.apache.hudi.keygen.KeyGenerator;
import org.apache.hudi.keygen.factory.HoodieAvroKeyGeneratorFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -54,19 +51,16 @@ import java.util.Map;
public class KafkaConnectTransactionServices implements ConnectTransactionServices {
private static final Logger LOG = LogManager.getLogger(KafkaConnectTransactionServices.class);
private static final String TABLE_FORMAT = "PARQUET";
private final Option<HoodieTableMetaClient> tableMetaClient;
private final Configuration hadoopConf;
private final FileSystem fs;
private final String tableBasePath;
private final String tableName;
private final HoodieEngineContext context;
private final HoodieJavaWriteClient<HoodieAvroPayload> javaClient;
public KafkaConnectTransactionServices(
KafkaConnectConfigs connectConfigs) throws HoodieException {
public KafkaConnectTransactionServices(KafkaConnectConfigs connectConfigs) throws HoodieException {
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder()
.withProperties(connectConfigs.getProps()).build();
@@ -74,29 +68,25 @@ public class KafkaConnectTransactionServices implements ConnectTransactionServic
tableName = writeConfig.getTableName();
hadoopConf = KafkaConnectUtils.getDefaultHadoopConf();
context = new HoodieJavaEngineContext(hadoopConf);
fs = FSUtils.getFs(tableBasePath, hadoopConf);
try {
KeyGenerator keyGenerator = HoodieAvroKeyGeneratorFactory.createKeyGenerator(
new TypedProperties(connectConfigs.getProps()));
String recordKeyFields = KafkaConnectUtils.getRecordKeyColumns(keyGenerator);
String partitionColumns = KafkaConnectUtils.getPartitionColumns(keyGenerator,
new TypedProperties(connectConfigs.getProps()));
LOG.info(String.format("Setting record key %s and partitionfields %s for table %s",
recordKeyFields,
partitionColumns,
tableBasePath + tableName));
LOG.info(String.format("Setting record key %s and partition fields %s for table %s",
recordKeyFields, partitionColumns, tableBasePath + tableName));
tableMetaClient = Option.of(HoodieTableMetaClient.withPropertyBuilder()
.setTableType(HoodieTableType.COPY_ON_WRITE.name())
.setTableName(tableName)
.setPayloadClassName(HoodieAvroPayload.class.getName())
.setBaseFileFormat(TABLE_FORMAT)
.setRecordKeyFields(recordKeyFields)
.setPartitionFields(partitionColumns)
.setKeyGeneratorClassProp(writeConfig.getKeyGeneratorClass())
.fromProperties(connectConfigs.getProps())
.initTable(hadoopConf, tableBasePath));
javaClient = new HoodieJavaWriteClient<>(context, writeConfig);
@@ -113,8 +103,7 @@ public class KafkaConnectTransactionServices implements ConnectTransactionServic
}
public void endCommit(String commitTime, List<WriteStatus> writeStatuses, Map<String, String> extraMetadata) {
javaClient.commit(commitTime, writeStatuses, Option.of(extraMetadata),
HoodieActiveTimeline.COMMIT_ACTION, Collections.emptyMap());
javaClient.commit(commitTime, writeStatuses, Option.of(extraMetadata));
LOG.info("Ending Hudi commit " + commitTime);
}