/
HoodieWriteHelper.java
84 lines (75 loc) · 3.82 KB
/
HoodieWriteHelper.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
/*
* 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.table.action.commit;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.config.SerializableSchema;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.data.HoodieData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieOperation;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordMerger;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.HoodieTable;
import java.io.IOException;
public class HoodieWriteHelper<T, R> extends BaseWriteHelper<T, HoodieData<HoodieRecord<T>>,
HoodieData<HoodieKey>, HoodieData<WriteStatus>, R> {
private HoodieWriteHelper() {
super(HoodieData::getNumPartitions);
}
private static class WriteHelperHolder {
private static final HoodieWriteHelper HOODIE_WRITE_HELPER = new HoodieWriteHelper<>();
}
public static HoodieWriteHelper newInstance() {
return WriteHelperHolder.HOODIE_WRITE_HELPER;
}
@Override
protected HoodieData<HoodieRecord<T>> tag(HoodieData<HoodieRecord<T>> dedupedRecords, HoodieEngineContext context,
HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<HoodieKey>, HoodieData<WriteStatus>> table) {
return table.getIndex().tagLocation(dedupedRecords, context, table);
}
@Override
public HoodieData<HoodieRecord<T>> deduplicateRecords(
HoodieData<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, String schemaStr, TypedProperties props, HoodieRecordMerger merger) {
boolean isIndexingGlobal = index.isGlobal();
final SerializableSchema schema = new SerializableSchema(schemaStr);
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;
// NOTE: PLEASE READ CAREFULLY BEFORE CHANGING
// Here we have to make a copy of the incoming record, since it might be holding
// an instance of [[InternalRow]] pointing into shared, mutable buffer
return Pair.of(key, record.copy());
}).reduceByKey((rec1, rec2) -> {
HoodieRecord<T> reducedRecord;
try {
reducedRecord = merger.merge(rec1, schema.get(), rec2, schema.get(), props).get().getLeft();
} catch (IOException e) {
throw new HoodieException(String.format("Error to merge two records, %s, %s", rec1, rec2), e);
}
boolean choosePrev = rec1.getData().equals(reducedRecord.getData());
HoodieKey reducedKey = choosePrev ? rec1.getKey() : rec2.getKey();
HoodieOperation operation = choosePrev ? rec1.getOperation() : rec2.getOperation();
return reducedRecord.newInstance(reducedKey, operation);
}, parallelism).map(Pair::getRight);
}
}