walterddr commented on code in PR #10045:
URL: https://github.com/apache/pinot/pull/10045#discussion_r1062700111
##########
pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/PinotSinkFunction.java:
##########
@@ -117,31 +123,22 @@ public void close()
@Override
public void invoke(T value, Context context)
throws Exception {
- _segmentWriter.collect(_recordConverter.convertToRow(value));
+ GenericRow row = _recordConverter.convertToRow(value);
+ _pendingRows.add(row);
+ _segmentWriter.collect(row);
_segmentNumRecord++;
- if (_segmentNumRecord > _segmentFlushMaxNumRecords) {
+ if (_segmentNumRecord >= _segmentFlushMaxNumRecords) {
flush();
}
}
- @Override
- public void snapshotState(FunctionSnapshotContext functionSnapshotContext)
- throws Exception {
- throw new UnsupportedOperationException("snapshotState is invoked in Pinot
sink");
- }
-
- @Override
- public void initializeState(FunctionInitializationContext
functionInitializationContext)
- throws Exception {
- // no initialization needed
- // ...
- }
-
private void flush()
throws Exception {
URI segmentURI = _segmentWriter.flush();
LOG.info("Pinot segment writer flushed with {} records to {}",
_segmentNumRecord, segmentURI);
_segmentNumRecord = 0;
+ _pendingRows.clear();
Review Comment:
should clear after the upload was successful?
##########
pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/PinotSinkFunction.java:
##########
@@ -151,4 +148,18 @@ private void flush()
LOG.info("Pinot segment uploaded to {}", segmentURI);
});
}
+
+ @Override
+ public List<GenericRow> snapshotState(long checkpointId, long timestamp) {
+ return _pendingRows;
+ }
+
+ @Override
+ public void restoreState(List<GenericRow> pendingRows)
+ throws Exception {
+ for (GenericRow row: pendingRows) {
+ _segmentWriter.collect(row);
+ _segmentNumRecord++;
+ }
+ }
Review Comment:
need to put pendingRows back into _pendingRows. otherwise in the case of
immediate failure after restore, and a snapshot is triggered. those data a
lost.
##########
pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/sink/PinotSinkFunction.java:
##########
@@ -117,31 +123,22 @@ public void close()
@Override
public void invoke(T value, Context context)
throws Exception {
- _segmentWriter.collect(_recordConverter.convertToRow(value));
+ GenericRow row = _recordConverter.convertToRow(value);
+ _pendingRows.add(row);
+ _segmentWriter.collect(row);
Review Comment:
would it be easier if we just checkpoint the segmentWriter?
--
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]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]