[HUDI-2332] Add clustering and compaction in Kafka Connect Sink (#3857)
* [HUDI-2332] Add clustering and compaction in Kafka Connect Sink * Disable validation check on instant time for compaction and adjust configs * Add javadocs * Add clustering and compaction config * Fix transaction causing missing records in the target table * Add debugging logs * Fix kafka offset sync in participant * Adjust how clustering and compaction are configured in kafka-connect * Fix clustering strategy * Remove irrelevant changes from other published PRs * Update clustering logic and others * Update README * Fix test failures * Fix indentation * Fix clustering config * Add JavaCustomColumnsSortPartitioner and make async compaction enabled by default * Add test for JavaCustomColumnsSortPartitioner * Add more changes after IDE sync * Update README with clarification * Fix clustering logic after rebasing * Remove unrelated changes
This commit is contained in:
@@ -18,8 +18,10 @@
|
||||
|
||||
package org.apache.hudi.avro;
|
||||
|
||||
import org.apache.hudi.common.config.SerializableSchema;
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
@@ -229,10 +231,10 @@ public class HoodieAvroUtils {
|
||||
|
||||
public static Schema removeFields(Schema schema, List<String> fieldsToRemove) {
|
||||
List<Schema.Field> filteredFields = schema.getFields()
|
||||
.stream()
|
||||
.filter(field -> !fieldsToRemove.contains(field.name()))
|
||||
.map(field -> new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultVal()))
|
||||
.collect(Collectors.toList());
|
||||
.stream()
|
||||
.filter(field -> !fieldsToRemove.contains(field.name()))
|
||||
.map(field -> new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultVal()))
|
||||
.collect(Collectors.toList());
|
||||
Schema filteredSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), false);
|
||||
filteredSchema.setFields(filteredFields);
|
||||
return filteredSchema;
|
||||
@@ -289,7 +291,7 @@ public class HoodieAvroUtils {
|
||||
}
|
||||
|
||||
public static GenericRecord addHoodieKeyToRecord(GenericRecord record, String recordKey, String partitionPath,
|
||||
String fileName) {
|
||||
String fileName) {
|
||||
record.put(HoodieRecord.FILENAME_METADATA_FIELD, fileName);
|
||||
record.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, partitionPath);
|
||||
record.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recordKey);
|
||||
@@ -551,7 +553,7 @@ public class HoodieAvroUtils {
|
||||
} else if (fieldSchema.getType() == Schema.Type.BYTES) {
|
||||
ByteBuffer byteBuffer = (ByteBuffer) fieldValue;
|
||||
BigDecimal convertedValue = decimalConversion.fromBytes(byteBuffer, fieldSchema,
|
||||
LogicalTypes.decimal(dc.getPrecision(), dc.getScale()));
|
||||
LogicalTypes.decimal(dc.getPrecision(), dc.getScale()));
|
||||
byteBuffer.rewind();
|
||||
return convertedValue;
|
||||
}
|
||||
@@ -570,9 +572,51 @@ public class HoodieAvroUtils {
|
||||
* @return sanitized name
|
||||
*/
|
||||
public static String sanitizeName(String name) {
|
||||
if (name.substring(0,1).matches(INVALID_AVRO_FIRST_CHAR_IN_NAMES)) {
|
||||
if (name.substring(0, 1).matches(INVALID_AVRO_FIRST_CHAR_IN_NAMES)) {
|
||||
name = name.replaceFirst(INVALID_AVRO_FIRST_CHAR_IN_NAMES, MASK_FOR_INVALID_CHARS_IN_NAMES);
|
||||
}
|
||||
return name.replaceAll(INVALID_AVRO_CHARS_IN_NAMES, MASK_FOR_INVALID_CHARS_IN_NAMES);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets record column values into one object.
|
||||
*
|
||||
* @param record Hoodie record.
|
||||
* @param columns Names of the columns to get values.
|
||||
* @param schema {@link Schema} instance.
|
||||
* @return Column value if a single column, or concatenated String values by comma.
|
||||
*/
|
||||
public static Object getRecordColumnValues(HoodieRecord<? extends HoodieRecordPayload> record,
|
||||
String[] columns,
|
||||
Schema schema) {
|
||||
try {
|
||||
GenericRecord genericRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
|
||||
if (columns.length == 1) {
|
||||
return HoodieAvroUtils.getNestedFieldVal(genericRecord, columns[0], true);
|
||||
} else {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
for (String col : columns) {
|
||||
sb.append(HoodieAvroUtils.getNestedFieldValAsString(genericRecord, col, true));
|
||||
}
|
||||
|
||||
return sb.toString();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Unable to read record with key:" + record.getKey(), e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets record column values into one object.
|
||||
*
|
||||
* @param record Hoodie record.
|
||||
* @param columns Names of the columns to get values.
|
||||
* @param schema {@link SerializableSchema} instance.
|
||||
* @return Column value if a single column, or concatenated String values by comma.
|
||||
*/
|
||||
public static Object getRecordColumnValues(HoodieRecord<? extends HoodieRecordPayload> record,
|
||||
String[] columns,
|
||||
SerializableSchema schema) {
|
||||
return getRecordColumnValues(record, columns, schema.get());
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user