stevenzwu commented on code in PR #8553: URL: https://github.com/apache/iceberg/pull/8553#discussion_r1394590199
########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/TimestampBasedWatermarkExtractor.java: ########## @@ -0,0 +1,63 @@ +/* + * 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.source.reader; + +import java.io.Serializable; +import java.util.Comparator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +/** + * {@link IcebergWatermarkExtractor} implementation which uses an Iceberg timestamp column + * statistics to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by + * the {@link WatermarkExtractorRecordEmitter} along with the actual records. + */ Review Comment: nit: add `@Internal` ########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/TimestampBasedWatermarkExtractor.java: ########## @@ -0,0 +1,63 @@ +/* + * 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.source.reader; + +import java.io.Serializable; +import java.util.Comparator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +/** + * {@link IcebergWatermarkExtractor} implementation which uses an Iceberg timestamp column + * statistics to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by + * the {@link WatermarkExtractorRecordEmitter} along with the actual records. + */ +public class TimestampBasedWatermarkExtractor implements IcebergWatermarkExtractor, Serializable { Review Comment: nit: `TimestampBasedWatermarkExtractor` -> `ColumnStatsWatermarkExtractor`? ########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergWatermarkExtractor.java: ########## @@ -18,19 +18,11 @@ */ package org.apache.iceberg.flink.source.reader; -import org.apache.flink.api.connector.source.SourceOutput; -import org.apache.flink.connector.base.source.reader.RecordEmitter; +import java.io.Serializable; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -final class IcebergSourceRecordEmitter<T> - implements RecordEmitter<RecordAndPosition<T>, T, IcebergSourceSplit> { - - IcebergSourceRecordEmitter() {} - - @Override - public void emitRecord( - RecordAndPosition<T> element, SourceOutput<T> output, IcebergSourceSplit split) { - output.collect(element.record()); - split.updatePosition(element.fileOffset(), element.recordOffset()); - } +/** The interface used to extract watermarks from splits. */ +public interface IcebergWatermarkExtractor extends Serializable { + /** Get the watermark for a split. */ Review Comment: nit: should we add throws to javadoc? ``` @throws IllegalArgumentException if split doesn't contain column stats of min and max values for timestamp column ``` ########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java: ########## @@ -0,0 +1,65 @@ +/* + * 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.source.reader; + +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Emitter which emits the watermarks, records and updates the split position. + * + * <p>The Emitter emits watermarks at the beginning of every split provided by the {@link + * IcebergWatermarkExtractor}. + */ +class WatermarkExtractorRecordEmitter<T> implements SerializableRecordEmitter<T> { + private static final Logger LOG = LoggerFactory.getLogger(WatermarkExtractorRecordEmitter.class); + private final IcebergWatermarkExtractor timeExtractor; + private String lastSplit = null; Review Comment: nit: `lastSplitId` ########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java: ########## @@ -0,0 +1,65 @@ +/* + * 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.source.reader; + +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Emitter which emits the watermarks, records and updates the split position. + * + * <p>The Emitter emits watermarks at the beginning of every split provided by the {@link + * IcebergWatermarkExtractor}. + */ +class WatermarkExtractorRecordEmitter<T> implements SerializableRecordEmitter<T> { + private static final Logger LOG = LoggerFactory.getLogger(WatermarkExtractorRecordEmitter.class); + private final IcebergWatermarkExtractor timeExtractor; + private String lastSplit = null; + private long watermark; + + WatermarkExtractorRecordEmitter(IcebergWatermarkExtractor timeExtractor) { + this.timeExtractor = timeExtractor; + } + + @Override + public void emitRecord( + RecordAndPosition<T> element, SourceOutput<T> output, IcebergSourceSplit split) { + if (!split.splitId().equals(lastSplit)) { + long newWatermark = timeExtractor.extractWatermark(split); + if (newWatermark < watermark) { + LOG.info( + "Watermark decreased. PreviousWM {}, currentWM {}, previousSplit {}, currentSplit {}.", Review Comment: for the deceased case, should it be a warning? we use ordered assigner, normally, we shouldn't experience it. I can think of one scenario though. the next commit included files/splits with older timestamp. I guess `info` is probably fine. nit on log msg: `previous watermark = {}, current watermark = {}, previous split = {}, current split = {}` ########## flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java: ########## @@ -97,6 +98,11 @@ protected List<Record> generateRecords(int numRecords, long seed) { return RandomGenericData.generate(schema(), numRecords, seed); } + protected void assertRecords(Table table, List<Record> expectedRecords, Duration timeout) Review Comment: nit: is this helper necessary? ########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/TimestampBasedWatermarkExtractor.java: ########## @@ -0,0 +1,63 @@ +/* + * 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.source.reader; + +import java.io.Serializable; +import java.util.Comparator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +/** + * {@link IcebergWatermarkExtractor} implementation which uses an Iceberg timestamp column + * statistics to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by + * the {@link WatermarkExtractorRecordEmitter} along with the actual records. + */ +public class TimestampBasedWatermarkExtractor implements IcebergWatermarkExtractor, Serializable { + private final int tsFieldId; + + /** + * Creates the extractor. + * + * @param schema The schema of the Table + * @param tsFieldName The timestamp column which should be used as an event time + */ + public TimestampBasedWatermarkExtractor(Schema schema, String tsFieldName) { + Types.NestedField field = schema.findField(tsFieldName); + Preconditions.checkArgument( + field.type().typeId().equals(Type.TypeID.TIMESTAMP), "Type should be timestamp"); Review Comment: should we also support plain `long` type field? e.g., a lot of data stream uses milli-seconds `long` value to capture timestamp. Flink watermark is also in milli-seconds. that may require adding a 2nd arg of precision: `IcebergSource#watermarkColumn(String name, TimeUnit precision)` ########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java: ########## @@ -0,0 +1,65 @@ +/* + * 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.source.reader; + +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Emitter which emits the watermarks, records and updates the split position. + * + * <p>The Emitter emits watermarks at the beginning of every split provided by the {@link + * IcebergWatermarkExtractor}. + */ +class WatermarkExtractorRecordEmitter<T> implements SerializableRecordEmitter<T> { + private static final Logger LOG = LoggerFactory.getLogger(WatermarkExtractorRecordEmitter.class); + private final IcebergWatermarkExtractor timeExtractor; + private String lastSplit = null; + private long watermark; + + WatermarkExtractorRecordEmitter(IcebergWatermarkExtractor timeExtractor) { + this.timeExtractor = timeExtractor; + } + + @Override + public void emitRecord( + RecordAndPosition<T> element, SourceOutput<T> output, IcebergSourceSplit split) { + if (!split.splitId().equals(lastSplit)) { + long newWatermark = timeExtractor.extractWatermark(split); + if (newWatermark < watermark) { Review Comment: we can log a DEBUG level message for advancing watermark. ########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/TimestampBasedWatermarkExtractor.java: ########## @@ -0,0 +1,63 @@ +/* + * 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.source.reader; + +import java.io.Serializable; +import java.util.Comparator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +/** + * {@link IcebergWatermarkExtractor} implementation which uses an Iceberg timestamp column + * statistics to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by + * the {@link WatermarkExtractorRecordEmitter} along with the actual records. + */ +public class TimestampBasedWatermarkExtractor implements IcebergWatermarkExtractor, Serializable { + private final int tsFieldId; + + /** + * Creates the extractor. + * + * @param schema The schema of the Table + * @param tsFieldName The timestamp column which should be used as an event time + */ + public TimestampBasedWatermarkExtractor(Schema schema, String tsFieldName) { + Types.NestedField field = schema.findField(tsFieldName); + Preconditions.checkArgument( + field.type().typeId().equals(Type.TypeID.TIMESTAMP), "Type should be timestamp"); + this.tsFieldId = field.fieldId(); + } + + @Override + public long extractWatermark(IcebergSourceSplit split) { + return split.task().files().stream() + .map( + scanTask -> + (long) + Conversions.fromByteBuffer( + Types.LongType.get(), scanTask.file().lowerBounds().get(tsFieldId)) Review Comment: should check if lowerBounds have stats and throws IllegalArgumentException if not. ########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergWatermarkExtractor.java: ########## @@ -18,19 +18,11 @@ */ package org.apache.iceberg.flink.source.reader; -import org.apache.flink.api.connector.source.SourceOutput; -import org.apache.flink.connector.base.source.reader.RecordEmitter; +import java.io.Serializable; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -final class IcebergSourceRecordEmitter<T> - implements RecordEmitter<RecordAndPosition<T>, T, IcebergSourceSplit> { - - IcebergSourceRecordEmitter() {} - - @Override - public void emitRecord( - RecordAndPosition<T> element, SourceOutput<T> output, IcebergSourceSplit split) { - output.collect(element.record()); - split.updatePosition(element.fileOffset(), element.recordOffset()); - } +/** The interface used to extract watermarks from splits. */ +public interface IcebergWatermarkExtractor extends Serializable { Review Comment: can we call it `SplitWatermarkExtractor`? ########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java: ########## @@ -56,4 +57,20 @@ public static SerializableComparator<IcebergSourceSplit> fileSequenceNumber() { } }; } + + /** Comparator which orders the splits based on watermark of the splits */ + public static SerializableComparator<IcebergSourceSplit> watermarksAwareComparator( Review Comment: nit on naming. to be consistent with `fileSequenceNumber` comparator above, maybe this could be just `watermark`? ########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/TimestampBasedWatermarkExtractor.java: ########## @@ -0,0 +1,63 @@ +/* + * 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.source.reader; + +import java.io.Serializable; +import java.util.Comparator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +/** + * {@link IcebergWatermarkExtractor} implementation which uses an Iceberg timestamp column + * statistics to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by + * the {@link WatermarkExtractorRecordEmitter} along with the actual records. + */ +public class TimestampBasedWatermarkExtractor implements IcebergWatermarkExtractor, Serializable { + private final int tsFieldId; + + /** + * Creates the extractor. + * + * @param schema The schema of the Table + * @param tsFieldName The timestamp column which should be used as an event time + */ + public TimestampBasedWatermarkExtractor(Schema schema, String tsFieldName) { + Types.NestedField field = schema.findField(tsFieldName); + Preconditions.checkArgument( + field.type().typeId().equals(Type.TypeID.TIMESTAMP), "Type should be timestamp"); + this.tsFieldId = field.fieldId(); + } + + @Override + public long extractWatermark(IcebergSourceSplit split) { + return split.task().files().stream() + .map( + scanTask -> + (long) + Conversions.fromByteBuffer( + Types.LongType.get(), scanTask.file().lowerBounds().get(tsFieldId)) Review Comment: this also reminds me one thing. if watermark alignment is enabled, we should probably override the `read.split.open-file-cost` to some huge value (like `Long.MAX_VALUE` to force one file per split. -- 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: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org