[HUDI-1931] BucketAssignFunction use ValueState instead of MapState (#3026)
Co-authored-by: 854194341@qq.com <loukey_7821>
This commit is contained in:
@@ -32,7 +32,6 @@ import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.util.BaseFileUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.index.HoodieIndexUtils;
|
||||
import org.apache.hudi.sink.utils.PayloadCreation;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
@@ -43,6 +42,8 @@ import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.api.common.state.CheckpointListener;
|
||||
import org.apache.flink.api.common.state.MapState;
|
||||
import org.apache.flink.api.common.state.MapStateDescriptor;
|
||||
import org.apache.flink.api.common.state.ValueState;
|
||||
import org.apache.flink.api.common.state.ValueStateDescriptor;
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.api.common.typeinfo.Types;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
@@ -81,7 +82,7 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(BucketAssignFunction.class);
|
||||
|
||||
private HoodieFlinkEngineContext context;
|
||||
private BucketAssignOperator.Context context;
|
||||
|
||||
/**
|
||||
* Index cache(speed-up) state for the underneath file based(BloomFilter) indices.
|
||||
@@ -94,7 +95,7 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
* <li>If it does not, use the {@link BucketAssigner} to generate a new bucket ID</li>
|
||||
* </ul>
|
||||
*/
|
||||
private MapState<String, HoodieRecordGlobalLocation> indexState;
|
||||
private ValueState<HoodieRecordGlobalLocation> indexState;
|
||||
|
||||
/**
|
||||
* Bucket assigner to assign new bucket IDs or reuse existing ones.
|
||||
@@ -138,7 +139,7 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
super.open(parameters);
|
||||
HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(this.conf);
|
||||
this.hadoopConf = StreamerUtil.getHadoopConf();
|
||||
this.context = new HoodieFlinkEngineContext(
|
||||
HoodieFlinkEngineContext context = new HoodieFlinkEngineContext(
|
||||
new SerializableConfiguration(this.hadoopConf),
|
||||
new FlinkTaskContextSupplier(getRuntimeContext()));
|
||||
this.bucketAssigner = BucketAssigners.create(
|
||||
@@ -158,16 +159,15 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
|
||||
@Override
|
||||
public void initializeState(FunctionInitializationContext context) {
|
||||
MapStateDescriptor<String, HoodieRecordGlobalLocation> indexStateDesc =
|
||||
new MapStateDescriptor<>(
|
||||
ValueStateDescriptor<HoodieRecordGlobalLocation> indexStateDesc =
|
||||
new ValueStateDescriptor<>(
|
||||
"indexState",
|
||||
Types.STRING,
|
||||
TypeInformation.of(HoodieRecordGlobalLocation.class));
|
||||
double ttl = conf.getDouble(FlinkOptions.INDEX_STATE_TTL) * 24 * 60 * 60 * 1000;
|
||||
if (ttl > 0) {
|
||||
indexStateDesc.enableTimeToLive(StateTtlConfigUtil.createTtlConfig((long) ttl));
|
||||
}
|
||||
indexState = context.getKeyedStateStore().getMapState(indexStateDesc);
|
||||
indexState = context.getKeyedStateStore().getState(indexStateDesc);
|
||||
if (bootstrapIndex) {
|
||||
MapStateDescriptor<String, Integer> partitionLoadStateDesc =
|
||||
new MapStateDescriptor<>("partitionLoadState", Types.STRING, Types.INT);
|
||||
@@ -191,13 +191,13 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
// disabled by default.
|
||||
if (bootstrapIndex && !partitionLoadState.contains(partitionPath)) {
|
||||
// If the partition records are never loaded, load the records first.
|
||||
loadRecords(partitionPath);
|
||||
loadRecords(partitionPath, recordKey);
|
||||
}
|
||||
// Only changing records need looking up the index for the location,
|
||||
// append only records are always recognized as INSERT.
|
||||
if (isChangingRecords && indexState.contains(recordKey)) {
|
||||
HoodieRecordGlobalLocation oldLoc = indexState.value();
|
||||
if (isChangingRecords && oldLoc != null) {
|
||||
// Set up the instant time as "U" to mark the bucket as an update bucket.
|
||||
HoodieRecordGlobalLocation oldLoc = this.indexState.get(recordKey);
|
||||
if (!Objects.equals(oldLoc.getPartitionPath(), partitionPath)) {
|
||||
if (globalIndex) {
|
||||
// if partition path changes, emit a delete record for old partition path,
|
||||
@@ -209,7 +209,7 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
out.collect((O) deleteRecord);
|
||||
}
|
||||
location = getNewRecordLocation(partitionPath);
|
||||
updateIndexState(recordKey, partitionPath, location);
|
||||
updateIndexState(partitionPath, location);
|
||||
} else {
|
||||
location = oldLoc.toLocal("U");
|
||||
this.bucketAssigner.addUpdate(partitionPath, location.getFileId());
|
||||
@@ -217,7 +217,7 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
} else {
|
||||
location = getNewRecordLocation(partitionPath);
|
||||
if (isChangingRecords) {
|
||||
updateIndexState(recordKey, partitionPath, location);
|
||||
updateIndexState(partitionPath, location);
|
||||
}
|
||||
}
|
||||
record.setCurrentLocation(location);
|
||||
@@ -244,10 +244,9 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
}
|
||||
|
||||
private void updateIndexState(
|
||||
String recordKey,
|
||||
String partitionPath,
|
||||
HoodieRecordLocation localLoc) throws Exception {
|
||||
this.indexState.put(recordKey, HoodieRecordGlobalLocation.fromLocal(partitionPath, localLoc));
|
||||
this.indexState.update(HoodieRecordGlobalLocation.fromLocal(partitionPath, localLoc));
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -261,13 +260,18 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
this.bucketAssigner.close();
|
||||
}
|
||||
|
||||
public void setContext(BucketAssignOperator.Context context) {
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all the indices of give partition path into the backup state.
|
||||
*
|
||||
* @param partitionPath The partition path
|
||||
* @param curRecordKey The current record key
|
||||
* @throws Exception when error occurs for state update
|
||||
*/
|
||||
private void loadRecords(String partitionPath) throws Exception {
|
||||
private void loadRecords(String partitionPath, String curRecordKey) throws Exception {
|
||||
LOG.info("Start loading records under partition {} into the index state", partitionPath);
|
||||
HoodieTable<?, ?, ?, ?> hoodieTable = bucketAssigner.getTable();
|
||||
BaseFileUtils fileUtils = BaseFileUtils.getInstance(hoodieTable.getBaseFileFormat());
|
||||
@@ -294,14 +298,20 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
boolean shouldLoad = KeyGroupRangeAssignment.assignKeyToParallelOperator(
|
||||
hoodieKey.getRecordKey(), maxParallelism, parallelism) == taskID;
|
||||
if (shouldLoad) {
|
||||
this.indexState.put(hoodieKey.getRecordKey(),
|
||||
new HoodieRecordGlobalLocation(hoodieKey.getPartitionPath(), baseFile.getCommitTime(), baseFile.getFileId()));
|
||||
this.context.setCurrentKey(hoodieKey.getRecordKey());
|
||||
this.indexState.update(
|
||||
new HoodieRecordGlobalLocation(
|
||||
hoodieKey.getPartitionPath(),
|
||||
baseFile.getCommitTime(),
|
||||
baseFile.getFileId()));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("Error when putting record keys into the state from file: {}", baseFile);
|
||||
}
|
||||
});
|
||||
}
|
||||
// recover the currentKey
|
||||
this.context.setCurrentKey(curRecordKey);
|
||||
// Mark the partition path as loaded.
|
||||
partitionLoadState.put(partitionPath, 0);
|
||||
LOG.info("Finish loading records under partition {} into the index state", partitionPath);
|
||||
@@ -309,15 +319,7 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
|
||||
@VisibleForTesting
|
||||
public void clearIndexState() {
|
||||
this.partitionLoadState.clear();
|
||||
this.indexState.clear();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public boolean isKeyInState(HoodieKey hoodieKey) {
|
||||
try {
|
||||
return this.indexState.contains(hoodieKey.getRecordKey());
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,57 @@
|
||||
/*
|
||||
* 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.sink.partitioner;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
|
||||
import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
|
||||
|
||||
/**
|
||||
* Operator for {@link BucketAssignFunction}.
|
||||
*
|
||||
* @param <I> The input type
|
||||
*/
|
||||
public class BucketAssignOperator<K, I extends HoodieRecord<?>, O extends HoodieRecord<?>>
|
||||
extends KeyedProcessOperator<K, I, O> {
|
||||
private final BucketAssignFunction<K, I, O> function;
|
||||
|
||||
public BucketAssignOperator(BucketAssignFunction<K, I, O> function) {
|
||||
super(function);
|
||||
this.function = function;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open() throws Exception {
|
||||
super.open();
|
||||
this.function.setContext(new ContextImpl());
|
||||
}
|
||||
|
||||
/**
|
||||
* Context to give the function chance to operate the state handle.
|
||||
*/
|
||||
public interface Context {
|
||||
void setCurrentKey(Object key);
|
||||
}
|
||||
|
||||
public class ContextImpl implements Context {
|
||||
public void setCurrentKey(Object key) {
|
||||
BucketAssignOperator.this.setCurrentKey(key);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -29,6 +29,7 @@ import org.apache.hudi.sink.compact.CompactionCommitSink;
|
||||
import org.apache.hudi.sink.compact.CompactionPlanEvent;
|
||||
import org.apache.hudi.sink.compact.CompactionPlanOperator;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignFunction;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignOperator;
|
||||
import org.apache.hudi.sink.transform.RowDataToHoodieFunction;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
@@ -80,7 +81,7 @@ public class HoodieTableSink implements DynamicTableSink, SupportsPartitioning,
|
||||
.transform(
|
||||
"bucket_assigner",
|
||||
TypeInformation.of(HoodieRecord.class),
|
||||
new KeyedProcessOperator<>(new BucketAssignFunction<>(conf)))
|
||||
new BucketAssignOperator<>(new BucketAssignFunction<>(conf)))
|
||||
.uid("uid_bucket_assigner")
|
||||
// shuffle by fileId(bucket id)
|
||||
.keyBy(record -> record.getCurrentLocation().getFileId())
|
||||
|
||||
Reference in New Issue
Block a user