Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.HoodieTable;
Expand All @@ -43,7 +44,7 @@ public HoodieWriteMetadata<O> write(String instantTime,
try {
// De-dupe/merge if needed
I dedupedRecords =
combineOnCondition(shouldCombine, inputRecords, shuffleParallelism, table, executor.config.getSchema());
combineOnCondition(shouldCombine, inputRecords, shuffleParallelism, table, executor.config);

Instant lookupBegin = Instant.now();
I taggedRecords = dedupedRecords;
Expand All @@ -69,8 +70,8 @@ protected abstract I tag(
I dedupedRecords, HoodieEngineContext context, HoodieTable<T, I, K, O> table);

public I combineOnCondition(
boolean condition, I records, int parallelism, HoodieTable<T, I, K, O> table, String schemaString) {
return condition ? deduplicateRecords(records, table, parallelism, schemaString) : records;
boolean condition, I records, int parallelism, HoodieTable<T, I, K, O> table, HoodieWriteConfig writeConfig) {
return condition ? deduplicateRecords(records, table, parallelism, writeConfig) : records;
}

/**
Expand All @@ -81,10 +82,10 @@ public I combineOnCondition(
* @return Collection of HoodieRecord already be deduplicated
*/
public I deduplicateRecords(
I records, HoodieTable<T, I, K, O> table, int parallelism, String schemaString) {
return deduplicateRecords(records, table.getIndex(), parallelism, schemaString);
I records, HoodieTable<T, I, K, O> table, int parallelism, HoodieWriteConfig writeConfig) {
return deduplicateRecords(records, table.getIndex(), parallelism, writeConfig);
}

public abstract I deduplicateRecords(
I records, HoodieIndex<?, ?> index, int parallelism, String schemaString);
I records, HoodieIndex<?, ?> index, int parallelism, HoodieWriteConfig writeConfig);
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,10 +26,10 @@
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.HoodieTable;

import java.util.Properties;

public class HoodieWriteHelper<T extends HoodieRecordPayload, R> extends BaseWriteHelper<T, HoodieData<HoodieRecord<T>>,
HoodieData<HoodieKey>, HoodieData<WriteStatus>, R> {
Expand All @@ -53,18 +53,16 @@ protected HoodieData<HoodieRecord<T>> tag(HoodieData<HoodieRecord<T>> dedupedRec

@Override
public HoodieData<HoodieRecord<T>> deduplicateRecords(
HoodieData<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, String schemaString) {
HoodieData<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, HoodieWriteConfig writeConfig) {
boolean isIndexingGlobal = index.isGlobal();
Properties properties = new Properties();
properties.put("schema", schemaString);
return records.mapToPair(record -> {
HoodieKey hoodieKey = record.getKey();
// If index used is global, then records are expected to differ in their partitionPath
Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey;
return Pair.of(key, record);
}).reduceByKey((rec1, rec2) -> {
@SuppressWarnings("unchecked")
T reducedData = (T) rec2.getData().preCombine(rec1.getData(), properties);
T reducedData = (T) rec2.getData().preCombine(rec1.getData(), writeConfig.getProps());
HoodieKey reducedKey = rec1.getData().equals(reducedData) ? rec1.getKey() : rec2.getKey();

return new HoodieAvroRecord<>(reducedKey, reducedData);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.HoodieTable;
Expand All @@ -37,7 +38,6 @@
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Properties;
import java.util.stream.Collectors;

/**
Expand Down Expand Up @@ -89,7 +89,7 @@ protected List<HoodieRecord<T>> tag(List<HoodieRecord<T>> dedupedRecords, Hoodie

@Override
public List<HoodieRecord<T>> deduplicateRecords(
List<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, String schemaString) {
List<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, HoodieWriteConfig writeConfig) {
// If index used is global, then records are expected to differ in their partitionPath
Map<Object, List<HoodieRecord<T>>> keyedRecords = records.stream()
.collect(Collectors.groupingBy(record -> record.getKey().getRecordKey()));
Expand All @@ -98,9 +98,7 @@ public List<HoodieRecord<T>> deduplicateRecords(
final T data1 = rec1.getData();
final T data2 = rec2.getData();

Properties properties = new Properties();
properties.put("schema", schemaString);
@SuppressWarnings("unchecked") final T reducedData = (T) data2.preCombine(data1, properties);
@SuppressWarnings("unchecked") final T reducedData = (T) data2.preCombine(data1, writeConfig.getProps());
// we cannot allow the user to change the key or partitionPath, since that will affect
// everything
// so pick it from one of the records.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieOperation;
import org.apache.hudi.common.model.PartialUpdateAvroPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
Expand All @@ -48,6 +50,8 @@ public class FlinkWriteHelperTest {

private String preCombineFields = "";

private HoodieWriteConfig writeConfig;

public static final String SCHEMA = "{\n"
+ " \"type\": \"record\",\n"
+ " \"name\": \"partialRecord\", \"namespace\":\"org.apache.hudi\",\n"
Expand All @@ -67,12 +71,18 @@ public class FlinkWriteHelperTest {
public void setUp() throws Exception {
this.preCombineFields = "_ts1:fa;_ts2:fb";
this.avroSchema = new Schema.Parser().parse(SCHEMA);
this.writeConfig = HoodieWriteConfig
.newBuilder()
.withPath(tempFile.getAbsolutePath())
.withSchema(SCHEMA)
.withPreCombineField(preCombineFields)
.build();
}

@Test
void deduplicateRecords() throws IOException, InterruptedException {
List<HoodieAvroRecord> records = data();
records = FlinkWriteHelper.newInstance().deduplicateRecords(records, (HoodieIndex) null, -1, this.avroSchema.toString());
records = FlinkWriteHelper.newInstance().deduplicateRecords(records, (HoodieIndex) null, -1, this.writeConfig);
GenericRecord record = HoodieAvroUtils.bytesToAvro(((PartialUpdateAvroPayload) records.get(0).getData()).recordBytes, this.avroSchema);
System.out.println("======================================================================================");
System.out.println("last: " + record);
Expand All @@ -94,15 +104,13 @@ public List<HoodieAvroRecord> data() throws InterruptedException {
row2.put("_ts2", ts);
records.add(row1);
records.add(row2);
Thread.sleep(1);
//Thread.sleep(1);
}

return records.stream().map(genericRowData -> {
try {
String orderingFieldValText = HoodieAvroUtils.getMultipleNestedFieldVals(genericRowData,
preCombineFields, false).toString();
return new HoodieAvroRecord(new HoodieKey("1", "default"),
new PartialUpdateAvroPayload(genericRowData, orderingFieldValText), HoodieOperation.INSERT);
new PartialUpdateAvroPayload(Option.of(genericRowData)), HoodieOperation.INSERT);
} catch (Exception e) {
throw new RuntimeException(e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ public List<WriteStatus> bulkInsert(List<HoodieRecord<T>> inputRecords,

if (performDedupe) {
dedupedRecords = (List<HoodieRecord<T>>) JavaWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords,
parallelism, table, config.getSchema());
parallelism, table, config);
}

final List<HoodieRecord<T>> repartitionedRecords = (List<HoodieRecord<T>>) partitioner.repartitionRecords(dedupedRecords, parallelism);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ public HoodieData<WriteStatus> bulkInsert(HoodieData<HoodieRecord<T>> inputRecor

if (performDedupe) {
dedupedRecords = (HoodieData<HoodieRecord<T>>) HoodieWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords,
parallelism, table, config.getSchema());
parallelism, table, config);
}

// only JavaRDD is supported for Spark partitioner, but it is not enforced by BulkInsertPartitioner API. To improve this, TODO HUDI-3463
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
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.model.MultipleOrderingVal2ColsInfo;
import org.apache.hudi.common.model.MultiplePartialUpdateUnit;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.collection.Pair;
Expand Down Expand Up @@ -498,14 +498,14 @@ public static Object getNestedFieldValAsString(GenericRecord record, String fiel
}

public static Object getMultipleNestedFieldVals(GenericRecord record, String fieldMappings, boolean consistentLogicalTimestampEnabled) {
MultipleOrderingVal2ColsInfo multipleOrderingVal2ColsInfo = new MultipleOrderingVal2ColsInfo(fieldMappings);
multipleOrderingVal2ColsInfo.getOrderingVal2ColsInfoList().forEach(orderingVal2ColsInfo -> {
MultiplePartialUpdateUnit multipleOrderingVal2ColsInfo = new MultiplePartialUpdateUnit(fieldMappings);
multipleOrderingVal2ColsInfo.getMultiplePartialUpdateUnits().forEach(orderingVal2ColsInfo -> {
Object val = getNestedFieldVal(record, orderingVal2ColsInfo.getOrderingField(), true, consistentLogicalTimestampEnabled);
if (Objects.nonNull(val)) {
orderingVal2ColsInfo.setOrderingValue(val.toString());
}
});
return multipleOrderingVal2ColsInfo.generateOrderingText();
return multipleOrderingVal2ColsInfo.toString();
}

/**
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
/*
* 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.common.model;

import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;

/**
* MultipleOrderingVal2ColsInfo
* _ts1=999:name1,price1;_ts2=111:name2,price2
* _ts1:name1,price1=999;_ts2:name2,price2=111
*/
public class MultiplePartialUpdateUnit {
private List<PartialUpdateUnit> multiplePartialUpdateUnits = Collections.EMPTY_LIST;

public MultiplePartialUpdateUnit(String multipleUpdateUnitText) {
this.multiplePartialUpdateUnits =
Arrays.stream(multipleUpdateUnitText.split(";")).filter(Objects::nonNull)
.map(PartialUpdateUnit::new).collect(Collectors.toList());
}

public List<PartialUpdateUnit> getMultiplePartialUpdateUnits() {
return multiplePartialUpdateUnits;
}

public class PartialUpdateUnit {
private String orderingField;
private String orderingValue = "";
private List<String> columnNames;

public PartialUpdateUnit(String partialUpdateUnitText) {
List<String> partialUpdateList = Arrays.asList(partialUpdateUnitText.split(":|,"));
this.orderingField = partialUpdateList.get(0);
this.columnNames = partialUpdateList.subList(1, partialUpdateList.size());
}

public String getOrderingField() {
return orderingField;
}

public String getOrderingValue() {
return orderingValue;
}

public void setOrderingValue(String value) {
this.orderingValue = value;
}

public List<String> getColumnNames() {
return columnNames;
}

@Override
public String toString() {
return String.format("%s=%s:%s", this.orderingField, this.orderingValue, String.join(",", this.columnNames));
}
}

@Override
public String toString() {
int len = multiplePartialUpdateUnits.size();
return len > 1 ? this.multiplePartialUpdateUnits
.stream()
.map(PartialUpdateUnit::toString)
.collect(Collectors.joining(";")) : multiplePartialUpdateUnits.get(0).toString();
}
}
Loading