garyli1019 commented on a change in pull request #2176: URL: https://github.com/apache/hudi/pull/2176#discussion_r521232451
########## File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java ########## @@ -0,0 +1,49 @@ +/* + * 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.table.upgrade; + +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.table.MarkerFiles; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * Downgrade handle to assist in downgrading hoodie table from version 1 to 0. Review comment: what is the diff between version 1 and 0? ########## File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java ########## @@ -0,0 +1,186 @@ +/* + * 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; + +import com.beust.jcommander.IStringConverter; +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParameterException; +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; +import org.apache.hudi.constant.Operation; +import org.apache.hudi.operator.InstantGenerateOperator; +import org.apache.hudi.operator.KeyedWriteProcessFunction; +import org.apache.hudi.operator.KeyedWriteProcessOperator; +import org.apache.hudi.sink.CommitSink; +import org.apache.hudi.source.JsonStringToHoodieRecordMapFunction; +import org.apache.hudi.util.StreamerUtil; + +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +/** + * An Utility which can incrementally consume data from Kafka and apply it to the target table. + * currently, it only support MOR table and insert, upsert operation. + */ +public class HudiFlinkStreamer { + public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + final Config cfg = new Config(); + JCommander cmd = new JCommander(cfg, null, args); + if (cfg.help || args.length == 0) { + cmd.usage(); + System.exit(1); + } + env.enableCheckpointing(cfg.checkpointInterval); + env.getConfig().setGlobalJobParameters(cfg); + // We use checkpoint to trigger write operation, including instant generating and committing, + // There can only be one checkpoint at one time. + env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); + env.disableOperatorChaining(); + + if (cfg.flinkCheckPointPath != null) { + env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath)); + } + + Properties kafkaProps = StreamerUtil.getKafkaProps(cfg); + + // Read from kafka source + DataStream<HoodieRecord> inputRecords = + env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), kafkaProps)) + .map(new JsonStringToHoodieRecordMapFunction(cfg)) + .name("kafka_to_hudi_record") + .uid("kafka_to_hudi_record_uid"); + + // InstantGenerateOperator helps to emit globally unique instantTime, it must be executed in one parallelism + inputRecords.transform(InstantGenerateOperator.NAME, TypeInformation.of(HoodieRecord.class), new InstantGenerateOperator()) + .name("instant_generator") + .uid("instant_generator_id") + .setParallelism(1) Review comment: would you add a bit more comments here? not quite familiar with Flink but a bit worried about if this will impact the performance ########## File path: hudi-flink-writer/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.java ########## @@ -0,0 +1,79 @@ +/* + * 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.schema; + +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.util.StreamerUtil; + +import java.io.IOException; +import java.util.Collections; + +/** + * A simple schema provider, that reads off files on DFS. + */ +public class FilebasedSchemaProvider extends SchemaProvider { Review comment: we have `TableSchemaResolver` to handle the schema ########## File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java ########## @@ -0,0 +1,135 @@ +/* + * 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.index.state; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.index.FlinkHoodieIndex; +import org.apache.hudi.table.HoodieTable; + +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.List; + +/** + * Hoodie index implementation backed by flink state. + * + * @param <T> type of payload + */ +public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload> extends FlinkHoodieIndex<T> { Review comment: Is this only dedup data in one batch but not upserting into the historical hudi table? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
