simonsssu commented on a change in pull request #1515:
URL: https://github.com/apache/iceberg/pull/1515#discussion_r502376629



##########
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:
       You are right, it's no need to add this. 

##########
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:
       Yes initializeState will invoke even checkpoint disabled, and 
context.isRestored will be false.

##########
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:
       You are right, I will add more UTs

##########
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:
       I think it's better here to pass the conf If we want to add more 
parameters to control the behaviors, although currently we only need an 
interval here.

##########
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:
       Yes, here I think it's better to use commitInterval rather than 
flushCommitInterval. 




----------------------------------------------------------------
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]

Reply via email to