[HUDI-2738] Remove the bucketAssignFunction useless context (#3972)
Co-authored-by: yuzhaojing <yuzhaojing@bytedance.com>
This commit is contained in:
@@ -27,7 +27,6 @@ import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
|
||||
import org.apache.hudi.sink.bootstrap.BootstrapOperator;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
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;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
@@ -91,7 +90,7 @@ public class StreamWriteFunctionWrapper<I> implements TestFunctionWrapper<I> {
|
||||
/**
|
||||
* BucketAssignOperator context.
|
||||
**/
|
||||
private MockBucketAssignOperatorContext bucketAssignOperatorContext;
|
||||
private final MockBucketAssignFunctionContext bucketAssignFunctionContext;
|
||||
/**
|
||||
* Stream write function.
|
||||
*/
|
||||
@@ -125,11 +124,10 @@ public class StreamWriteFunctionWrapper<I> implements TestFunctionWrapper<I> {
|
||||
this.coordinatorContext = new MockOperatorCoordinatorContext(new OperatorID(), 1);
|
||||
this.coordinator = new StreamWriteOperatorCoordinator(conf, this.coordinatorContext);
|
||||
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
|
||||
this.bucketAssignOperatorContext = new MockBucketAssignOperatorContext();
|
||||
this.bucketAssignFunctionContext = new MockBucketAssignFunctionContext();
|
||||
this.stateInitializationContext = new MockStateInitializationContext();
|
||||
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
|
||||
this.asyncCompaction = StreamerUtil.needsAsyncCompaction(conf);
|
||||
this.bucketAssignOperatorContext = new MockBucketAssignOperatorContext();
|
||||
this.output = new CollectorOutput<>(new ArrayList<>());
|
||||
this.streamConfig = new StreamConfig(conf);
|
||||
streamConfig.setOperatorID(new OperatorID());
|
||||
@@ -155,7 +153,6 @@ public class StreamWriteFunctionWrapper<I> implements TestFunctionWrapper<I> {
|
||||
bucketAssignerFunction = new BucketAssignFunction<>(conf);
|
||||
bucketAssignerFunction.setRuntimeContext(runtimeContext);
|
||||
bucketAssignerFunction.open(conf);
|
||||
bucketAssignerFunction.setContext(bucketAssignOperatorContext);
|
||||
bucketAssignerFunction.initializeState(this.stateInitializationContext);
|
||||
|
||||
setupWriteFunction();
|
||||
@@ -187,15 +184,16 @@ public class StreamWriteFunctionWrapper<I> implements TestFunctionWrapper<I> {
|
||||
if (streamElement.isRecord()) {
|
||||
HoodieRecord<?> bootstrapRecord = (HoodieRecord<?>) streamElement.asRecord().getValue();
|
||||
bucketAssignerFunction.processElement(bootstrapRecord, null, collector);
|
||||
bucketAssignFunctionContext.setCurrentKey(bootstrapRecord.getRecordKey());
|
||||
}
|
||||
}
|
||||
|
||||
bootstrapOperator.processElement(new StreamRecord<>(hoodieRecord));
|
||||
list.clear();
|
||||
this.bucketAssignOperatorContext.setCurrentKey(hoodieRecord.getRecordKey());
|
||||
}
|
||||
|
||||
bucketAssignerFunction.processElement(hoodieRecord, null, collector);
|
||||
bucketAssignFunctionContext.setCurrentKey(hoodieRecord.getRecordKey());
|
||||
writeFunction.processElement(hoodieRecords[0], null, null);
|
||||
}
|
||||
|
||||
@@ -267,13 +265,8 @@ public class StreamWriteFunctionWrapper<I> implements TestFunctionWrapper<I> {
|
||||
return coordinatorContext;
|
||||
}
|
||||
|
||||
public void clearIndexState() {
|
||||
this.bucketAssignerFunction.clearIndexState();
|
||||
this.bucketAssignOperatorContext.clearIndexState();
|
||||
}
|
||||
|
||||
public boolean isKeyInState(HoodieKey hoodieKey) {
|
||||
return this.bucketAssignOperatorContext.isKeyInState(hoodieKey.getRecordKey());
|
||||
return this.bucketAssignFunctionContext.isKeyInState(hoodieKey.getRecordKey());
|
||||
}
|
||||
|
||||
public boolean isConforming() {
|
||||
@@ -303,18 +296,13 @@ public class StreamWriteFunctionWrapper<I> implements TestFunctionWrapper<I> {
|
||||
// Inner Class
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
private static class MockBucketAssignOperatorContext implements BucketAssignOperator.Context {
|
||||
private static class MockBucketAssignFunctionContext {
|
||||
private final Set<Object> updateKeys = new HashSet<>();
|
||||
|
||||
@Override
|
||||
public void setCurrentKey(Object key) {
|
||||
this.updateKeys.add(key);
|
||||
}
|
||||
|
||||
public void clearIndexState() {
|
||||
this.updateKeys.clear();
|
||||
}
|
||||
|
||||
public boolean isKeyInState(String key) {
|
||||
return this.updateKeys.contains(key);
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user