1
0

[HUDI-1931] BucketAssignFunction use ValueState instead of MapState (#3026)

Co-authored-by: 854194341@qq.com <loukey_7821>
This commit is contained in:
Danny Chan
2021-06-06 10:40:15 +08:00
committed by GitHub
parent 2a7e1e091e
commit 08464a6a5b
8 changed files with 177 additions and 38 deletions

View File

@@ -27,6 +27,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.AvroSchemaConverter;
import org.apache.hudi.util.StreamerUtil;
@@ -121,7 +122,7 @@ public class StreamWriteITCase extends TestLogger {
.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())
@@ -181,7 +182,7 @@ public class StreamWriteITCase extends TestLogger {
.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())

View File

@@ -55,7 +55,7 @@ public class CompactFunctionWrapper {
private final MockFunctionInitializationContext functionInitializationContext;
/** Function that generates the {@link HoodieCompactionPlan}. */
private CompactionPlanOperator compactionPlanFunction;
private CompactionPlanOperator compactionPlanOperator;
/** Function that executes the compaction task. */
private CompactFunction compactFunction;
/** Stream sink to handle compaction commits. */
@@ -74,8 +74,8 @@ public class CompactFunctionWrapper {
}
public void openFunction() throws Exception {
compactionPlanFunction = new CompactionPlanOperator(conf);
compactionPlanFunction.open();
compactionPlanOperator = new CompactionPlanOperator(conf);
compactionPlanOperator.open();
compactFunction = new CompactFunction(conf);
compactFunction.setRuntimeContext(runtimeContext);
@@ -118,8 +118,8 @@ public class CompactFunctionWrapper {
}
};
compactionPlanFunction.setOutput(output);
compactionPlanFunction.notifyCheckpointComplete(checkpointID);
compactionPlanOperator.setOutput(output);
compactionPlanOperator.notifyCheckpointComplete(checkpointID);
// collect the CompactCommitEvents
List<CompactionCommitEvent> compactCommitEvents = new ArrayList<>();
for (CompactionPlanEvent event: events) {

View File

@@ -49,6 +49,7 @@ public class MockOperatorStateStore implements KeyedStateStore, OperatorStateSto
private Map<String, TestUtils.MockListState> lastSuccessStateMap;
private MapState mapState;
private Map<String, ValueState> valueStateMap;
public MockOperatorStateStore() {
this.historyStateMap = new HashMap<>();
@@ -57,6 +58,7 @@ public class MockOperatorStateStore implements KeyedStateStore, OperatorStateSto
this.lastSuccessStateMap = new HashMap<>();
this.mapState = new MockMapState<>();
this.valueStateMap = new HashMap<>();
}
@Override
@@ -66,7 +68,9 @@ public class MockOperatorStateStore implements KeyedStateStore, OperatorStateSto
@Override
public <T> ValueState<T> getState(ValueStateDescriptor<T> valueStateDescriptor) {
return null;
String name = valueStateDescriptor.getName();
valueStateMap.putIfAbsent(name, new MockValueState());
return valueStateMap.get(name);
}
@Override

View File

@@ -0,0 +1,45 @@
/*
* 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.utils;
import org.apache.flink.api.common.state.ValueState;
/**
* Mock value state for testing.
*
* @param <V> Type of state value
*/
public class MockValueState<V> implements ValueState<V> {
private V v = null;
@Override
public V value() {
return v;
}
@Override
public void update(V value) {
this.v = value;
}
@Override
public void clear() {
v = null;
}
}

View File

@@ -27,6 +27,7 @@ import org.apache.hudi.sink.StreamWriteFunction;
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
import org.apache.hudi.sink.event.BatchWriteSuccessEvent;
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.utils.TestConfigurations;
@@ -44,8 +45,10 @@ import org.apache.flink.streaming.api.operators.collect.utils.MockOperatorEventG
import org.apache.flink.table.data.RowData;
import org.apache.flink.util.Collector;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
/**
@@ -67,6 +70,8 @@ public class StreamWriteFunctionWrapper<I> {
private RowDataToHoodieFunction<RowData, HoodieRecord<?>> toHoodieFunction;
/** Function that assigns bucket ID. */
private BucketAssignFunction<String, HoodieRecord<?>, HoodieRecord<?>> bucketAssignerFunction;
/** BucketAssignOperator context. **/
private MockBucketAssignOperatorContext bucketAssignOperatorContext;
/** Stream write function. */
private StreamWriteFunction<Object, HoodieRecord<?>, Object> writeFunction;
@@ -91,6 +96,7 @@ public class StreamWriteFunctionWrapper<I> {
this.coordinator = new StreamWriteOperatorCoordinator(conf, this.coordinatorContext);
this.functionInitializationContext = new MockFunctionInitializationContext();
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
this.bucketAssignOperatorContext = new MockBucketAssignOperatorContext();
}
public void openFunction() throws Exception {
@@ -103,6 +109,7 @@ public class StreamWriteFunctionWrapper<I> {
bucketAssignerFunction = new BucketAssignFunction<>(conf);
bucketAssignerFunction.setRuntimeContext(runtimeContext);
bucketAssignerFunction.open(conf);
bucketAssignerFunction.setContext(bucketAssignOperatorContext);
bucketAssignerFunction.initializeState(this.functionInitializationContext);
writeFunction = new StreamWriteFunction<>(conf);
@@ -150,7 +157,7 @@ public class StreamWriteFunctionWrapper<I> {
return this.writeFunction.getWriteClient();
}
public void checkpointFunction(long checkpointId) throws Exception {
public void checkpointFunction(long checkpointId) {
// checkpoint the coordinator first
this.coordinator.checkpointCoordinator(checkpointId, new CompletableFuture<>());
bucketAssignerFunction.snapshotState(null);
@@ -192,13 +199,35 @@ public class StreamWriteFunctionWrapper<I> {
public void clearIndexState() {
this.bucketAssignerFunction.clearIndexState();
this.bucketAssignOperatorContext.clearIndexState();
}
public boolean isKeyInState(HoodieKey hoodieKey) {
return this.bucketAssignerFunction.isKeyInState(hoodieKey);
return this.bucketAssignOperatorContext.isKeyInState(hoodieKey.getRecordKey());
}
public boolean isConforming() {
return this.writeFunction.isConfirming();
}
// -------------------------------------------------------------------------
// Inner Class
// -------------------------------------------------------------------------
private static class MockBucketAssignOperatorContext implements BucketAssignOperator.Context {
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);
}
}
}