openinx commented on a change in pull request #1515: URL: https://github.com/apache/iceberg/pull/1515#discussion_r502301819
########## File path: flink/src/test/java/org/apache/iceberg/flink/NonCheckpointFiniteTestSource.java ########## @@ -0,0 +1,50 @@ +/* + * 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.iceberg.flink; + +import java.util.Arrays; +import java.util.List; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; + +public class NonCheckpointFiniteTestSource<T> implements Review comment: nit: we don't need to break this line into two lines ? ########## File path: flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java ########## @@ -57,6 +66,14 @@ public void open() { // Initialize the task writer. this.writer = taskWriterFactory.create(); + + // If we don't enable checkpoint, we will use processingTimeSerice to emit datafiles to downstream, Review comment: nit: this sentence ends with ',' ? incomplete sentence or change to '.' ? ########## File path: flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java ########## @@ -73,7 +75,9 @@ // TableLoader to load iceberg table lazily. private final TableLoader tableLoader; private final SerializableConfiguration hadoopConf; + private final org.apache.flink.configuration.Configuration flinkConf; private final boolean replacePartitions; + private final long flushCommitInterval; Review comment: It sounds a confusing name ? For StreamingFileSink, we use the rollingCheckInterval. ```java public static final ConfigOption<Duration> SINK_ROLLING_POLICY_CHECK_INTERVAL = key("sink.rolling-policy.check-interval") .durationType() .defaultValue(Duration.ofMinutes(1)) .withDescription("The interval for checking time based rolling policies. " + "This controls the frequency to check whether a part file should rollover based on 'sink.rolling-policy.rollover-interval'."); ``` ########## File path: flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java ########## @@ -528,7 +528,8 @@ private static TestOperatorFactory of(String tablePath) { @Override @SuppressWarnings("unchecked") public <T extends StreamOperator<Void>> T createStreamOperator(StreamOperatorParameters<Void> param) { - IcebergFilesCommitter committer = new IcebergFilesCommitter(TableLoader.fromHadoopTable(tablePath), CONF, false); + IcebergFilesCommitter committer = new IcebergFilesCommitter(TableLoader.fromHadoopTable(tablePath), CONF, false, Review comment: We will need more unit tests in this operator test, addressing that the `IcebergFilesCommitter` will work as expected exactly. ########## File path: flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java ########## @@ -317,7 +317,8 @@ public void testPromotedFlinkDataType() throws Exception { private OneInputStreamOperatorTestHarness<RowData, DataFile> createIcebergStreamWriter( Table icebergTable, TableSchema flinkSchema) throws Exception { - IcebergStreamWriter<RowData> streamWriter = FlinkSink.createStreamWriter(icebergTable, flinkSchema); + IcebergStreamWriter<RowData> streamWriter = FlinkSink.createStreamWriter(icebergTable, flinkSchema, Review comment: ditto. ########## File path: flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java ########## @@ -103,10 +107,26 @@ private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor(); private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState; - IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf, boolean replacePartitions) { + IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf, boolean replacePartitions, + org.apache.flink.configuration.Configuration flinkConf) { this.tableLoader = tableLoader; this.hadoopConf = new SerializableConfiguration(hadoopConf); this.replacePartitions = replacePartitions; + this.flinkConf = flinkConf; + this.flushCommitInterval = flinkConf.getLong(FlinkSink.FLINK_ICEBERG_SINK_FLUSHINTERVAL, + FlinkSink.DEFAULT_FLINK_ICEBERG_SINK_FLUSHINTERVAL); + } + + @Override + public void open() throws Exception { + super.open(); + boolean isCheckpointEnabled = getRuntimeContext().isCheckpointingEnabled(); + // If we don't enable checkpoint, we will use processingTimeSerice to do commit, Review comment: nit: `don't enable` -> disable ########## File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java ########## @@ -153,6 +158,11 @@ public Builder hadoopConf(Configuration newHadoopConf) { return this; } + public Builder flinkConf(org.apache.flink.configuration.Configuration config) { + this.flinkConf = config != null ? config : new org.apache.flink.configuration.Configuration(); + return this; + } Review comment: Do we need to pass the whole flink configuration to `IcebergStreamWriter` and `IcebergFilesCommitter` ? Here we only need the `flushInterval` to emit the data files to downstream , how about just pass the interval value ? and we could have a default value inside this builder. ########## File path: flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java ########## @@ -145,6 +147,38 @@ public void testWriteRowData() throws Exception { SimpleDataUtil.assertTableRows(tablePath, expectedRows); } + @Test + public void testWriteRowDataWithoutCheckpoint() throws Exception { + List<Row> rows = Lists.newArrayList( + Row.of(1, "hello"), + Row.of(2, "world"), + Row.of(3, "foo") + ); + + env = StreamExecutionEnvironment.getExecutionEnvironment() + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + DataStream<RowData> dataStream = env.addSource(new NonCheckpointFiniteTestSource<>(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, RowDataTypeInfo.of(SimpleDataUtil.ROW_TYPE)); + + org.apache.flink.configuration.Configuration flinkConf = new org.apache.flink.configuration.Configuration(); + flinkConf.setLong(FlinkSink.FLINK_ICEBERG_SINK_FLUSHINTERVAL, 100L); + + FlinkSink.forRowData(dataStream) + .table(table) Review comment: nit: We don't have to set the `table` parameter when directing the intput data stream into iceberg table, because it will load the table automatically if table is null. ########## File path: flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java ########## @@ -103,10 +107,26 @@ private static final ListStateDescriptor<SortedMap<Long, List<DataFile>>> STATE_DESCRIPTOR = buildStateDescriptor(); private transient ListState<SortedMap<Long, List<DataFile>>> checkpointsState; - IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf, boolean replacePartitions) { + IcebergFilesCommitter(TableLoader tableLoader, Configuration hadoopConf, boolean replacePartitions, + org.apache.flink.configuration.Configuration flinkConf) { this.tableLoader = tableLoader; this.hadoopConf = new SerializableConfiguration(hadoopConf); this.replacePartitions = replacePartitions; + this.flinkConf = flinkConf; + this.flushCommitInterval = flinkConf.getLong(FlinkSink.FLINK_ICEBERG_SINK_FLUSHINTERVAL, + FlinkSink.DEFAULT_FLINK_ICEBERG_SINK_FLUSHINTERVAL); + } + + @Override + public void open() throws Exception { + super.open(); + boolean isCheckpointEnabled = getRuntimeContext().isCheckpointingEnabled(); + // If we don't enable checkpoint, we will use processingTimeSerice to do commit, + if (!isCheckpointEnabled) { + ProcessingTimeService processingTimeService = getRuntimeContext().getProcessingTimeService(); + final long currentTimestamp = processingTimeService.getCurrentProcessingTime(); + processingTimeService.registerTimer(currentTimestamp + flushCommitInterval, this); + } } @Override Review comment: We will invoke the `initializeState` method even if disabled the flink checkpoint ? If not, then many variables (such as table , maxCommittedCheckpointId) won't be initialized, that would introduce unexpected bug. ########## File path: flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java ########## @@ -528,7 +528,8 @@ private static TestOperatorFactory of(String tablePath) { @Override @SuppressWarnings("unchecked") public <T extends StreamOperator<Void>> T createStreamOperator(StreamOperatorParameters<Void> param) { - IcebergFilesCommitter committer = new IcebergFilesCommitter(TableLoader.fromHadoopTable(tablePath), CONF, false); + IcebergFilesCommitter committer = new IcebergFilesCommitter(TableLoader.fromHadoopTable(tablePath), CONF, false, Review comment: BTW, do we need sql test for the disabled checkpoint case ? ---------------------------------------------------------------- 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] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
