[HUDI-1821] Remove legacy code for Flink writer (#2868)
This commit is contained in:
@@ -18,7 +18,6 @@
|
||||
|
||||
package org.apache.hudi.sink;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
@@ -29,7 +28,6 @@ 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.transform.RowDataToHoodieFunction;
|
||||
import org.apache.hudi.streamer.FlinkStreamerConfig;
|
||||
import org.apache.hudi.util.AvroSchemaConverter;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
@@ -39,10 +37,8 @@ import org.apache.hudi.utils.source.ContinuousFileSource;
|
||||
import org.apache.flink.api.common.JobStatus;
|
||||
import org.apache.flink.api.common.io.FilePathFilter;
|
||||
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
|
||||
import org.apache.flink.api.common.typeinfo.TypeHint;
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.api.java.io.TextInputFormat;
|
||||
import org.apache.flink.api.java.tuple.Tuple3;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.core.execution.JobClient;
|
||||
import org.apache.flink.core.fs.Path;
|
||||
@@ -140,75 +136,6 @@ public class StreamWriteITCase extends TestLogger {
|
||||
TestData.checkWrittenFullData(tempFile, EXPECTED);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteToHoodieLegacy() throws Exception {
|
||||
FlinkStreamerConfig streamerConf = TestConfigurations.getDefaultStreamerConf(tempFile.getAbsolutePath());
|
||||
Configuration conf = FlinkOptions.fromStreamerConfig(streamerConf);
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
|
||||
execEnv.getConfig().disableObjectReuse();
|
||||
execEnv.setParallelism(4);
|
||||
// set up checkpoint interval
|
||||
execEnv.enableCheckpointing(4000, CheckpointingMode.EXACTLY_ONCE);
|
||||
execEnv.getConfig().setGlobalJobParameters(streamerConf);
|
||||
|
||||
// Read from file source
|
||||
RowType rowType =
|
||||
(RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf))
|
||||
.getLogicalType();
|
||||
|
||||
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
|
||||
rowType,
|
||||
InternalTypeInfo.of(rowType),
|
||||
false,
|
||||
true,
|
||||
TimestampFormat.ISO_8601
|
||||
);
|
||||
String sourcePath = Objects.requireNonNull(Thread.currentThread()
|
||||
.getContextClassLoader().getResource("test_source.data")).toString();
|
||||
|
||||
execEnv
|
||||
// use continuous file source to trigger checkpoint
|
||||
.addSource(new ContinuousFileSource.BoundedSourceFunction(new Path(sourcePath), 2))
|
||||
.name("continuous_file_source")
|
||||
.setParallelism(1)
|
||||
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)))
|
||||
.setParallelism(4)
|
||||
.map(new RowDataToHoodieFunction<>(rowType, conf), TypeInformation.of(HoodieRecord.class))
|
||||
.transform(InstantGenerateOperator.NAME, TypeInformation.of(HoodieRecord.class), new InstantGenerateOperator())
|
||||
.name("instant_generator")
|
||||
.uid("instant_generator_id")
|
||||
|
||||
// Key-by record key, to avoid multiple subtasks write to a bucket at the same time
|
||||
.keyBy(HoodieRecord::getRecordKey)
|
||||
// use the bucket assigner to generate bucket IDs
|
||||
.transform(
|
||||
"bucket_assigner",
|
||||
TypeInformation.of(HoodieRecord.class),
|
||||
new KeyedProcessOperator<>(new BucketAssignFunction<>(conf)))
|
||||
.uid("uid_bucket_assigner")
|
||||
// shuffle by fileId(bucket id)
|
||||
.keyBy(record -> record.getCurrentLocation().getFileId())
|
||||
// write operator, where the write operation really happens
|
||||
.transform(KeyedWriteProcessOperator.NAME, TypeInformation.of(new TypeHint<Tuple3<String, List<WriteStatus>, Integer>>() {
|
||||
}), new KeyedWriteProcessOperator(new KeyedWriteProcessFunction()))
|
||||
.name("write_process")
|
||||
.uid("write_process_uid")
|
||||
.setParallelism(4)
|
||||
|
||||
// Commit can only be executed once, so make it one parallelism
|
||||
.addSink(new CommitSink())
|
||||
.name("commit_sink")
|
||||
.uid("commit_sink_uid")
|
||||
.setParallelism(1);
|
||||
|
||||
JobClient client = execEnv.executeAsync(execEnv.getStreamGraph(conf.getString(FlinkOptions.TABLE_NAME)));
|
||||
// wait for the streaming job to finish
|
||||
client.getJobExecutionResult().get();
|
||||
|
||||
TestData.checkWrittenFullData(tempFile, EXPECTED);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMergeOnReadWriteWithCompaction() throws Exception {
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
|
||||
@@ -1,89 +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.transform;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
import org.apache.hudi.testutils.HoodieFlinkClientTestHarness;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA;
|
||||
|
||||
public class TestJsonStringToHoodieRecordMapFunction extends HoodieFlinkClientTestHarness {
|
||||
@BeforeEach
|
||||
public void init() {
|
||||
initPath();
|
||||
initTestDataGenerator();
|
||||
initFileSystem();
|
||||
initFlinkMiniCluster();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void clean() throws Exception {
|
||||
cleanupTestDataGenerator();
|
||||
cleanupFileSystem();
|
||||
cleanupFlinkMiniCluster();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMapFunction() throws Exception {
|
||||
final String newCommitTime = "001";
|
||||
final int numRecords = 10;
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, numRecords);
|
||||
List<String> recordStr = RawTripTestPayload.recordsToStrings(records);
|
||||
Schema schema = AVRO_SCHEMA;
|
||||
|
||||
TypedProperties props = new TypedProperties();
|
||||
props.put(HoodieWriteConfig.WRITE_PAYLOAD_CLASS, OverwriteWithLatestAvroPayload.class.getName());
|
||||
props.put(HoodieWriteConfig.PRECOMBINE_FIELD_PROP, "timestamp");
|
||||
props.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key");
|
||||
props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "current_date");
|
||||
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
|
||||
env.setParallelism(1);
|
||||
|
||||
SimpleTestSinkFunction.valuesList.clear();
|
||||
env.fromCollection(recordStr)
|
||||
.map(new JsonStringToHoodieRecordMapFunction(props, Option.of(schema.toString())))
|
||||
.addSink(new SimpleTestSinkFunction());
|
||||
env.execute();
|
||||
|
||||
// input records all present in the sink
|
||||
Assertions.assertEquals(10, SimpleTestSinkFunction.valuesList.size());
|
||||
|
||||
// input keys all present in the sink
|
||||
Set<String> inputKeySet = records.stream().map(r -> r.getKey().getRecordKey()).collect(Collectors.toSet());
|
||||
Assertions.assertEquals(10, SimpleTestSinkFunction.valuesList.stream()
|
||||
.map(r -> inputKeySet.contains(r.getRecordKey())).filter(b -> b).count());
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user