fapaul commented on a change in pull request #16796:
URL: https://github.com/apache/flink/pull/16796#discussion_r688448306



##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.flink.streaming.connectors.kafka.table;
+
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.RowKind;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+import static 
org.apache.flink.streaming.connectors.kafka.table.DynamicKafkaRecordSerializationSchema.createProjectedRow;
+import static org.apache.flink.types.RowKind.DELETE;
+import static org.apache.flink.types.RowKind.UPDATE_AFTER;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class ReducingUpsertWriter<WriterState> implements SinkWriter<RowData, Void, 
WriterState> {
+
+    private final SinkWriter<RowData, ?, WriterState> wrappedWriter;
+    private final WrappedContext wrappedContext = new WrappedContext();
+    private final int batchMaxRowNums;
+    private final Function<RowData, RowData> valueCopyFunction;
+    private final Map<RowData, Tuple2<RowData, Long>> reduceBuffer = new 
HashMap<>();
+    private final Function<RowData, RowData> keyExtractor;
+    private final Sink.ProcessingTimeService timeService;
+    private final long batchIntervalMs;
+
+    private boolean closed = false;
+    private int batchCount = 0;
+
+    ReducingUpsertWriter(
+            SinkWriter<RowData, ?, WriterState> wrappedWriter,
+            DataType physicalDataType,
+            int[] keyProjection,
+            SinkBufferFlushMode bufferFlushMode,
+            Sink.ProcessingTimeService timeService,
+            Function<RowData, RowData> valueCopyFunction) {
+        checkArgument(bufferFlushMode != null && bufferFlushMode.isEnabled());
+        this.wrappedWriter = checkNotNull(wrappedWriter);
+        this.timeService = checkNotNull(timeService);
+        registerFlush();
+        this.batchMaxRowNums = bufferFlushMode.getBatchSize();
+        this.batchIntervalMs = bufferFlushMode.getBatchIntervalMs();
+
+        List<LogicalType> fields = 
physicalDataType.getLogicalType().getChildren();
+        final RowData.FieldGetter[] keyFieldGetters =
+                Arrays.stream(keyProjection)
+                        .mapToObj(
+                                targetField ->
+                                        RowData.createFieldGetter(
+                                                fields.get(targetField), 
targetField))
+                        .toArray(RowData.FieldGetter[]::new);
+        this.keyExtractor = rowData -> createProjectedRow(rowData, 
RowKind.INSERT, keyFieldGetters);
+        this.valueCopyFunction = valueCopyFunction;
+    }
+
+    @Override
+    public void write(RowData element, Context context) throws IOException, 
InterruptedException {
+        wrappedContext.setContext(context);
+        addToBuffer(element, context.timestamp());
+    }
+
+    @Override
+    public List<Void> prepareCommit(boolean flush) throws IOException, 
InterruptedException {
+        flush();
+        return Collections.emptyList();
+    }
+
+    @Override
+    public List<WriterState> snapshotState() throws IOException {
+        return wrappedWriter.snapshotState();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (!closed) {
+            closed = true;
+            wrappedWriter.close();
+        }
+    }
+
+    private void addToBuffer(RowData row, Long timestamp) throws IOException, 
InterruptedException {
+        RowData key = keyExtractor.apply(row);
+        RowData value = valueCopyFunction.apply(row);
+        reduceBuffer.put(key, new Tuple2<>(changeFlag(value), timestamp));
+        batchCount++;
+
+        if (batchCount >= batchMaxRowNums) {
+            flush();
+        }
+    }
+
+    private void registerFlush() {
+        if (closed) {
+            return;
+        }
+        timeService.registerProcessingTimer(
+                System.currentTimeMillis() + batchIntervalMs, (t) -> flush());

Review comment:
       > Currently, the timer is not reset on `flush`. Is this also the old 
behavior?
   
   Yes, the timer was also not reset in the old implementation. I've adjusted 
the new version to the only flush if the last flush was before the flush 
interval.
   
   > To reset the timer, you'd `registerFlush` on `flush` (which you forgot to 
do here as far as I can see) and also remember the the time at which it should 
trigger. Then you'd compare `t` against the time and ignore if it's smaller.
   
   I now call `registerFlush` after every flushing in the processingTimer.
   
   

##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.flink.streaming.connectors.kafka.table;
+
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.RowKind;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+import static 
org.apache.flink.streaming.connectors.kafka.table.DynamicKafkaRecordSerializationSchema.createProjectedRow;
+import static org.apache.flink.types.RowKind.DELETE;
+import static org.apache.flink.types.RowKind.UPDATE_AFTER;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class ReducingUpsertWriter<WriterState> implements SinkWriter<RowData, Void, 
WriterState> {
+
+    private final SinkWriter<RowData, ?, WriterState> wrappedWriter;
+    private final WrappedContext wrappedContext = new WrappedContext();
+    private final int batchMaxRowNums;
+    private final Function<RowData, RowData> valueCopyFunction;
+    private final Map<RowData, Tuple2<RowData, Long>> reduceBuffer = new 
HashMap<>();
+    private final Function<RowData, RowData> keyExtractor;
+    private final Sink.ProcessingTimeService timeService;
+    private final long batchIntervalMs;
+
+    private boolean closed = false;
+    private int batchCount = 0;
+
+    ReducingUpsertWriter(
+            SinkWriter<RowData, ?, WriterState> wrappedWriter,
+            DataType physicalDataType,
+            int[] keyProjection,
+            SinkBufferFlushMode bufferFlushMode,
+            Sink.ProcessingTimeService timeService,
+            Function<RowData, RowData> valueCopyFunction) {
+        checkArgument(bufferFlushMode != null && bufferFlushMode.isEnabled());
+        this.wrappedWriter = checkNotNull(wrappedWriter);
+        this.timeService = checkNotNull(timeService);
+        registerFlush();
+        this.batchMaxRowNums = bufferFlushMode.getBatchSize();
+        this.batchIntervalMs = bufferFlushMode.getBatchIntervalMs();
+
+        List<LogicalType> fields = 
physicalDataType.getLogicalType().getChildren();
+        final RowData.FieldGetter[] keyFieldGetters =
+                Arrays.stream(keyProjection)
+                        .mapToObj(
+                                targetField ->
+                                        RowData.createFieldGetter(
+                                                fields.get(targetField), 
targetField))
+                        .toArray(RowData.FieldGetter[]::new);
+        this.keyExtractor = rowData -> createProjectedRow(rowData, 
RowKind.INSERT, keyFieldGetters);
+        this.valueCopyFunction = valueCopyFunction;
+    }
+
+    @Override
+    public void write(RowData element, Context context) throws IOException, 
InterruptedException {
+        wrappedContext.setContext(context);
+        addToBuffer(element, context.timestamp());
+    }
+
+    @Override
+    public List<Void> prepareCommit(boolean flush) throws IOException, 
InterruptedException {
+        flush();
+        return Collections.emptyList();
+    }
+
+    @Override
+    public List<WriterState> snapshotState() throws IOException {
+        return wrappedWriter.snapshotState();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (!closed) {
+            closed = true;
+            wrappedWriter.close();
+        }
+    }
+
+    private void addToBuffer(RowData row, Long timestamp) throws IOException, 
InterruptedException {
+        RowData key = keyExtractor.apply(row);
+        RowData value = valueCopyFunction.apply(row);
+        reduceBuffer.put(key, new Tuple2<>(changeFlag(value), timestamp));
+        batchCount++;
+
+        if (batchCount >= batchMaxRowNums) {
+            flush();
+        }
+    }
+
+    private void registerFlush() {
+        if (closed) {
+            return;
+        }
+        timeService.registerProcessingTimer(
+                System.currentTimeMillis() + batchIntervalMs, (t) -> flush());

Review comment:
       > Currently, the timer is not reset on `flush`. Is this also the old 
behavior?
   
   Yes, the timer was also not reset in the old implementation. I've adjusted 
the new version to the only flush if the last flush was before the flush 
interval.
   
   > To reset the timer, you'd `registerFlush` on `flush` (which you forgot to 
do here as far as I can see) and also remember the time at which it should 
trigger. Then you'd compare `t` against the time and ignore if it's smaller.
   
   I now call `registerFlush` after every flushing in the processingTimer.
   
   




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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to