1
0

[HUDI-2738] Remove the bucketAssignFunction useless context (#3972)

Co-authored-by: yuzhaojing <yuzhaojing@bytedance.com>
This commit is contained in:
yuzhaojing
2021-11-11 21:03:01 +08:00
committed by GitHub
parent 90529aa552
commit 6b93ccca9b
4 changed files with 8 additions and 91 deletions

View File

@@ -35,7 +35,6 @@ import org.apache.hudi.sink.utils.PayloadCreation;
import org.apache.hudi.table.action.commit.BucketInfo;
import org.apache.hudi.util.StreamerUtil;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.state.CheckpointListener;
import org.apache.flink.api.common.state.StateTtlConfig;
import org.apache.flink.api.common.state.ValueState;
@@ -70,8 +69,6 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
extends KeyedProcessFunction<K, I, O>
implements CheckpointedFunction, CheckpointListener {
private BucketAssignOperator.Context context;
/**
* Index cache(speed-up) state for the underneath file based(BloomFilter) indices.
* When a record came in, we do these check:
@@ -158,7 +155,6 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
public void processElement(I value, Context ctx, Collector<O> out) throws Exception {
if (value instanceof IndexRecord) {
IndexRecord<?> indexRecord = (IndexRecord<?>) value;
this.context.setCurrentKey(indexRecord.getRecordKey());
this.indexState.update((HoodieRecordGlobalLocation) indexRecord.getCurrentLocation());
} else {
processRecord((HoodieRecord<?>) value, out);
@@ -198,7 +194,6 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
}
} else {
location = getNewRecordLocation(partitionPath);
this.context.setCurrentKey(recordKey);
}
// always refresh the index
if (isChangingRecords) {
@@ -243,13 +238,4 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
public void close() throws Exception {
this.bucketAssigner.close();
}
public void setContext(BucketAssignOperator.Context context) {
this.context = context;
}
@VisibleForTesting
public void clearIndexState() {
this.indexState.clear();
}
}

View File

@@ -1,57 +0,0 @@
/*
* 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);
}
}
}

View File

@@ -35,7 +35,6 @@ 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.RowDataToHoodieFunctions;
import org.apache.hudi.table.format.FilePathUtils;
@@ -44,6 +43,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
import org.apache.flink.streaming.api.operators.ProcessOperator;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -163,7 +163,7 @@ public class Pipelines {
.transform(
"bucket_assigner",
TypeInformation.of(HoodieRecord.class),
new BucketAssignOperator<>(new BucketAssignFunction<>(conf)))
new KeyedProcessOperator<>(new BucketAssignFunction<>(conf)))
.uid("uid_bucket_assigner_" + conf.getString(FlinkOptions.TABLE_NAME))
.setParallelism(conf.getOptional(FlinkOptions.BUCKET_ASSIGN_TASKS).orElse(defaultParallelism))
// shuffle by fileId(bucket id)